def transform( self, func: Callable[[Column], Column]) -> Union["ps.Series", "ps.Index"]: """ Applies a function that takes and returns a Spark column. It allows to natively apply a Spark function and column APIs with the Spark column internally used in Series or Index. The output length of the Spark column should be same as input's. .. note:: It requires to have the same input and output length; therefore, the aggregate Spark functions such as count does not work. Parameters ---------- func : function Function to use for transforming the data by using Spark columns. Returns ------- Series or Index Raises ------ ValueError : If the output from the function is not a Spark column. Examples -------- >>> from pyspark.sql.functions import log >>> df = ps.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}, columns=["a", "b"]) >>> df a b 0 1 4 1 2 5 2 3 6 >>> df.a.spark.transform(lambda c: log(c)) 0 0.000000 1 0.693147 2 1.098612 Name: a, dtype: float64 >>> df.index.spark.transform(lambda c: c + 10) Int64Index([10, 11, 12], dtype='int64') >>> df.a.spark.transform(lambda c: c + df.b.spark.column) 0 5 1 7 2 9 Name: a, dtype: int64 """ from pyspark.pandas import MultiIndex if isinstance(self._data, MultiIndex): raise NotImplementedError( "MultiIndex does not support spark.transform yet.") output = func(self._data.spark.column) if not isinstance(output, Column): raise ValueError("The output of the function [%s] should be of a " "pyspark.sql.Column; however, got [%s]." % (func, type(output))) # Trigger the resolution so it throws an exception if anything does wrong # within the function, for example, # `df1.a.spark.transform(lambda _: F.col("non-existent"))`. field = InternalField.from_struct_field( self._data._internal.spark_frame.select(output).schema.fields[0]) return cast(Union["ps.Series", "ps.Index"], self._data._with_new_scol(scol=output, field=field))
def combine_frames( this: "DataFrame", *args: DataFrameOrSeries, how: str = "full", preserve_order_column: bool = False, ) -> "DataFrame": """ This method combines `this` DataFrame with a different `that` DataFrame or Series from a different DataFrame. It returns a DataFrame that has prefix `this_` and `that_` to distinct the columns names from both DataFrames It internally performs a join operation which can be expensive in general. So, if `compute.ops_on_diff_frames` option is False, this method throws an exception. """ from pyspark.pandas.config import get_option from pyspark.pandas.frame import DataFrame from pyspark.pandas.internal import ( InternalField, InternalFrame, HIDDEN_COLUMNS, NATURAL_ORDER_COLUMN_NAME, SPARK_INDEX_NAME_FORMAT, ) from pyspark.pandas.series import Series if all(isinstance(arg, Series) for arg in args): assert all( same_anchor(arg, args[0]) for arg in args ), "Currently only one different DataFrame (from given Series) is supported" assert not same_anchor( this, args[0]), "We don't need to combine. All series is in this." that = args[0]._psdf[list(args)] elif len(args) == 1 and isinstance(args[0], DataFrame): assert isinstance(args[0], DataFrame) assert not same_anchor( this, args[0]), "We don't need to combine. `this` and `that` are same." that = args[0] else: raise AssertionError("args should be single DataFrame or " "single/multiple Series") if get_option("compute.ops_on_diff_frames"): def resolve(internal: InternalFrame, side: str) -> InternalFrame: def rename(col: str) -> str: return "__{}_{}".format(side, col) internal = internal.resolved_copy sdf = internal.spark_frame sdf = internal.spark_frame.select( *[ scol_for(sdf, col).alias(rename(col)) for col in sdf.columns if col not in HIDDEN_COLUMNS ], *HIDDEN_COLUMNS, ) return internal.copy( spark_frame=sdf, index_spark_columns=[ scol_for(sdf, rename(col)) for col in internal.index_spark_column_names ], index_fields=[ field.copy(name=rename(field.name)) for field in internal.index_fields ], data_spark_columns=[ scol_for(sdf, rename(col)) for col in internal.data_spark_column_names ], data_fields=[ field.copy(name=rename(field.name)) for field in internal.data_fields ], ) this_internal = resolve(this._internal, "this") that_internal = resolve(that._internal, "that") this_index_map = list( zip( this_internal.index_spark_column_names, this_internal.index_names, this_internal.index_fields, )) that_index_map = list( zip( that_internal.index_spark_column_names, that_internal.index_names, that_internal.index_fields, )) assert len(this_index_map) == len(that_index_map) join_scols = [] merged_index_scols = [] # Note that the order of each element in index_map is guaranteed according to the index # level. this_and_that_index_map = list(zip(this_index_map, that_index_map)) this_sdf = this_internal.spark_frame.alias("this") that_sdf = that_internal.spark_frame.alias("that") # If the same named index is found, that's used. index_column_names = [] index_use_extension_dtypes = [] for ( i, ((this_column, this_name, this_field), (that_column, that_name, that_field)), ) in enumerate(this_and_that_index_map): if this_name == that_name: # We should merge the Spark columns into one # to mimic pandas' behavior. this_scol = scol_for(this_sdf, this_column) that_scol = scol_for(that_sdf, that_column) join_scol = this_scol == that_scol join_scols.append(join_scol) column_name = SPARK_INDEX_NAME_FORMAT(i) index_column_names.append(column_name) index_use_extension_dtypes.append( any(field.is_extension_dtype for field in [this_field, that_field])) merged_index_scols.append( F.when(this_scol.isNotNull(), this_scol).otherwise(that_scol).alias(column_name)) else: raise ValueError( "Index names must be exactly matched currently.") assert len( join_scols) > 0, "cannot join with no overlapping index names" joined_df = this_sdf.join(that_sdf, on=join_scols, how=how) if preserve_order_column: order_column = [scol_for(this_sdf, NATURAL_ORDER_COLUMN_NAME)] else: order_column = [] joined_df = joined_df.select( *merged_index_scols, *(scol_for(this_sdf, this_internal.spark_column_name_for(label)) for label in this_internal.column_labels), *(scol_for(that_sdf, that_internal.spark_column_name_for(label)) for label in that_internal.column_labels), *order_column, ) index_spark_columns = [ scol_for(joined_df, col) for col in index_column_names ] index_columns = set(index_column_names) new_data_columns = [ col for col in joined_df.columns if col not in index_columns and col != NATURAL_ORDER_COLUMN_NAME ] schema = joined_df.select(*index_spark_columns, *new_data_columns).schema index_fields = [ InternalField.from_struct_field( struct_field, use_extension_dtypes=use_extension_dtypes) for struct_field, use_extension_dtypes in zip( schema.fields[:len(index_spark_columns)], index_use_extension_dtypes) ] data_fields = [ InternalField.from_struct_field( struct_field, use_extension_dtypes=field.is_extension_dtype) for struct_field, field in zip( schema.fields[len(index_spark_columns):], this_internal.data_fields + that_internal.data_fields, ) ] level = max(this_internal.column_labels_level, that_internal.column_labels_level) def fill_label(label: Optional[Label]) -> List: if label is None: return ([""] * (level - 1)) + [None] else: return ([""] * (level - len(label))) + list(label) column_labels = [ tuple(["this"] + fill_label(label)) for label in this_internal.column_labels ] + [ tuple(["that"] + fill_label(label)) for label in that_internal.column_labels ] column_label_names = (cast(List[Optional[Label]], [None]) * (1 + level - this_internal.column_labels_level) ) + this_internal.column_label_names return DataFrame( InternalFrame( spark_frame=joined_df, index_spark_columns=index_spark_columns, index_names=this_internal.index_names, index_fields=index_fields, column_labels=column_labels, data_spark_columns=[ scol_for(joined_df, col) for col in new_data_columns ], data_fields=data_fields, column_label_names=column_label_names, )) else: raise ValueError(ERROR_MESSAGE_CANNOT_COMBINE)
def attach_id_column(self, id_type: str, column: Union[Any, Tuple]) -> "DataFrame": """ Attach a column to be used as identifier of rows similar to the default index. See also `Default Index type <https://koalas.readthedocs.io/en/latest/user_guide/options.html#default-index-type>`_. Parameters ---------- id_type : string The id type. - 'sequence' : a sequence that increases one by one. .. note:: this uses Spark's Window without specifying partition specification. This leads to move all data into single partition in single machine and could cause serious performance degradation. Avoid this method against very large dataset. - 'distributed-sequence' : a sequence that increases one by one, by group-by and group-map approach in a distributed manner. - 'distributed' : a monotonically increasing sequence simply by using PySpark’s monotonically_increasing_id function in a fully distributed manner. column : string or tuple of string The column name. Returns ------- DataFrame The DataFrame attached the column. Examples -------- >>> df = ps.DataFrame({"x": ['a', 'b', 'c']}) >>> df.pandas_on_spark.attach_id_column(id_type="sequence", column="id") x id 0 a 0 1 b 1 2 c 2 >>> df.pandas_on_spark.attach_id_column(id_type="distributed-sequence", column=0) x 0 0 a 0 1 b 1 2 c 2 >>> df.pandas_on_spark.attach_id_column(id_type="distributed", column=0.0) ... # doctest: +ELLIPSIS +NORMALIZE_WHITESPACE x 0.0 0 a ... 1 b ... 2 c ... For multi-index columns: >>> df = ps.DataFrame({("x", "y"): ['a', 'b', 'c']}) >>> df.pandas_on_spark.attach_id_column(id_type="sequence", column=("id-x", "id-y")) x id-x y id-y 0 a 0 1 b 1 2 c 2 >>> df.pandas_on_spark.attach_id_column(id_type="distributed-sequence", column=(0, 1.0)) x 0 y 1.0 0 a 0 1 b 1 2 c 2 """ from pyspark.pandas.frame import DataFrame if id_type == "sequence": attach_func = InternalFrame.attach_sequence_column elif id_type == "distributed-sequence": attach_func = InternalFrame.attach_distributed_sequence_column elif id_type == "distributed": attach_func = InternalFrame.attach_distributed_column else: raise ValueError( "id_type should be one of 'sequence', 'distributed-sequence' and 'distributed'" ) assert is_name_like_value(column, allow_none=False), column if not is_name_like_tuple(column): column = (column,) internal = self._psdf._internal if len(column) != internal.column_labels_level: raise ValueError( "The given column `{}` must be the same length as the existing columns.".format( column ) ) elif column in internal.column_labels: raise ValueError( "The given column `{}` already exists.".format(name_like_string(column)) ) # Make sure the underlying Spark column names are the form of # `name_like_string(column_label)`. sdf = internal.spark_frame.select( [ scol.alias(SPARK_INDEX_NAME_FORMAT(i)) for i, scol in enumerate(internal.index_spark_columns) ] + [ scol.alias(name_like_string(label)) for scol, label in zip(internal.data_spark_columns, internal.column_labels) ] ) sdf, force_nullable = attach_func(sdf, name_like_string(column)) return DataFrame( InternalFrame( spark_frame=sdf, index_spark_columns=[ scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i)) for i in range(internal.index_level) ], index_names=internal.index_names, index_fields=( [field.copy(nullable=True) for field in internal.index_fields] if force_nullable else internal.index_fields ), column_labels=internal.column_labels + [column], data_spark_columns=( [scol_for(sdf, name_like_string(label)) for label in internal.column_labels] + [scol_for(sdf, name_like_string(column))] ), data_fields=( ( [field.copy(nullable=True) for field in internal.data_fields] if force_nullable else internal.data_fields ) + [ InternalField.from_struct_field( StructField(name_like_string(column), LongType(), nullable=False) ) ] ), column_label_names=internal.column_label_names, ).resolved_copy )
def eq(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex: if isinstance(right, (list, tuple)): from pyspark.pandas.series import first_series, scol_for from pyspark.pandas.frame import DataFrame from pyspark.pandas.internal import NATURAL_ORDER_COLUMN_NAME, InternalField len_right = len(right) if len(left) != len(right): raise ValueError("Lengths must be equal") sdf = left._internal.spark_frame structed_scol = F.struct( sdf[NATURAL_ORDER_COLUMN_NAME], *left._internal.index_spark_columns, left.spark.column, ) # The size of the list is expected to be small. collected_structed_scol = F.collect_list(structed_scol) # Sort the array by NATURAL_ORDER_COLUMN so that we can guarantee the order. collected_structed_scol = F.array_sort(collected_structed_scol) right_values_scol = F.array(*(F.lit(x) for x in right)) index_scol_names = left._internal.index_spark_column_names scol_name = left._internal.spark_column_name_for( left._internal.column_labels[0]) # Compare the values of left and right by using zip_with function. cond = F.zip_with( collected_structed_scol, right_values_scol, lambda x, y: F.struct( *[ x[index_scol_name].alias(index_scol_name) for index_scol_name in index_scol_names ], F.when(x[scol_name].isNull() | y.isNull(), False). otherwise(x[scol_name] == y, ).alias(scol_name), ), ).alias(scol_name) # 1. `sdf_new` here looks like the below (the first field of each set is Index): # +----------------------------------------------------------+ # |0 | # +----------------------------------------------------------+ # |[{0, false}, {1, true}, {2, false}, {3, true}, {4, false}]| # +----------------------------------------------------------+ sdf_new = sdf.select(cond) # 2. `sdf_new` after the explode looks like the below: # +----------+ # | col| # +----------+ # |{0, false}| # | {1, true}| # |{2, false}| # | {3, true}| # |{4, false}| # +----------+ sdf_new = sdf_new.select(F.explode(scol_name)) # 3. Here, the final `sdf_new` looks like the below: # +-----------------+-----+ # |__index_level_0__| 0| # +-----------------+-----+ # | 0|false| # | 1| true| # | 2|false| # | 3| true| # | 4|false| # +-----------------+-----+ sdf_new = sdf_new.select("col.*") index_spark_columns = [ scol_for(sdf_new, index_scol_name) for index_scol_name in index_scol_names ] data_spark_columns = [scol_for(sdf_new, scol_name)] internal = left._internal.copy( spark_frame=sdf_new, index_spark_columns=index_spark_columns, data_spark_columns=data_spark_columns, index_fields=[ InternalField.from_struct_field(index_field) for index_field in sdf_new.select( index_spark_columns).schema.fields ], data_fields=[ InternalField.from_struct_field( sdf_new.select(data_spark_columns).schema.fields[0]) ], ) return first_series(DataFrame(internal)) else: from pyspark.pandas.base import column_op return column_op(Column.__eq__)(left, right)