def var( self, axis=None, skipna=True, ddof=1, split_every=False, dtype=None, out=None, ): axis = self._validate_axis(axis) meta = self._meta_nonempty.var(axis=axis, skipna=skipna) if axis == 1: result = map_partitions( M.var, self, meta=meta, token=self._token_prefix + "var", axis=axis, skipna=skipna, ddof=ddof, ) return handle_out(out, result) else: num = self._get_numeric_data() x = 1.0 * num.sum(skipna=skipna, split_every=split_every) x2 = 1.0 * (num ** 2).sum(skipna=skipna, split_every=split_every) n = num.count(split_every=split_every) name = self._token_prefix + "var" result = map_partitions( var_aggregate, x2, x, n, token=name, meta=meta, ddof=ddof ) if isinstance(self, DataFrame): result.divisions = (min(self.columns), max(self.columns)) return handle_out(out, result)
def apply(self, func, columns=no_default): """ Parallel version of pandas GroupBy.apply This mimics the pandas version except for the following: 1. The user should provide output columns. 2. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. Parameters ---------- func: function Function to apply columns: list, scalar or None If list is given, the result is a DataFrame which columns is specified list. Otherwise, the result is a Series which name is given scalar or None (no name). If name keyword is not given, dask tries to infer the result type using its beggining of data. This inference may take some time and lead to unexpected result Returns ------- applied : Series or DataFrame depending on columns keyword """ if columns is no_default: msg = ( "columns is not specified, inferred from partial data. " "Please provide columns if the result is unexpected.\n" " Before: .apply(func)\n" " After: .apply(func, columns=['x', 'y']) for dataframe result\n" " or: .apply(func, columns='x') for series result") warnings.warn(msg) dummy = self._head().apply(func) columns = dummy.columns if isinstance(dummy, pd.DataFrame) else dummy.name else: dummy = columns columns = self._slice if isinstance(self.index, Series): if self.index._name == self.obj.index._name: df = self.obj else: df = self.obj.set_index(self.index, drop=False, **self.kwargs) return map_partitions(_groupby_apply_level0, dummy, df, columns, func) else: from .shuffle import shuffle df = shuffle(self.obj, self.index, **self.kwargs) return map_partitions(_groupby_apply_index, dummy, df, self.index, columns, func)
def apply(self, func, columns=no_default): """ Parallel version of pandas GroupBy.apply This mimics the pandas version except for the following: 1. The user should provide output columns. 2. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. Parameters ---------- func: function Function to apply columns: list, scalar or None If list is given, the result is a DataFrame which columns is specified list. Otherwise, the result is a Series which name is given scalar or None (no name). If name keyword is not given, dask tries to infer the result type using its beggining of data. This inference may take some time and lead to unexpected result Returns ------- applied : Series or DataFrame depending on columns keyword """ if columns is no_default: msg = ("columns is not specified, inferred from partial data. " "Please provide columns if the result is unexpected.\n" " Before: .apply(func)\n" " After: .apply(func, columns=['x', 'y']) for dataframe result\n" " or: .apply(func, columns='x') for series result") warnings.warn(msg) dummy = self._head().apply(func) columns = dummy.columns if isinstance(dummy, pd.DataFrame) else dummy.name else: dummy = columns columns = self._slice if isinstance(self.index, Series): if self.index._name == self.obj.index._name: df = self.obj else: df = self.obj.set_index(self.index, drop=False, **self.kwargs) return map_partitions(_groupby_apply_level0, dummy, df, columns, func) else: from .shuffle import shuffle df = shuffle(self.obj, self.index, **self.kwargs) return map_partitions(_groupby_apply_index, dummy, df, self.index, columns, func)
def apply(self, func, columns=None): """ Apply function to each group. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. """ # df = set_index(self.df, self.index, **self.kwargs) if self.index._name == self.df.index._name: return map_partitions(_groupby_level0_getitem_apply, self.df, self.key, func, columns=columns) else: from .shuffle import shuffle df = shuffle(self.df, self.index, **self.kwargs) return map_partitions(_groupby_apply, columns or self.df.columns, self.df, self.index, func)
def apply(self, func, columns=None): """ Apply function to each group. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. """ if (isinstance(self.index, Series) and self.index._name == self.df.index._name): return map_partitions(_groupby_apply_level0, columns or self.df.columns, self.df, func) else: from .shuffle import shuffle # df = set_index(self.df, self.index, **self.kwargs) df = shuffle(self.df, self.index, **self.kwargs) return map_partitions(_groupby_apply, columns or self.df.columns, self.df, self.index, func)
def var( self, axis=None, skipna=True, ddof=1, split_every=False, dtype=None, out=None, naive=False, ): axis = self._validate_axis(axis) meta = self._meta_nonempty.var(axis=axis, skipna=skipna) if axis == 1: result = map_partitions( M.var, self, meta=meta, token=self._token_prefix + "var", axis=axis, skipna=skipna, ddof=ddof, ) return handle_out(out, result) elif naive: return _naive_var(self, meta, skipna, ddof, split_every, out) else: return _parallel_var(self, meta, skipna, split_every, out)
def get_group(self, key): token = self._token_prefix + 'get_group' return map_partitions(_groupby_get_group, self.column_info, self.df, self.index, key, self.column_info, token=token)
def meth(self, other, *args, **kwargs): meta = _emulate(op, self, other) return map_partitions(op, self, other, meta=meta, enforce_metadata=False, *args, **kwargs)
def get_group(self, key): token = self._token_prefix + 'get_group' dummy = self._pd.obj if isinstance(dummy, pd.DataFrame) and self._slice is not None: dummy = dummy[self._slice] columns = dummy.columns if isinstance(dummy, pd.DataFrame) else dummy.name return map_partitions(_groupby_get_group, dummy, self.obj, self.index, key, columns, token=token)
def apply(self, func, columns=None): """ Apply function to each group. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. """ # df = set_index(self.df, self.index, **self.kwargs) if self.index._name == self.df.index._name: df = self.df return map_partitions(_groupby_level0_getitem_apply, self.df, self.key, func, columns=columns) else: from .shuffle import shuffle df = shuffle(self.df, self.index, **self.kwargs) return map_partitions(_groupby_apply, columns or self.df.columns, self.df, self.index, func)
def apply(self, func, columns=None): """ Apply function to each group. If the grouper does not align with the index then this causes a full shuffle. The order of rows within each group may not be preserved. """ if (isinstance(self.index, Series) and self.index._name == self.df.index._name): df = self.df return map_partitions(_groupby_apply_level0, columns or self.df.columns, self.df, func) else: from .shuffle import shuffle # df = set_index(self.df, self.index, **self.kwargs) df = shuffle(self.df, self.index, **self.kwargs) return map_partitions(_groupby_apply, columns or self.df.columns, self.df, self.index, func)
def _naive_var(ddf, meta, skipna, ddof, split_every, out): num = ddf._get_numeric_data() x = 1.0 * num.sum(skipna=skipna, split_every=split_every) x2 = 1.0 * (num ** 2).sum(skipna=skipna, split_every=split_every) n = num.count(split_every=split_every) name = ddf._token_prefix + "var" result = map_partitions( var_aggregate, x2, x, n, token=name, meta=meta, ddof=ddof ) if isinstance(ddf, DataFrame): result.divisions = (min(ddf.columns), max(ddf.columns)) return handle_out(out, result)
def set_sorted_index( df: DataFrame, index: Union[str, Series], drop: bool = True, divisions: Optional[Sequence] = None, **kwargs, ) -> DataFrame: if isinstance(index, Series): meta = df._meta.set_index(index._meta, drop=drop) else: meta = df._meta.set_index(index, drop=drop) result = map_partitions( M.set_index, df, index, drop=drop, meta=meta, align_dataframes=False, transform_divisions=False, ) if not divisions: return compute_and_set_divisions(result, **kwargs) elif len(divisions) != len(df.divisions): msg = ( "When doing `df.set_index(col, sorted=True, divisions=...)`, " "divisions indicates known splits in the index column. In this " "case divisions must be the same length as the existing " "divisions in `df`\n\n" "If the intent is to repartition into new divisions after " "setting the index, you probably want:\n\n" "`df.set_index(col, sorted=True).repartition(divisions=divisions)`" ) raise ValueError(msg) result.divisions = tuple(divisions) return result
def get_group(self, key): token = self._token_prefix + "get_group" return map_partitions( _groupby_get_group, self.column_info, self.df, self.index, key, self.column_info, token=token )
def get_dummies( data, prefix=None, prefix_sep="_", dummy_na=False, columns=None, sparse=False, drop_first=False, dtype=np.uint8, **kwargs, ): """ Convert categorical variable into dummy/indicator variables. Data must have category dtype to infer result's ``columns``. Parameters ---------- data : Series, or DataFrame For Series, the dtype must be categorical. For DataFrame, at least one column must be categorical. prefix : string, list of strings, or dict of strings, default None String to append DataFrame column names. Pass a list with length equal to the number of columns when calling get_dummies on a DataFrame. Alternatively, `prefix` can be a dictionary mapping column names to prefixes. prefix_sep : string, default '_' If appending prefix, separator/delimiter to use. Or pass a list or dictionary as with `prefix.` dummy_na : bool, default False Add a column to indicate NaNs, if False NaNs are ignored. columns : list-like, default None Column names in the DataFrame to be encoded. If `columns` is None then all the columns with `category` dtype will be converted. sparse : bool, default False Whether the dummy columns should be sparse or not. Returns SparseDataFrame if `data` is a Series or if all columns are included. Otherwise returns a DataFrame with some SparseBlocks. .. versionadded:: 0.18.2 drop_first : bool, default False Whether to get k-1 dummies out of k categorical levels by removing the first level. dtype : dtype, default np.uint8 Data type for new columns. Only a single dtype is allowed. .. versionadded:: 0.18.2 Returns ------- dummies : DataFrame Examples -------- Dask's version only works with Categorical data, as this is the only way to know the output shape without computing all the data. >>> import pandas as pd >>> import dask.dataframe as dd >>> s = dd.from_pandas(pd.Series(list('abca')), npartitions=2) >>> dd.get_dummies(s) Traceback (most recent call last): ... NotImplementedError: `get_dummies` with non-categorical dtypes is not supported... With categorical data: >>> s = dd.from_pandas(pd.Series(list('abca'), dtype='category'), npartitions=2) >>> dd.get_dummies(s) # doctest: +NORMALIZE_WHITESPACE Dask DataFrame Structure: a b c npartitions=2 0 uint8 uint8 uint8 2 ... ... ... 3 ... ... ... Dask Name: get_dummies, 4 tasks >>> dd.get_dummies(s).compute() # doctest: +ELLIPSIS a b c 0 1 0 0 1 0 1 0 2 0 0 1 3 1 0 0 See Also -------- pandas.get_dummies """ if isinstance(data, (pd.Series, pd.DataFrame)): return pd.get_dummies( data, prefix=prefix, prefix_sep=prefix_sep, dummy_na=dummy_na, columns=columns, sparse=sparse, drop_first=drop_first, dtype=dtype, **kwargs, ) not_cat_msg = ("`get_dummies` with non-categorical dtypes is not " "supported. Please use `df.categorize()` beforehand to " "convert to categorical dtype.") unknown_cat_msg = ("`get_dummies` with unknown categories is not " "supported. Please use `column.cat.as_known()` or " "`df.categorize()` beforehand to ensure known " "categories") if isinstance(data, Series): if not methods.is_categorical_dtype(data): raise NotImplementedError(not_cat_msg) if not has_known_categories(data): raise NotImplementedError(unknown_cat_msg) elif isinstance(data, DataFrame): if columns is None: if (data.dtypes == "object").any(): raise NotImplementedError(not_cat_msg) columns = data._meta.select_dtypes(include=["category"]).columns else: if not all(methods.is_categorical_dtype(data[c]) for c in columns): raise NotImplementedError(not_cat_msg) if not all(has_known_categories(data[c]) for c in columns): raise NotImplementedError(unknown_cat_msg) package_name = data._meta.__class__.__module__.split(".")[0] dummies = sys.modules[package_name].get_dummies return map_partitions( dummies, data, prefix=prefix, prefix_sep=prefix_sep, dummy_na=dummy_na, columns=columns, sparse=sparse, drop_first=drop_first, dtype=dtype, **kwargs, )
def std(self, ddof=1): v = self.var(ddof) result = map_partitions(np.sqrt, v, v) return result