def with_new_columns( self, scols_or_ksers: List[Union[spark.Column, "Series"]], column_labels: Optional[List[Tuple[str, ...]]] = None, keep_order: bool = True, ) -> "_InternalFrame": """ Copy the immutable _InternalFrame with the updates by the specified Spark Columns or Series. :param scols_or_ksers: the new Spark Columns or Series. :param column_labels: the new column index. If None, the its column_labels is used when the corresponding `scols_or_ksers` is Series, otherwise the original one is used. :return: the copied _InternalFrame. """ from databricks.koalas.series import Series if column_labels is None: if all(isinstance(scol_or_kser, Series) for scol_or_kser in scols_or_ksers): column_labels = [kser._internal.column_labels[0] for kser in scols_or_ksers] else: assert len(scols_or_ksers) == len(self.column_labels), ( len(scols_or_ksers), len(self.column_labels), ) column_labels = [] for scol_or_kser, label in zip(scols_or_ksers, self.column_labels): if isinstance(scol_or_kser, Series): column_labels.append(scol_or_kser._internal.column_labels[0]) else: column_labels.append(label) else: assert len(scols_or_ksers) == len(column_labels), ( len(scols_or_ksers), len(column_labels), ) column_scols = [] for scol_or_kser, label in zip(scols_or_ksers, column_labels): if isinstance(scol_or_kser, Series): scol = scol_or_kser._internal.scol else: scol = scol_or_kser column_scols.append(scol) hidden_columns = [] if keep_order: hidden_columns.append(NATURAL_ORDER_COLUMN_NAME) sdf = self._sdf.select(self.index_scols + column_scols + hidden_columns) return self.copy( sdf=sdf, column_labels=column_labels, column_scols=[scol_for(sdf, col) for col in self._sdf.select(column_scols).columns], scol=None, )
def intersection(self, other) -> "MultiIndex": """ Form the intersection of two Index objects. This returns a new Index with elements common to the index and `other`. Parameters ---------- other : Index or array-like Returns ------- intersection : MultiIndex Examples -------- >>> midx1 = ks.MultiIndex.from_tuples([("a", "x"), ("b", "y"), ("c", "z")]) >>> midx2 = ks.MultiIndex.from_tuples([("c", "z"), ("d", "w")]) >>> midx1.intersection(midx2).sort_values() # doctest: +SKIP MultiIndex([('c', 'z')], ) """ if isinstance(other, Series) or not is_list_like(other): raise TypeError("other must be a MultiIndex or a list of tuples") elif isinstance(other, DataFrame): raise ValueError("Index data must be 1-dimensional") elif isinstance(other, MultiIndex): spark_frame_other = other.to_frame().to_spark() keep_name = self.names == other.names elif isinstance(other, Index): # Always returns an empty MultiIndex if `other` is Index. return self.to_frame().head(0).index # type: ignore elif not all(isinstance(item, tuple) for item in other): raise TypeError("other must be a MultiIndex or a list of tuples") else: other = MultiIndex.from_tuples(list(other)) spark_frame_other = other.to_frame().to_spark() keep_name = True default_name = [ SPARK_INDEX_NAME_FORMAT(i) for i in range(self.nlevels) ] spark_frame_self = self.to_frame(name=default_name).to_spark() spark_frame_intersected = spark_frame_self.intersect(spark_frame_other) if keep_name: index_names = self._internal.index_names else: index_names = None internal = InternalFrame( # TODO: dtypes? spark_frame=spark_frame_intersected, index_spark_columns=[ scol_for(spark_frame_intersected, col) for col in default_name ], index_names=index_names, ) return cast(MultiIndex, DataFrame(internal).index)
def from_pandas(pdf: pd.DataFrame) -> '_InternalFrame': """ Create an immutable DataFrame from pandas DataFrame. :param pdf: :class:`pd.DataFrame` :return: the created immutable DataFrame """ columns = pdf.columns data_columns = [name_like_string(col) for col in columns] if isinstance(columns, pd.MultiIndex): column_index = columns.tolist() else: column_index = None column_index_names = columns.names index = pdf.index index_map = [] # type: List[IndexMap] if isinstance(index, pd.MultiIndex): if index.names is None: index_map = [(SPARK_INDEX_NAME_FORMAT(i), None) for i in range(len(index.levels))] else: index_map = [ (SPARK_INDEX_NAME_FORMAT(i) if name is None else name, name if name is None or isinstance(name, tuple) else (name, )) for i, name in enumerate(index.names) ] else: name = index.name index_map = [(name_like_string(name) if name is not None else SPARK_INDEX_NAME_FORMAT(0), name if name is None or isinstance(name, tuple) else (name, ))] index_columns = [index_column for index_column, _ in index_map] reset_index = pdf.reset_index() reset_index.columns = index_columns + data_columns schema = StructType([ StructField(name_like_string(name), infer_pd_series_spark_type(col), nullable=bool(col.isnull().any())) for name, col in reset_index.iteritems() ]) for name, col in reset_index.iteritems(): dt = col.dtype if is_datetime64_dtype(dt) or is_datetime64tz_dtype(dt): continue reset_index[name] = col.replace({np.nan: None}) sdf = default_session().createDataFrame(reset_index, schema=schema) return _InternalFrame( sdf=sdf, index_map=index_map, column_index=column_index, column_scols=[scol_for(sdf, col) for col in data_columns], column_index_names=column_index_names)
def resolved_copy(self): """ Copy the immutable InternalFrame with the updates resolved. """ sdf = self.spark_frame.select(self.spark_columns + list(HIDDEN_COLUMNS)) return self.copy( spark_frame=sdf, data_spark_columns=[ scol_for(sdf, col) for col in self.data_spark_column_names ], )
def scol_for(self, column_name_or_index: Union[str, Tuple[str]]) -> spark.Column: """ Return Spark Column for the given column name or index. """ if self._scol is not None and ( column_name_or_index == self._data_columns[0] or column_name_or_index == self._column_index[0]): return self._scol else: return scol_for(self._sdf, self.column_name_for(column_name_or_index))
def _internal(self): internal = super(iLocIndexer, self)._internal if self._is_series: sdf = internal.spark_frame.select(internal.index_spark_columns + [internal.spark_column]) scol = scol_for(sdf, internal.data_spark_column_names[0]) else: sdf = internal.spark_frame scol = None sdf = _InternalFrame.attach_distributed_sequence_column( sdf, column_name=self._sequence_col) return internal.copy( spark_frame=sdf.orderBy(NATURAL_ORDER_COLUMN_NAME), spark_column=scol)
def from_pandas(pdf: pd.DataFrame) -> "InternalFrame": """ Create an immutable DataFrame from pandas DataFrame. :param pdf: :class:`pd.DataFrame` :return: the created immutable DataFrame """ columns = pdf.columns data_columns = [name_like_string(col) for col in columns] if isinstance(columns, pd.MultiIndex): column_labels = columns.tolist() else: column_labels = [(col, ) for col in columns] column_label_names = [ name if name is None or isinstance(name, tuple) else (name, ) for name in columns.names ] index_names = [ name if name is None or isinstance(name, tuple) else (name, ) for name in pdf.index.names ] index_columns = [ SPARK_INDEX_NAME_FORMAT(i) for i in range(len(index_names)) ] pdf = pdf.copy() pdf.index.names = index_columns reset_index = pdf.reset_index() reset_index.columns = index_columns + data_columns schema = StructType([ StructField( name, infer_pd_series_spark_type(col), nullable=bool(col.isnull().any()), ) for name, col in reset_index.iteritems() ]) for name, col in reset_index.iteritems(): dt = col.dtype if is_datetime64_dtype(dt) or is_datetime64tz_dtype(dt): continue reset_index[name] = col.replace({np.nan: None}) sdf = default_session().createDataFrame(reset_index, schema=schema) return InternalFrame( spark_frame=sdf, index_spark_column_names=index_columns, index_names=index_names, column_labels=column_labels, data_spark_columns=[scol_for(sdf, col) for col in data_columns], column_label_names=column_label_names, )
def attach_default_index(sdf, default_index_type=None): """ This method attaches a default index to Spark DataFrame. Spark does not have the index notion so corresponding column should be generated. There are several types of default index can be configured by `compute.default_index_type`. >>> spark_frame = ks.range(10).to_spark() >>> spark_frame DataFrame[id: bigint] It adds the default index column '__index_level_0__'. >>> spark_frame = InternalFrame.attach_default_index(spark_frame) >>> spark_frame DataFrame[__index_level_0__: int, id: bigint] It throws an exception if the given column name already exists. >>> InternalFrame.attach_default_index(spark_frame) ... # doctest: +ELLIPSIS Traceback (most recent call last): ... AssertionError: '__index_level_0__' already exists... """ index_column = SPARK_DEFAULT_INDEX_NAME assert (index_column not in sdf.columns ), "'%s' already exists in the Spark column names '%s'" % ( index_column, sdf.columns) if default_index_type is None: default_index_type = get_option("compute.default_index_type") scols = [scol_for(sdf, column) for column in sdf.columns] if default_index_type == "sequence": sequential_index = (F.row_number().over( Window.orderBy(F.monotonically_increasing_id())) - 1) return sdf.select(sequential_index.alias(index_column), *scols) elif default_index_type == "distributed-sequence": return InternalFrame.attach_distributed_sequence_column( sdf, column_name=index_column) elif default_index_type == "distributed": return InternalFrame.attach_distributed_column( sdf, column_name=index_column) else: raise ValueError( "'compute.default_index_type' should be one of 'sequence'," " 'distributed-sequence' and 'distributed'")
def with_new_sdf( self, sdf: spark.DataFrame, data_columns: Optional[List[str]] = None ) -> "_InternalFrame": """ Copy the immutable _InternalFrame with the updates by the specified Spark DataFrame. :param sdf: the new Spark DataFrame :param data_columns: the new column names. If None, the original one is used. :return: the copied _InternalFrame. """ if data_columns is None: data_columns = self.data_columns else: assert len(data_columns) == len(self.column_labels), ( len(data_columns), len(self.column_labels), ) sdf = sdf.drop(NATURAL_ORDER_COLUMN_NAME) return self.copy(sdf=sdf, column_scols=[scol_for(sdf, col) for col in data_columns])
def _is_monotonic_decreasing(self): scol = self.spark.column window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween(-1, -1) prev = F.lag(scol, 1).over(window) cond = F.lit(True) has_not_null = F.lit(True) for field in self.spark.data_type[::-1]: left = scol.getField(field.name) right = prev.getField(field.name) compare = MultiIndex._comparator_for_monotonic_decreasing( field.dataType) # Since pandas 1.1.4, null value is not allowed at any levels of MultiIndex. # Therefore, we should check `has_not_null` over the all levels. has_not_null = has_not_null & left.isNotNull() cond = F.when(left.eqNullSafe(right), cond).otherwise( compare(left, right, spark.Column.__lt__)) cond = has_not_null & (prev.isNull() | cond) cond_name = verify_temp_column_name( self._internal.spark_frame.select( self._internal.index_spark_columns), "__is_monotonic_decreasing_cond__", ) sdf = self._internal.spark_frame.select( self._internal.index_spark_columns + [cond.alias(cond_name)]) internal = InternalFrame( spark_frame=sdf, index_spark_columns=[ scol_for(sdf, col) for col in self._internal.index_spark_column_names ], index_names=self._internal.index_names, index_dtypes=self._internal.index_dtypes, ) return first_series(DataFrame(internal))
def with_filter(self, pred: Union[spark.Column, "Series"]): """ Copy the immutable _InternalFrame with the updates by the predicate. :param pred: the predicate to filter. :return: the copied _InternalFrame. """ from databricks.koalas.series import Series if isinstance(pred, Series): assert isinstance(pred.spark.data_type, BooleanType), pred.spark.data_type pred = pred.spark.column else: spark_type = self.spark_frame.select(pred).schema[0].dataType assert isinstance(spark_type, BooleanType), spark_type sdf = self.spark_frame.filter(pred).select(self.spark_columns) if self.spark_column is None: return self.with_new_sdf(sdf) else: return self.copy( spark_frame=sdf, spark_column=scol_for(sdf, self.data_spark_column_names[0]) )
def with_new_columns( self, scols_or_ksers: List[Union[spark.Column, "Series"]], column_labels: Optional[List[Tuple[str, ...]]] = None, column_label_names: Optional[Union[List[str], _NoValueType]] = _NoValue, keep_order: bool = True, ) -> "InternalFrame": """ Copy the immutable InternalFrame with the updates by the specified Spark Columns or Series. :param scols_or_ksers: the new Spark Columns or Series. :param column_labels: the new column index. If None, the its column_labels is used when the corresponding `scols_or_ksers` is Series, otherwise the original one is used. :return: the copied InternalFrame. """ from databricks.koalas.series import Series if column_labels is None: if all( isinstance(scol_or_kser, Series) for scol_or_kser in scols_or_ksers): column_labels = [ kser._internal.column_labels[0] for kser in scols_or_ksers ] else: assert len(scols_or_ksers) == len(self.column_labels), ( len(scols_or_ksers), len(self.column_labels), ) column_labels = [] for scol_or_kser, label in zip(scols_or_ksers, self.column_labels): if isinstance(scol_or_kser, Series): column_labels.append( scol_or_kser._internal.column_labels[0]) else: column_labels.append(label) else: assert len(scols_or_ksers) == len(column_labels), ( len(scols_or_ksers), len(column_labels), ) data_spark_columns = [] for scol_or_kser in scols_or_ksers: if isinstance(scol_or_kser, Series): scol = scol_or_kser.spark.column else: scol = scol_or_kser data_spark_columns.append(scol) sdf = self.spark_frame if not keep_order: sdf = self.spark_frame.select(self.index_spark_columns + data_spark_columns) data_spark_columns = [ scol_for(sdf, col) for col in self.spark_frame.select(data_spark_columns).columns ] if column_label_names is _NoValue: column_label_names = self._column_label_names return self.copy( spark_frame=sdf, column_labels=column_labels, data_spark_columns=data_spark_columns, column_label_names=column_label_names, )
def scol_for(self, column_name: str) -> spark.Column: """ Return Spark Column for the given column name. """ if self._scol is not None and column_name == self._data_columns[0]: return self._scol else: return scol_for(self._sdf, column_name)
def index_spark_columns(self) -> List[spark.Column]: """ Return Spark Columns for the managed index columns. """ return [ scol_for(self.spark_frame, column) for column in self.index_spark_column_names ]
def attach_default_index(sdf, default_index_type=None): """ This method attaches a default index to Spark DataFrame. Spark does not have the index notion so corresponding column should be generated. There are several types of default index can be configured by `compute.default_index_type`. """ if default_index_type is None: default_index_type = get_option("compute.default_index_type") scols = [scol_for(sdf, column) for column in sdf.columns] if default_index_type == "sequence": sequential_index = F.row_number().over( Window.orderBy(F.monotonically_increasing_id())) - 1 return sdf.select( sequential_index.alias(SPARK_INDEX_NAME_FORMAT(0)), *scols) elif default_index_type == "distributed-sequence": # 1. Calculates counts per each partition ID. `counts` here is, for instance, # { # 1: 83, # 6: 83, # 3: 83, # ... # } sdf = sdf.withColumn("__spark_partition_id", F.spark_partition_id()) counts = map( lambda x: (x["key"], x["count"]), sdf.groupby(sdf['__spark_partition_id'].alias( "key")).count().collect()) # 2. Calculates cumulative sum in an order of partition id. # Note that it does not matter if partition id guarantees its order or not. # We just need a one-by-one sequential id. # sort by partition key. sorted_counts = sorted(counts, key=lambda x: x[0]) # get cumulative sum in an order of partition key. cumulative_counts = [0] + list( accumulate(map(lambda count: count[1], sorted_counts))) # zip it with partition key. sums = dict( zip(map(lambda count: count[0], sorted_counts), cumulative_counts)) # 3. Attach offset for each partition. @pandas_udf(LongType(), PandasUDFType.SCALAR) def offset(id): current_partition_offset = sums[id.iloc[0]] return pd.Series(current_partition_offset).repeat(len(id)) sdf = sdf.withColumn('__offset__', offset('__spark_partition_id')) # 4. Calculate row_number in each partition. w = Window.partitionBy('__spark_partition_id').orderBy( F.monotonically_increasing_id()) row_number = F.row_number().over(w) sdf = sdf.withColumn('__row_number__', row_number) # 5. Calcuate the index. return sdf.select( F.expr('__offset__ + __row_number__ - 1').alias( SPARK_INDEX_NAME_FORMAT(0)), *scols) elif default_index_type == "distributed": return sdf.select( F.monotonically_increasing_id().alias( SPARK_INDEX_NAME_FORMAT(0)), *scols) else: raise ValueError( "'compute.default_index_type' should be one of 'sequence'," " 'distributed-sequence' and 'distributed'")
def attach_distributed_column(sdf, column_name): scols = [scol_for(sdf, column) for column in sdf.columns] return sdf.select(F.monotonically_increasing_id().alias(column_name), *scols)
def attach_distributed_sequence_column(sdf, column_name): """ This method attaches a Spark column that has a sequence in a distributed manner. This is equivalent to the column assigned when default index type 'distributed-sequence'. >>> sdf = ks.DataFrame(['a', 'b', 'c']).to_spark() >>> sdf = InternalFrame.attach_distributed_sequence_column(sdf, column_name="sequence") >>> sdf.sort("sequence").show() # doctest: +NORMALIZE_WHITESPACE +--------+---+ |sequence| 0| +--------+---+ | 0| a| | 1| b| | 2| c| +--------+---+ """ scols = [scol_for(sdf, column) for column in sdf.columns] spark_partition_column = verify_temp_column_name( sdf, "__spark_partition_id__") offset_column = verify_temp_column_name(sdf, "__offset__") row_number_column = verify_temp_column_name(sdf, "__row_number__") # 1. Calculates counts per each partition ID. `counts` here is, for instance, # { # 1: 83, # 6: 83, # 3: 83, # ... # } sdf = sdf.withColumn(spark_partition_column, F.spark_partition_id()) counts = map( lambda x: (x["key"], x["count"]), sdf.groupby( sdf[spark_partition_column].alias("key")).count().collect(), ) # 2. Calculates cumulative sum in an order of partition id. # Note that it does not matter if partition id guarantees its order or not. # We just need a one-by-one sequential id. # sort by partition key. sorted_counts = sorted(counts, key=lambda x: x[0]) # get cumulative sum in an order of partition key. cumulative_counts = [0] + list( accumulate(map(lambda count: count[1], sorted_counts))) # zip it with partition key. sums = dict( zip(map(lambda count: count[0], sorted_counts), cumulative_counts)) # 3. Attach offset for each partition. @pandas_udf(LongType(), PandasUDFType.SCALAR) def offset(id): current_partition_offset = sums[id.iloc[0]] return pd.Series(current_partition_offset).repeat(len(id)) sdf = sdf.withColumn(offset_column, offset(spark_partition_column)) # 4. Calculate row_number in each partition. w = Window.partitionBy(spark_partition_column).orderBy( F.monotonically_increasing_id()) row_number = F.row_number().over(w) sdf = sdf.withColumn(row_number_column, row_number) # 5. Calculate the index. return sdf.select((sdf[offset_column] + sdf[row_number_column] - 1).alias(column_name), *scols)
def __init__( self, spark_frame: spark.DataFrame, index_map: Optional[Dict[str, Optional[Tuple[str, ...]]]], column_labels: Optional[List[Tuple[str, ...]]] = None, data_spark_columns: Optional[List[spark.Column]] = None, column_label_names: Optional[List[str]] = None, ) -> None: """ Create a new internal immutable DataFrame to manage Spark DataFrame, column fields and index fields and names. :param spark_frame: Spark DataFrame to be managed. :param index_map: dictionary of string pairs Each pair holds the index field name which exists in Spark fields, and the index name. :param column_labels: list of tuples with the same length The multi-level values in the tuples. :param data_spark_columns: list of Spark Column Spark Columns to appear as columns. If spark_column is not None, this argument is ignored, otherwise if this is None, calculated from spark_frame. :param column_label_names: Names for each of the index levels. See the examples below to refer what each parameter means. >>> column_labels = pd.MultiIndex.from_tuples( ... [('a', 'x'), ('a', 'y'), ('b', 'z')], names=["column_labels_a", "column_labels_b"]) >>> row_index = pd.MultiIndex.from_tuples( ... [('foo', 'bar'), ('foo', 'bar'), ('zoo', 'bar')], ... names=["row_index_a", "row_index_b"]) >>> kdf = ks.DataFrame( ... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=row_index, columns=column_labels) >>> kdf.set_index(('a', 'x'), append=True, inplace=True) >>> kdf # doctest: +NORMALIZE_WHITESPACE column_labels_a a b column_labels_b y z row_index_a row_index_b (a, x) foo bar 1 2 3 4 5 6 zoo bar 7 8 9 >>> internal = kdf[('a', 'y')]._internal >>> internal._sdf.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS +-----------------+-----------------+------+------+------+... |__index_level_0__|__index_level_1__|(a, x)|(a, y)|(b, z)|... +-----------------+-----------------+------+------+------+... | foo| bar| 1| 2| 3|... | foo| bar| 4| 5| 6|... | zoo| bar| 7| 8| 9|... +-----------------+-----------------+------+------+------+... >>> internal._index_map # doctest: +NORMALIZE_WHITESPACE OrderedDict([('__index_level_0__', ('row_index_a',)), ('__index_level_1__', ('row_index_b',)), ('(a, x)', ('a', 'x'))]) >>> internal._column_labels [('a', 'y')] >>> internal._data_spark_columns [Column<b'(a, y)'>] >>> list(internal._column_label_names) ['column_labels_a', 'column_labels_b'] """ assert isinstance(spark_frame, spark.DataFrame) assert not spark_frame.isStreaming, "Koalas does not support Structured Streaming." if index_map is None: assert not any( SPARK_INDEX_NAME_PATTERN.match(name) for name in spark_frame.columns ), ("Index columns should not appear in columns of the Spark DataFrame. Avoid " "index column names [%s]." % SPARK_INDEX_NAME_PATTERN) # Create default index. spark_frame = InternalFrame.attach_default_index(spark_frame) index_map = OrderedDict({SPARK_DEFAULT_INDEX_NAME: None}) if NATURAL_ORDER_COLUMN_NAME not in spark_frame.columns: spark_frame = spark_frame.withColumn( NATURAL_ORDER_COLUMN_NAME, F.monotonically_increasing_id()) assert isinstance(index_map, OrderedDict), index_map assert all( isinstance(index_field, str) and ( index_name is None or (isinstance(index_name, tuple) and all( isinstance(name, str) for name in index_name))) for index_field, index_name in index_map.items()), index_map assert data_spark_columns is None or all( isinstance(scol, spark.Column) for scol in data_spark_columns) self._sdf = spark_frame # type: spark.DataFrame self._index_map = index_map # type: Dict[str, Optional[Tuple[str, ...]]] if data_spark_columns is None: index_columns = set(index_column for index_column in self._index_map) self._data_spark_columns = [ scol_for(spark_frame, col) for col in spark_frame.columns if col not in index_columns and col not in HIDDEN_COLUMNS ] else: self._data_spark_columns = data_spark_columns if column_labels is None: self._column_labels = [ (col, ) for col in spark_frame.select(self._data_spark_columns).columns ] # type: List[Tuple[str, ...]] else: assert len(column_labels) == len(self._data_spark_columns), ( len(column_labels), len(self._data_spark_columns), ) if len(column_labels) == 1: column_label = column_labels[0] assert column_label is None or ( isinstance(column_label, tuple) and len(column_label) > 0), column_label else: assert all( isinstance(label, tuple) and len(label) > 0 for label in column_labels), column_labels assert len(set(len(label) for label in column_labels)) <= 1, column_labels self._column_labels = column_labels if column_label_names is not None and not is_list_like( column_label_names): raise ValueError( "Column_index_names should be list-like or None for a MultiIndex" ) if isinstance(column_label_names, list): if all(name is None for name in column_label_names): self._column_label_names = None else: self._column_label_names = column_label_names else: self._column_label_names = column_label_names
def __getitem__(self, key): from databricks.koalas.frame import DataFrame from databricks.koalas.series import Series if self._is_series: if isinstance(key, Series) and key._kdf is not self._kdf_or_kser._kdf: kdf = self._kdf_or_kser.to_frame() kdf["__temp_col__"] = key return type(self)( kdf[self._kdf_or_kser.name])[kdf["__temp_col__"]] cond, limit, remaining_index = self._select_rows(key) if cond is None and limit is None: return self._kdf_or_kser column_labels = self._internal.column_labels column_scols = self._internal.column_scols returns_series = True else: assert self._is_df if isinstance(key, tuple): if len(key) != 2: raise SparkPandasIndexingError( "Only accepts pairs of candidates") rows_sel, cols_sel = key else: rows_sel = key cols_sel = None if isinstance(rows_sel, Series) and rows_sel._kdf is not self._kdf_or_kser: kdf = self._kdf_or_kser.copy() kdf["__temp_col__"] = rows_sel return type(self)(kdf)[kdf["__temp_col__"], cols_sel][list( self._kdf_or_kser.columns)] cond, limit, remaining_index = self._select_rows(rows_sel) column_labels, column_scols, returns_series = self._select_cols( cols_sel) if cond is None and limit is None and returns_series: return self._kdf_or_kser._kser_for(column_labels[0]) if remaining_index is not None: index_scols = self._internal.index_scols[-remaining_index:] index_map = self._internal.index_map[-remaining_index:] else: index_scols = self._internal.index_scols index_map = self._internal.index_map if len(column_labels) > 0: column_labels = column_labels.copy() column_labels_level = max( len(label) if label is not None else 1 for label in column_labels) none_column = 0 for i, label in enumerate(column_labels): if label is None: label = (str(none_column), ) none_column += 1 if len(label) < column_labels_level: label = tuple( list(label) + ([""]) * (column_labels_level - len(label))) column_labels[i] = label if self._internal.column_label_names is None: column_label_names = None else: # Manage column index names column_label_names = self._internal.column_label_names[ -column_labels_level:] else: column_label_names = None try: sdf = self._internal._sdf if cond is not None: sdf = sdf.drop(NATURAL_ORDER_COLUMN_NAME).filter(cond) if limit is not None: if limit >= 0: sdf = sdf.limit(limit) else: sdf = sdf.limit(sdf.count() + limit) data_columns = sdf.select(column_scols).columns sdf = sdf.select(index_scols + column_scols) except AnalysisException: raise KeyError("[{}] don't exist in columns".format( [col._jc.toString() for col in column_scols])) internal = _InternalFrame( sdf=sdf, index_map=index_map, column_labels=column_labels, column_scols=[scol_for(sdf, col) for col in data_columns], column_label_names=column_label_names, ) kdf = DataFrame(internal) if returns_series: kdf_or_kser = Series( kdf._internal.copy(scol=kdf._internal.column_scols[0]), anchor=kdf) else: kdf_or_kser = kdf if remaining_index is not None and remaining_index == 0: pdf_or_pser = kdf_or_kser.head(2).to_pandas() length = len(pdf_or_pser) if length == 0: raise KeyError(name_like_string(key)) elif length == 1: return pdf_or_pser.iloc[0] else: return kdf_or_kser else: return kdf_or_kser
def attach_default_index(sdf): """ This method attaches a default index to Spark DataFrame. Spark does not have the index notion so corresponding column should be generated. There are three types of default index that can be controlled by `DEFAULT_INDEX` environment variable. - one-by-one: It implements an one-by-one sequence by Window function without specifying partition. Therefore, it ends up with whole partition in single node. This index type should be avoided when the data is large. This is default. - distributed-one-by-one: It implements an one-by-one sequence by group-by and group-map approach. It still generates a one-by-one sequential index globally. If the default index must be an one-by-one sequence in a large dataset, this index has to be used. Note that if more data are added to the data source after creating this index, then it does not guarantee the sequential index. - distributed: It implements a monotonically increasing sequence simply by using Spark's `monotonically_increasing_id` function. If the index does not have to be a one-by-one sequence, this index should be used. Performance-wise, this index almost does not have any penalty comparing to other index types. Note that we cannot use this type of index for combining two dataframes because it is not guaranteed to have the same indexes in two dataframes. """ default_index_type = os.environ.get("DEFAULT_INDEX", "one-by-one") if default_index_type == "one-by-one": sequential_index = F.row_number().over( Window.orderBy(F.monotonically_increasing_id().asc())) - 1 scols = [scol_for(sdf, column) for column in sdf.columns] return sdf.select(sequential_index.alias("__index_level_0__"), *scols) elif default_index_type == "distributed-one-by-one": # 1. Calculates counts per each partition ID. `counts` here is, for instance, # { # 1: 83, # 6: 83, # 3: 83, # ... # } counts = map( lambda x: (x["key"], x["count"]), sdf.groupby( F.spark_partition_id().alias("key")).count().collect()) # 2. Calculates cumulative sum in an order of partition id. # Note that it does not matter if partition id guarantees its order or not. # We just need a one-by-one sequential id. # sort by partition key. sorted_counts = sorted(counts, key=lambda x: x[0]) # get cumulative sum in an order of partition key. cumulative_counts = accumulate( map(lambda count: count[1], sorted_counts)) # zip it with partition key. sums = dict( zip(map(lambda count: count[0], sorted_counts), cumulative_counts)) # 3. Group by partition id and assign each range. def default_index(pdf): current_partition_max = sums[ pdf["__spark_partition_id"].iloc[0]] offset = len(pdf) pdf["__index_level_0__"] = list( range(current_partition_max - offset, current_partition_max)) return pdf.drop(columns=["__spark_partition_id"]) return_schema = StructType( [StructField("__index_level_0__", LongType())] + list(sdf.schema)) grouped_map_func = pandas_udf( return_schema, PandasUDFType.GROUPED_MAP)(default_index) sdf = sdf.withColumn("__spark_partition_id", F.spark_partition_id()) return sdf.groupBy("__spark_partition_id").apply(grouped_map_func) elif default_index_type == "distributed": scols = [scol_for(sdf, column) for column in sdf.columns] return sdf.select( F.monotonically_increasing_id().alias("__index_level_0__"), *scols) else: raise ValueError( "'DEFAULT_INDEX' environment variable should be one of 'one-by-one'," " 'distributed-one-by-one' and 'distributed'")
def attach_sequence_column(sdf, column_name): scols = [scol_for(sdf, column) for column in sdf.columns] sequential_index = (F.row_number().over( Window.orderBy(F.monotonically_increasing_id())).cast("long") - 1) return sdf.select(sequential_index.alias(column_name), *scols)
def __init__(self, sdf: spark.DataFrame, index_map: Optional[List[IndexMap]] = None, column_index: Optional[List[Tuple[str, ...]]] = None, column_scols: Optional[List[spark.Column]] = None, column_index_names: Optional[List[str]] = None, scol: Optional[spark.Column] = None) -> None: """ Create a new internal immutable DataFrame to manage Spark DataFrame, column fields and index fields and names. :param sdf: Spark DataFrame to be managed. :param index_map: list of string pair Each pair holds the index field name which exists in Spark fields, and the index name. :param column_index: list of tuples with the same length The multi-level values in the tuples. :param column_scols: list of Spark Column Spark Columns to appear as columns. If scol is not None, this argument is ignored, otherwise if this is None, calculated from sdf. :param column_index_names: Names for each of the index levels. :param scol: Spark Column to be managed. """ assert isinstance(sdf, spark.DataFrame) if NATURAL_ORDER_COLUMN_NAME not in sdf.columns: sdf = sdf.withColumn(NATURAL_ORDER_COLUMN_NAME, F.monotonically_increasing_id()) if index_map is None: # Here is when Koalas DataFrame is created directly from Spark DataFrame. assert not any(SPARK_INDEX_NAME_PATTERN.match(name) for name in sdf.schema.names), \ "Index columns should not appear in columns of the Spark DataFrame. Avoid " \ "index colum names [%s]." % SPARK_INDEX_NAME_PATTERN # Create default index. index_map = [(SPARK_INDEX_NAME_FORMAT(0), None)] sdf = _InternalFrame.attach_default_index(sdf) assert index_map is not None assert all( isinstance(index_field, str) and ( index_name is None or (isinstance(index_name, tuple) and all( isinstance(name, str) for name in index_name))) for index_field, index_name in index_map), index_map assert scol is None or isinstance(scol, spark.Column) assert column_scols is None or all( isinstance(scol, spark.Column) for scol in column_scols) self._sdf = sdf # type: spark.DataFrame self._index_map = index_map # type: List[IndexMap] self._scol = scol # type: Optional[spark.Column] if scol is not None: self._column_scols = [scol] elif column_scols is None: index_columns = set(index_column for index_column, _ in self._index_map) self._column_scols = [ scol_for(sdf, col) for col in sdf.columns if col not in index_columns and col not in HIDDEN_COLUMNS ] else: self._column_scols = column_scols if scol is not None: assert column_index is not None and len( column_index) == 1, column_index assert all(idx is None or (isinstance(idx, tuple) and len(idx) > 0) for idx in column_index), column_index self._column_index = column_index elif column_index is None: self._column_index = [(sdf.select(scol).columns[0], ) for scol in self._column_scols] else: assert len(column_index) == len(self._column_scols), \ (len(column_index), len(self._column_scols)) assert all(isinstance(i, tuple) for i in column_index), column_index assert len(set(len(i) for i in column_index)) <= 1, column_index self._column_index = column_index if column_index_names is not None and not is_list_like( column_index_names): raise ValueError( 'Column_index_names should be list-like or None for a MultiIndex' ) if isinstance(column_index_names, list): if all(name is None for name in column_index_names): self._column_index_names = None else: self._column_index_names = column_index_names else: self._column_index_names = column_index_names
def value_counts(self, normalize=False, sort=True, ascending=False, bins=None, dropna=True): """ Return a Series containing counts of unique values. The resulting object will be in descending order so that the first element is the most frequently-occurring element. Excludes NA values by default. Parameters ---------- normalize : boolean, default False If True then the object returned will contain the relative frequencies of the unique values. sort : boolean, default True Sort by values. ascending : boolean, default False Sort in ascending order. bins : Not Yet Supported dropna : boolean, default True Don't include counts of NaN. Returns ------- counts : Series See Also -------- Series.count: Number of non-NA elements in a Series. Examples -------- For Series >>> df = ks.DataFrame({'x':[0, 0, 1, 1, 1, np.nan]}) >>> df.x.value_counts() # doctest: +NORMALIZE_WHITESPACE 1.0 3 0.0 2 Name: x, dtype: int64 With `normalize` set to `True`, returns the relative frequency by dividing all values by the sum of values. >>> df.x.value_counts(normalize=True) # doctest: +NORMALIZE_WHITESPACE 1.0 0.6 0.0 0.4 Name: x, dtype: float64 **dropna** With `dropna` set to `False` we can also see NaN index values. >>> df.x.value_counts(dropna=False) # doctest: +NORMALIZE_WHITESPACE 1.0 3 0.0 2 NaN 1 Name: x, dtype: int64 For Index >>> idx = ks.Index([3, 1, 2, 3, 4, np.nan]) >>> idx Float64Index([3.0, 1.0, 2.0, 3.0, 4.0, nan], dtype='float64') >>> idx.value_counts().sort_index() 1.0 1 2.0 1 3.0 2 4.0 1 dtype: int64 **sort** With `sort` set to `False`, the result wouldn't be sorted by number of count. >>> idx.value_counts(sort=True).sort_index() 1.0 1 2.0 1 3.0 2 4.0 1 dtype: int64 **normalize** With `normalize` set to `True`, returns the relative frequency by dividing all values by the sum of values. >>> idx.value_counts(normalize=True).sort_index() 1.0 0.2 2.0 0.2 3.0 0.4 4.0 0.2 dtype: float64 **dropna** With `dropna` set to `False` we can also see NaN index values. >>> idx.value_counts(dropna=False).sort_index() # doctest: +SKIP 1.0 1 2.0 1 3.0 2 4.0 1 NaN 1 dtype: int64 For MultiIndex. >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], ... ['speed', 'weight', 'length']], ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], ... [1, 1, 1, 1, 1, 2, 1, 2, 2]]) >>> s = ks.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx) >>> s.index # doctest: +SKIP MultiIndex([( 'lama', 'weight'), ( 'lama', 'weight'), ( 'lama', 'weight'), ( 'cow', 'weight'), ( 'cow', 'weight'), ( 'cow', 'length'), ('falcon', 'weight'), ('falcon', 'length'), ('falcon', 'length')], ) >>> s.index.value_counts().sort_index() (cow, length) 1 (cow, weight) 2 (falcon, length) 2 (falcon, weight) 1 (lama, weight) 3 dtype: int64 >>> s.index.value_counts(normalize=True).sort_index() (cow, length) 0.111111 (cow, weight) 0.222222 (falcon, length) 0.222222 (falcon, weight) 0.111111 (lama, weight) 0.333333 dtype: float64 If Index has name, keep the name up. >>> idx = ks.Index([0, 0, 0, 1, 1, 2, 3], name='koalas') >>> idx.value_counts().sort_index() 0 3 1 2 2 1 3 1 Name: koalas, dtype: int64 """ from databricks.koalas.series import first_series if bins is not None: raise NotImplementedError( "value_counts currently does not support bins") if dropna: sdf_dropna = self._internal.spark_frame.select( self.spark.column).dropna() else: sdf_dropna = self._internal.spark_frame.select(self.spark.column) index_name = SPARK_DEFAULT_INDEX_NAME column_name = self._internal.data_spark_column_names[0] sdf = sdf_dropna.groupby( scol_for(sdf_dropna, column_name).alias(index_name)).count() if sort: if ascending: sdf = sdf.orderBy(F.col("count")) else: sdf = sdf.orderBy(F.col("count").desc()) if normalize: sum = sdf_dropna.count() sdf = sdf.withColumn("count", F.col("count") / F.lit(sum)) internal = InternalFrame( spark_frame=sdf, index_map=OrderedDict({index_name: None}), column_labels=self._internal.column_labels, data_spark_columns=[scol_for(sdf, "count")], column_label_names=self._internal.column_label_names, ) return first_series(DataFrame(internal))
def attach_default_index(sdf): """ This method attaches a default index to Spark DataFrame. Spark does not have the index notion so corresponding column should be generated. There are several types of default index can be configured by `compute.default_index_type`. """ default_index_type = get_option("compute.default_index_type") if default_index_type == "sequence": sequential_index = F.row_number().over( Window.orderBy(F.monotonically_increasing_id().asc())) - 1 scols = [scol_for(sdf, column) for column in sdf.columns] return sdf.select(sequential_index.alias("__index_level_0__"), *scols) elif default_index_type == "distributed-sequence": # 1. Calculates counts per each partition ID. `counts` here is, for instance, # { # 1: 83, # 6: 83, # 3: 83, # ... # } counts = map( lambda x: (x["key"], x["count"]), sdf.groupby( F.spark_partition_id().alias("key")).count().collect()) # 2. Calculates cumulative sum in an order of partition id. # Note that it does not matter if partition id guarantees its order or not. # We just need a one-by-one sequential id. # sort by partition key. sorted_counts = sorted(counts, key=lambda x: x[0]) # get cumulative sum in an order of partition key. cumulative_counts = accumulate( map(lambda count: count[1], sorted_counts)) # zip it with partition key. sums = dict( zip(map(lambda count: count[0], sorted_counts), cumulative_counts)) # 3. Group by partition id and assign each range. def default_index(pdf): current_partition_max = sums[ pdf["__spark_partition_id"].iloc[0]] offset = len(pdf) pdf["__index_level_0__"] = list( range(current_partition_max - offset, current_partition_max)) return pdf.drop(columns=["__spark_partition_id"]) return_schema = StructType( [StructField("__index_level_0__", LongType())] + list(sdf.schema)) grouped_map_func = pandas_udf( return_schema, PandasUDFType.GROUPED_MAP)(default_index) sdf = sdf.withColumn("__spark_partition_id", F.spark_partition_id()) return sdf.groupBy("__spark_partition_id").apply(grouped_map_func) elif default_index_type == "distributed": scols = [scol_for(sdf, column) for column in sdf.columns] return sdf.select( F.monotonically_increasing_id().alias("__index_level_0__"), *scols) else: raise ValueError( "'compute.default_index_type' should be one of 'sequence'," " 'distributed-sequence' and 'distributed'")
def __init__(self, sdf: spark.DataFrame, index_map: Optional[List[IndexMap]], column_index: Optional[List[Tuple[str, ...]]] = None, column_scols: Optional[List[spark.Column]] = None, column_index_names: Optional[List[str]] = None, scol: Optional[spark.Column] = None) -> None: """ Create a new internal immutable DataFrame to manage Spark DataFrame, column fields and index fields and names. :param sdf: Spark DataFrame to be managed. :param index_map: list of string pair Each pair holds the index field name which exists in Spark fields, and the index name. :param column_index: list of tuples with the same length The multi-level values in the tuples. :param column_scols: list of Spark Column Spark Columns to appear as columns. If scol is not None, this argument is ignored, otherwise if this is None, calculated from sdf. :param column_index_names: Names for each of the index levels. :param scol: Spark Column to be managed. See the examples below to refer what each parameter means. >>> column_index = pd.MultiIndex.from_tuples( ... [('a', 'x'), ('a', 'y'), ('b', 'z')], names=["column_index_a", "column_index_b"]) >>> row_index = pd.MultiIndex.from_tuples( ... [('foo', 'bar'), ('foo', 'bar'), ('zoo', 'bar')], ... names=["row_index_a", "row_index_b"]) >>> kdf = ks.DataFrame( ... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=row_index, columns=column_index) >>> kdf.set_index(('a', 'x'), append=True, inplace=True) >>> kdf # doctest: +NORMALIZE_WHITESPACE column_index_a a b column_index_b y z row_index_a row_index_b (a, x) foo bar 1 2 3 4 5 6 zoo bar 7 8 9 >>> internal = kdf[('a', 'y')]._internal >>> internal._sdf.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS +-----------+-----------+------+------+------+... |row_index_a|row_index_b|(a, x)|(a, y)|(b, z)|... +-----------+-----------+------+------+------+... | foo| bar| 1| 2| 3|... | foo| bar| 4| 5| 6|... | zoo| bar| 7| 8| 9|... +-----------+-----------+------+------+------+... >>> internal._index_map # doctest: +NORMALIZE_WHITESPACE [('row_index_a', ('row_index_a',)), ('row_index_b', ('row_index_b',)), ('(a, x)', ('a', 'x'))] >>> internal._column_index [('a', 'y')] >>> internal._column_scols [Column<b'(a, y)'>] >>> list(internal._column_index_names) ['column_index_a', 'column_index_b'] >>> internal._scol Column<b'(a, y)'> """ assert isinstance(sdf, spark.DataFrame) if NATURAL_ORDER_COLUMN_NAME not in sdf.columns: sdf = sdf.withColumn(NATURAL_ORDER_COLUMN_NAME, F.monotonically_increasing_id()) if index_map is None: # Here is when Koalas DataFrame is created directly from Spark DataFrame. assert not any(SPARK_INDEX_NAME_PATTERN.match(name) for name in sdf.schema.names), \ "Index columns should not appear in columns of the Spark DataFrame. Avoid " \ "index colum names [%s]." % SPARK_INDEX_NAME_PATTERN # Create default index. index_map = [(SPARK_INDEX_NAME_FORMAT(0), None)] sdf = _InternalFrame.attach_default_index(sdf) assert all(isinstance(index_field, str) and (index_name is None or (isinstance(index_name, tuple) and all(isinstance(name, str) for name in index_name))) for index_field, index_name in index_map), index_map assert scol is None or isinstance(scol, spark.Column) assert column_scols is None or all(isinstance(scol, spark.Column) for scol in column_scols) self._sdf = sdf # type: spark.DataFrame self._index_map = index_map # type: List[IndexMap] self._scol = scol # type: Optional[spark.Column] if scol is not None: self._column_scols = [scol] elif column_scols is None: index_columns = set(index_column for index_column, _ in self._index_map) self._column_scols = [scol_for(sdf, col) for col in sdf.columns if col not in index_columns and col not in HIDDEN_COLUMNS] else: self._column_scols = column_scols if scol is not None: assert column_index is not None and len(column_index) == 1, column_index assert all(idx is None or (isinstance(idx, tuple) and len(idx) > 0) for idx in column_index), column_index self._column_index = column_index elif column_index is None: self._column_index = [(sdf.select(scol).columns[0],) for scol in self._column_scols] else: assert len(column_index) == len(self._column_scols), \ (len(column_index), len(self._column_scols)) assert all(isinstance(i, tuple) for i in column_index), column_index assert len(set(len(i) for i in column_index)) <= 1, column_index self._column_index = column_index if column_index_names is not None and not is_list_like(column_index_names): raise ValueError('Column_index_names should be list-like or None for a MultiIndex') if isinstance(column_index_names, list): if all(name is None for name in column_index_names): self._column_index_names = None else: self._column_index_names = column_index_names else: self._column_index_names = column_index_names
def all(self, axis: Union[int, str] = 0) -> bool: """ Return whether all elements are True. Returns True unless there at least one element within a series that is False or equivalent (e.g. zero or empty) Parameters ---------- axis : {0 or 'index'}, default 0 Indicate which axis or axes should be reduced. * 0 / 'index' : reduce the index, return a Series whose index is the original column labels. Examples -------- >>> ks.Series([True, True]).all() True >>> ks.Series([True, False]).all() False >>> ks.Series([0, 1]).all() False >>> ks.Series([1, 2, 3]).all() True >>> ks.Series([True, True, None]).all() True >>> ks.Series([True, False, None]).all() False >>> ks.Series([]).all() True >>> ks.Series([np.nan]).all() True >>> df = ks.Series([True, False, None]).rename("a").to_frame() >>> df.set_index("a").index.all() False """ if axis not in [0, 'index']: raise ValueError('axis should be either 0 or "index" currently.') sdf = self._kdf._sdf.select(self._scol) col = scol_for(sdf, sdf.columns[0]) # Note that we're ignoring `None`s here for now. # any and every was added as of Spark 3.0 # ret = sdf.select(F.expr("every(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0] # Here we use min as its alternative: ret = sdf.select(F.min(F.coalesce(col.cast('boolean'), F.lit(True)))).collect()[0][0] if ret is None: return True else: return ret
def scol_for(self, column_name_or_index: Union[str, Tuple[str, ...]]): """ Return Spark Column for the given column name or index. """ if column_name_or_index in self._column_index_to_scol: return self._column_index_to_scol[column_name_or_index] else: return scol_for(self._sdf, self.column_name_for(column_name_or_index))
def __getitem__(self, key): from databricks.koalas.frame import DataFrame from databricks.koalas.series import Series def raiseNotImplemented(description): raise SparkPandasNotImplementedError( description=description, pandas_function=".loc[..., ...]", spark_target_function="select, where") rows_sel, cols_sel = _unfold(key, self._kser) sdf = self._kdf._sdf if isinstance(rows_sel, Series): sdf_for_check_schema = sdf.select(rows_sel._scol) assert isinstance(sdf_for_check_schema.schema.fields[0].dataType, BooleanType), \ (str(sdf_for_check_schema), sdf_for_check_schema.schema.fields[0].dataType) sdf = sdf.where(rows_sel._scol) elif isinstance(rows_sel, slice): assert len(self._kdf._internal.index_columns) > 0 if rows_sel.step is not None: raiseNotImplemented("Cannot use step with Spark.") if rows_sel == slice(None): # If slice is None - select everything, so nothing to do pass elif len(self._kdf._internal.index_columns) == 1: start = rows_sel.start stop = rows_sel.stop index_column = self._kdf.index.to_series() index_data_type = index_column.spark_type cond = [] if start is not None: cond.append(index_column._scol >= F.lit(start).cast( index_data_type)) if stop is not None: cond.append( index_column._scol <= F.lit(stop).cast(index_data_type) ) if len(cond) > 0: sdf = sdf.where(reduce(lambda x, y: x & y, cond)) else: raiseNotImplemented( "Cannot use slice for MultiIndex with Spark.") elif isinstance(rows_sel, str): raiseNotImplemented( "Cannot use a scalar value for row selection with Spark.") else: try: rows_sel = list(rows_sel) except TypeError: raiseNotImplemented( "Cannot use a scalar value for row selection with Spark.") if len(rows_sel) == 0: sdf = sdf.where(F.lit(False)) elif len(self._kdf._internal.index_columns) == 1: index_column = self._kdf.index.to_series() index_data_type = index_column.spark_type if len(rows_sel) == 1: sdf = sdf.where(index_column._scol == F.lit( rows_sel[0]).cast(index_data_type)) else: sdf = sdf.where( index_column._scol.isin([ F.lit(r).cast(index_data_type) for r in rows_sel ])) else: raiseNotImplemented( "Cannot select with MultiIndex with Spark.") # make cols_sel a 1-tuple of string if a single string column_index = self._kdf._internal.column_index if isinstance(cols_sel, str): kdf = DataFrame(self._kdf._internal.copy(sdf=sdf)) return kdf._get_from_multiindex_column((cols_sel, )) elif isinstance(cols_sel, Series): cols_sel = _make_col(cols_sel) elif isinstance(cols_sel, slice) and cols_sel != slice(None): raise raiseNotImplemented( "Can only select columns either by name or reference or all") elif isinstance(cols_sel, slice) and cols_sel == slice(None): cols_sel = None if cols_sel is None: columns = self._kdf._internal.column_scols elif isinstance(cols_sel, spark.Column): columns = [cols_sel] column_index = None elif all(isinstance(key, Series) for key in cols_sel): columns = [_make_col(key) for key in cols_sel] column_index = [key._internal.column_index[0] for key in cols_sel] elif all(isinstance(key, spark.Column) for key in cols_sel): columns = cols_sel column_index = None elif (any(isinstance(key, str) for key in cols_sel) and any(isinstance(key, tuple) for key in cols_sel)): raise TypeError('Expected tuple, got str') else: if all(isinstance(key, tuple) for key in cols_sel): level = self._kdf._internal.column_index_level if any(len(key) != level for key in cols_sel): raise ValueError( 'All the key level should be the same as column index level.' ) column_to_index = list( zip(self._kdf._internal.data_columns, self._kdf._internal.column_index)) columns = [] column_index = [] for key in cols_sel: found = False for column, idx in column_to_index: if idx == key or idx[0] == key: columns.append(_make_col(column)) column_index.append(idx) found = True if not found: raise KeyError("['{}'] not in index".format(key)) try: sdf = sdf.select(self._kdf._internal.index_scols + columns) index_columns = self._kdf._internal.index_columns data_columns = [ column for column in sdf.columns if column not in index_columns ] column_scols = [scol_for(sdf, col) for col in data_columns] internal = _InternalFrame(sdf=sdf, index_map=self._kdf._internal.index_map, column_index=column_index, column_scols=column_scols) kdf = DataFrame(internal) except AnalysisException: raise KeyError('[{}] don\'t exist in columns'.format( [col._jc.toString() for col in columns])) if cols_sel is not None and isinstance(cols_sel, spark.Column): from databricks.koalas.series import _col return _col(kdf) else: return kdf
def any(self, axis: Union[int, str] = 0) -> bool: """ Return whether any element is True. Returns False unless there at least one element within a series that is True or equivalent (e.g. non-zero or non-empty). Parameters ---------- axis : {0 or 'index'}, default 0 Indicate which axis or axes should be reduced. * 0 / 'index' : reduce the index, return a Series whose index is the original column labels. Examples -------- >>> ks.Series([False, False]).any() False >>> ks.Series([True, False]).any() True >>> ks.Series([0, 0]).any() False >>> ks.Series([0, 1, 2]).any() True >>> ks.Series([False, False, None]).any() False >>> ks.Series([True, False, None]).any() True >>> ks.Series([]).any() False >>> ks.Series([np.nan]).any() False >>> df = ks.Series([True, False, None]).rename("a").to_frame() >>> df.set_index("a").index.any() True """ axis = validate_axis(axis) if axis != 0: raise NotImplementedError( 'axis should be either 0 or "index" currently.') sdf = self._internal.spark_frame.select(self.spark.column) col = scol_for(sdf, sdf.columns[0]) # Note that we're ignoring `None`s here for now. # any and every was added as of Spark 3.0 # ret = sdf.select(F.expr("any(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0] # Here we use max as its alternative: ret = sdf.select(F.max(F.coalesce(col.cast("boolean"), F.lit(False)))).collect()[0][0] if ret is None: return False else: return ret
def attach_id_column(self, id_type: str, column: Union[str, Tuple[str, ...]]) -> "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 = ks.DataFrame({"x": ['a', 'b', 'c']}) >>> df.koalas.attach_id_column(id_type="sequence", column="id") x id 0 a 0 1 b 1 2 c 2 >>> df.koalas.attach_id_column(id_type="distributed-sequence", column="id").sort_index() x id 0 a 0 1 b 1 2 c 2 >>> df.koalas.attach_id_column(id_type="distributed", column="id") ... # doctest: +ELLIPSIS +NORMALIZE_WHITESPACE x id 0 a ... 1 b ... 2 c ... For multi-index columns: >>> df = ks.DataFrame({("x", "y"): ['a', 'b', 'c']}) >>> df.koalas.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 """ from databricks.koalas.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'" ) if isinstance(column, str): column = (column, ) else: assert isinstance(column, tuple), type(column) internal = self._kdf._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 = attach_func(sdf, name_like_string(column)) return DataFrame( InternalFrame( spark_frame=sdf, index_map=OrderedDict([ (SPARK_INDEX_NAME_FORMAT(i), name) for i, name in enumerate(internal.index_names) ]), 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))]), column_label_names=internal.column_label_names, ).resolved_copy)