def check_matching_columns(meta, actual): # Need nan_to_num otherwise nan comparison gives False if not np.array_equal(np.nan_to_num(meta.columns), np.nan_to_num(actual.columns)): extra = methods.tolist(actual.columns.difference(meta.columns)) missing = methods.tolist(meta.columns.difference(actual.columns)) if extra or missing: extra_info = f" Extra: {extra}\n Missing: {missing}" else: extra_info = "Order of columns does not match" raise ValueError("The columns in the computed data do not match" " the columns in the provided metadata\n" f"{extra_info}")
def _resample_bin_and_out_divs(divisions, rule, closed="left", label="left"): rule = pd.tseries.frequencies.to_offset(rule) g = pd.Grouper(freq=rule, how="count", closed=closed, label=label) # Determine bins to apply `how` to. Disregard labeling scheme. divs = pd.Series(range(len(divisions)), index=divisions) temp = divs.resample(rule, closed=closed, label="left").count() tempdivs = temp.loc[temp > 0].index # Cleanup closed == 'right' and label == 'right' res = pd.offsets.Nano() if hasattr(rule, "delta") else pd.offsets.Day() if g.closed == "right": newdivs = tempdivs + res else: newdivs = tempdivs if g.label == "right": outdivs = tempdivs + rule else: outdivs = tempdivs newdivs = methods.tolist(newdivs) outdivs = methods.tolist(outdivs) # Adjust ends if newdivs[0] < divisions[0]: newdivs[0] = divisions[0] if newdivs[-1] < divisions[-1]: if len(newdivs) < len(divs): setter = lambda a, val: a.append(val) else: setter = lambda a, val: a.__setitem__(-1, val) setter(newdivs, divisions[-1] + res) if outdivs[-1] > divisions[-1]: setter(outdivs, outdivs[-1]) elif outdivs[-1] < divisions[-1]: setter(outdivs, temp.index[-1]) return tuple(map(pd.Timestamp, newdivs)), tuple(map(pd.Timestamp, outdivs))
def _maybe_sort(a, check_index: bool): # sort by value, then index try: if is_dataframe_like(a): if set(a.index.names) & set(a.columns): a.index.names = [ "-overlapped-index-name-%d" % i for i in range(len(a.index.names)) ] a = a.sort_values(by=methods.tolist(a.columns)) else: a = a.sort_values() except (TypeError, IndexError, ValueError): pass return a.sort_index() if check_index else a
def sort_values( df, by, max_branch=None, divisions=None, set_divisions=False, ignore_index=False, ): """ Sort by the given list/tuple of column names. """ npartitions = df.npartitions if isinstance(by, tuple): by = list(by) elif not isinstance(by, list): by = [by] # Step 1 - Calculate new divisions (if necessary) if divisions is None: divisions = quantile_divisions(df, by, npartitions) # Step 2 - Perform repartitioning shuffle meta = df._meta._constructor_sliced([0]) if not isinstance(divisions, (gd.Series, gd.DataFrame)): dtype = df[by[0]].dtype divisions = df._meta._constructor_sliced(divisions, dtype=dtype) partitions = df[by].map_partitions( _set_partitions_pre, divisions=divisions, meta=meta ) df2 = df.assign(_partitions=partitions) df3 = rearrange_by_column( df2, "_partitions", max_branch=max_branch, npartitions=len(divisions) - 1, shuffle="tasks", ignore_index=ignore_index, ).drop(columns=["_partitions"]) df3.divisions = (None,) * (df3.npartitions + 1) # Step 3 - Return final sorted df df4 = df3.map_partitions(M.sort_values, by) if not isinstance(divisions, gd.DataFrame) and set_divisions: # Can't have multi-column divisions elsewhere in dask (yet) df4.divisions = methods.tolist(divisions) return df4
def _compute_partition_stats(column: Series, allow_overlap: bool = False, **kwargs) -> Tuple[List, List, List[int]]: """For a given column, compute the min, max, and len of each partition. And make sure that the partitions are sorted relative to each other. NOTE: this does not guarantee that every partition is internally sorted. """ mins = column.map_partitions(M.min, meta=column) maxes = column.map_partitions(M.max, meta=column) lens = column.map_partitions(len, meta=column) mins, maxes, lens = compute(mins, maxes, lens, **kwargs) mins = remove_nans(mins) maxes = remove_nans(maxes) non_empty_mins = [m for m, length in zip(mins, lens) if length != 0] non_empty_maxes = [m for m, length in zip(maxes, lens) if length != 0] if (sorted(non_empty_mins) != non_empty_mins or sorted(non_empty_maxes) != non_empty_maxes): raise ValueError( f"Partitions are not sorted ascending by {column.name or 'the index'}", f"In your dataset the (min, max, len) values of {column.name or 'the index'} " f"for each partition are : {list(zip(mins, maxes, lens))}", ) if not allow_overlap and any( a <= b for a, b in zip(non_empty_mins[1:], non_empty_maxes[:-1])): warnings.warn( "Partitions have overlapping values, so divisions are non-unique." "Use `set_index(sorted=True)` with no `divisions` to allow dask to fix the overlap. " f"In your dataset the (min, max, len) values of {column.name or 'the index'} " f"for each partition are : {list(zip(mins, maxes, lens))}", UserWarning, ) lens = methods.tolist(lens) if not allow_overlap: return (mins, maxes, lens) else: return (non_empty_mins, non_empty_maxes, lens)
def _calculate_divisions( df: DataFrame, partition_col: Series, repartition: bool, npartitions: int, upsample: float = 1.0, partition_size: float = 128e6, ) -> Tuple[List, List, List]: """ Utility function to calculate divisions for calls to `map_partitions` """ sizes = df.map_partitions(sizeof) if repartition else [] divisions = partition_col._repartition_quantiles(npartitions, upsample=upsample) mins = partition_col.map_partitions(M.min) maxes = partition_col.map_partitions(M.max) try: divisions, sizes, mins, maxes = compute(divisions, sizes, mins, maxes) except TypeError as e: # When there are nulls and a column is non-numeric, a TypeError is sometimes raised as a result of # 1) computing mins/maxes above, 2) every null being switched to NaN, and 3) NaN being a float. # Also, Pandas ExtensionDtypes may cause TypeErrors when dealing with special nulls such as pd.NaT or pd.NA. # If this happens, we hint the user about eliminating nulls beforehand. if not is_numeric_dtype(partition_col.dtype): obj, suggested_method = ( ("column", f"`.dropna(subset=['{partition_col.name}'])`") if any( partition_col._name == df[c]._name for c in df) else ("series", "`.loc[series[~series.isna()]]`")) raise NotImplementedError( f"Divisions calculation failed for non-numeric {obj} '{partition_col.name}'.\n" f"This is probably due to the presence of nulls, which Dask does not entirely support in the index.\n" f"We suggest you try with {suggested_method}.") from e # For numeric types there shouldn't be problems with nulls, so we raise as-it-is this particular TypeError else: raise e divisions = methods.tolist(divisions) if type(sizes) is not list: sizes = methods.tolist(sizes) mins = methods.tolist(mins) maxes = methods.tolist(maxes) empty_dataframe_detected = pd.isna(divisions).all() if repartition or empty_dataframe_detected: total = sum(sizes) npartitions = max(math.ceil(total / partition_size), 1) npartitions = min(npartitions, df.npartitions) n = len(divisions) try: divisions = np.interp( x=np.linspace(0, n - 1, npartitions + 1), xp=np.linspace(0, n - 1, n), fp=divisions, ).tolist() except (TypeError, ValueError): # str type indexes = np.linspace(0, n - 1, npartitions + 1).astype(int) divisions = [divisions[i] for i in indexes] else: # Drop duplicate divisions returned by partition quantiles divisions = list(toolz.unique(divisions[:-1])) + [divisions[-1]] mins = remove_nans(mins) maxes = remove_nans(maxes) if pd.api.types.is_categorical_dtype(partition_col.dtype): dtype = partition_col.dtype mins = pd.Categorical(mins, dtype=dtype).codes.tolist() maxes = pd.Categorical(maxes, dtype=dtype).codes.tolist() return divisions, mins, maxes
def set_partition( df: DataFrame, index: Union[str, Series], divisions: Sequence, max_branch: int = 32, drop: bool = True, shuffle: Optional[str] = None, compute: Optional[bool] = None, ) -> DataFrame: """Group DataFrame by index Sets a new index and partitions data along that index according to divisions. Divisions are often found by computing approximate quantiles. The function ``set_index`` will do both of these steps. Parameters ---------- df: DataFrame/Series Data that we want to re-partition index: string or Series Column to become the new index divisions: list Values to form new divisions between partitions drop: bool, default True Whether to delete columns to be used as the new index shuffle: str (optional) Either 'disk' for an on-disk shuffle or 'tasks' to use the task scheduling framework. Use 'disk' if you are on a single machine and 'tasks' if you are on a distributed cluster. max_branch: int (optional) If using the task-based shuffle, the amount of splitting each partition undergoes. Increase this for fewer copies but more scheduler overhead. See Also -------- set_index shuffle partd """ meta = df._meta._constructor_sliced([0]) if isinstance(divisions, tuple): # pd.isna considers tuples to be scalars. Convert to a list. divisions = list(divisions) if not isinstance(index, Series): dtype = df[index].dtype else: dtype = index.dtype if pd.isna(divisions).any() and pd.api.types.is_integer_dtype(dtype): # Can't construct a Series[int64] when any / all of the divisions are NaN. divisions = df._meta._constructor_sliced(divisions) elif (pd.api.types.is_categorical_dtype(dtype) and UNKNOWN_CATEGORIES in dtype.categories): # If categories are unknown, leave as a string dtype instead. divisions = df._meta._constructor_sliced(divisions) else: divisions = df._meta._constructor_sliced(divisions, dtype=dtype) if not isinstance(index, Series): partitions = df[index].map_partitions(set_partitions_pre, divisions=divisions, meta=meta) df2 = df.assign(_partitions=partitions) else: partitions = index.map_partitions(set_partitions_pre, divisions=divisions, meta=meta) df2 = df.assign(_partitions=partitions, _index=index) df3 = rearrange_by_column( df2, "_partitions", max_branch=max_branch, npartitions=len(divisions) - 1, shuffle=shuffle, compute=compute, ignore_index=True, ) if not isinstance(index, Series): df4 = df3.map_partitions( set_index_post_scalar, index_name=index, drop=drop, column_dtype=df.columns.dtype, ) else: df4 = df3.map_partitions( set_index_post_series, index_name=index.name, drop=drop, column_dtype=df.columns.dtype, ) df4.divisions = tuple(methods.tolist(divisions)) return df4.map_partitions(M.sort_index)
def _old_read_sql_table( table, uri, index_col, divisions=None, npartitions=None, limits=None, columns=None, bytes_per_chunk="256 MiB", head_rows=5, schema=None, meta=None, engine_kwargs=None, **kwargs, ): """ Create dataframe from an SQL table. If neither divisions or npartitions is given, the memory footprint of the first few rows will be determined, and partitions of size ~256MB will be used. Parameters ---------- table : string or sqlalchemy expression Select columns from here. uri : string Full sqlalchemy URI for the database connection index_col : string Column which becomes the index, and defines the partitioning. Should be a indexed column in the SQL server, and any orderable type. If the type is number or time, then partition boundaries can be inferred from npartitions or bytes_per_chunk; otherwide must supply explicit ``divisions=``. ``index_col`` could be a function to return a value, e.g., ``sql.func.abs(sql.column('value')).label('abs(value)')``. ``index_col=sql.func.abs(sql.column("value")).label("abs(value)")``, or ``index_col=cast(sql.column("id"),types.BigInteger).label("id")`` to convert the textfield ``id`` to ``BigInteger``. Note ``sql``, ``cast``, ``types`` methods comes from ``sqlalchemy`` module. Labeling columns created by functions or arithmetic operations is required. divisions: sequence Values of the index column to split the table by. If given, this will override npartitions and bytes_per_chunk. The divisions are the value boundaries of the index column used to define the partitions. For example, ``divisions=list('acegikmoqsuwz')`` could be used to partition a string column lexographically into 12 partitions, with the implicit assumption that each partition contains similar numbers of records. npartitions : int Number of partitions, if divisions is not given. Will split the values of the index column linearly between limits, if given, or the column max/min. The index column must be numeric or time for this to work limits: 2-tuple or None Manually give upper and lower range of values for use with npartitions; if None, first fetches max/min from the DB. Upper limit, if given, is inclusive. columns : list of strings or None Which columns to select; if None, gets all; can include sqlalchemy functions, e.g., ``sql.func.abs(sql.column('value')).label('abs(value)')``. Labeling columns created by functions or arithmetic operations is recommended. bytes_per_chunk : str, int If both divisions and npartitions is None, this is the target size of each partition, in bytes head_rows : int How many rows to load for inferring the data-types, unless passing meta meta : empty DataFrame or None If provided, do not attempt to infer dtypes, but use these, coercing all chunks on load schema : str or None If using a table name, pass this to sqlalchemy to select which DB schema to use within the URI connection engine_kwargs : dict or None Specific db engine parameters for sqlalchemy kwargs : dict Additional parameters to pass to `pd.read_sql()` Returns ------- dask.dataframe Examples -------- >>> df = dd.read_sql_table('accounts', 'sqlite:///path/to/bank.db', ... npartitions=10, index_col='id') # doctest: +SKIP """ import sqlalchemy as sa from sqlalchemy import sql from sqlalchemy.sql import elements warnings.warn( "You are using a compatibility version of `read_sql_table` that will be " "removed in a future version of dask. This function existst to support " "old versions of SQLAlchemy (< 1.4). This compatibility function is less " "stable than the new version. We recommend you update your code.", DeprecationWarning, ) if index_col is None: raise ValueError("Must specify index column to partition on") engine_kwargs = {} if engine_kwargs is None else engine_kwargs engine = sa.create_engine(uri, **engine_kwargs) m = sa.MetaData() if isinstance(table, str): table = sa.Table(table, m, autoload=True, autoload_with=engine, schema=schema) index = table.columns[index_col] if isinstance(index_col, str) else index_col if not isinstance(index_col, (str, elements.Label)): raise ValueError( "Use label when passing an SQLAlchemy instance as the index (%s)" % index) if divisions and npartitions: raise TypeError( "Must supply either divisions or npartitions, not both") columns = ([(table.columns[c] if isinstance(c, str) else c) for c in columns] if columns else list(table.columns)) if index not in columns: columns.append(index) if isinstance(index_col, str): kwargs["index_col"] = index_col else: # function names get pandas auto-named kwargs["index_col"] = index_col.name if head_rows > 0: # derive metadata from first few rows q = sql.select(columns).limit(head_rows).select_from(table) head = pd.read_sql(q, engine, **kwargs) if len(head) == 0: # no results at all name = table.name schema = table.schema head = pd.read_sql_table(name, uri, schema=schema, index_col=index_col) return from_pandas(head, npartitions=1) bytes_per_row = (head.memory_usage(deep=True, index=True)).sum() / head_rows if meta is None: meta = head.iloc[:0] elif meta is None: raise ValueError("Must provide meta if head_rows is 0") else: if divisions is None and npartitions is None: raise ValueError( "Must provide divisions or npartitions when using explicit meta." ) if divisions is None: if limits is None: # calculate max and min for given index q = sql.select([sql.func.max(index), sql.func.min(index)]).select_from(table) minmax = pd.read_sql(q, engine) maxi, mini = minmax.iloc[0] dtype = minmax.dtypes["max_1"] else: mini, maxi = limits dtype = pd.Series(limits).dtype if npartitions is None: q = sql.select([sql.func.count(index)]).select_from(table) count = pd.read_sql(q, engine)["count_1"][0] npartitions = (int( round(count * bytes_per_row / parse_bytes(bytes_per_chunk))) or 1) if dtype.kind == "M": divisions = methods.tolist( pd.date_range( start=mini, end=maxi, freq="%iS" % ((maxi - mini).total_seconds() / npartitions), )) divisions[0] = mini divisions[-1] = maxi elif dtype.kind in ["i", "u", "f"]: divisions = np.linspace(mini, maxi, npartitions + 1).tolist() else: raise TypeError( 'Provided index column is of type "{}". If divisions is not provided the ' "index column type must be numeric or datetime.".format(dtype)) parts = [] lowers, uppers = divisions[:-1], divisions[1:] for i, (lower, upper) in enumerate(zip(lowers, uppers)): cond = index <= upper if i == len(lowers) - 1 else index < upper q = sql.select(columns).where(sql.and_(index >= lower, cond)).select_from(table) parts.append( delayed(_read_sql_chunk)(q, uri, meta, engine_kwargs=engine_kwargs, **kwargs)) engine.dispose() return from_delayed(parts, meta, divisions=divisions)
def read_sql_query( sql, con, index_col, divisions=None, npartitions=None, limits=None, bytes_per_chunk="256 MiB", head_rows=5, meta=None, engine_kwargs=None, **kwargs, ): """ Read SQL query into a DataFrame. If neither ``divisions`` or ``npartitions`` is given, the memory footprint of the first few rows will be determined, and partitions of size ~256MB will be used. Parameters ---------- sql : SQLAlchemy Selectable SQL query to be executed. TextClause is not supported con : str Full sqlalchemy URI for the database connection index_col : str Column which becomes the index, and defines the partitioning. Should be a indexed column in the SQL server, and any orderable type. If the type is number or time, then partition boundaries can be inferred from ``npartitions`` or ``bytes_per_chunk``; otherwise must supply explicit ``divisions``. divisions: sequence Values of the index column to split the table by. If given, this will override ``npartitions`` and ``bytes_per_chunk``. The divisions are the value boundaries of the index column used to define the partitions. For example, ``divisions=list('acegikmoqsuwz')`` could be used to partition a string column lexographically into 12 partitions, with the implicit assumption that each partition contains similar numbers of records. npartitions : int Number of partitions, if ``divisions`` is not given. Will split the values of the index column linearly between ``limits``, if given, or the column max/min. The index column must be numeric or time for this to work limits: 2-tuple or None Manually give upper and lower range of values for use with ``npartitions``; if None, first fetches max/min from the DB. Upper limit, if given, is inclusive. bytes_per_chunk : str or int If both ``divisions`` and ``npartitions`` is None, this is the target size of each partition, in bytes head_rows : int How many rows to load for inferring the data-types, and memory per row meta : empty DataFrame or None If provided, do not attempt to infer dtypes, but use these, coercing all chunks on load engine_kwargs : dict or None Specific db engine parameters for sqlalchemy kwargs : dict Additional parameters to pass to `pd.read_sql()` Returns ------- dask.dataframe See Also -------- read_sql_table : Read SQL database table into a DataFrame. """ import sqlalchemy as sa if not isinstance(con, str): raise TypeError( "'con' must be of type str, not " + str(type(con)) + "Note: Dask does not support SQLAlchemy connectables here") if index_col is None: raise ValueError("Must specify index column to partition on") if not isinstance(index_col, (str, sa.Column, sa.sql.elements.ColumnClause)): raise ValueError("'index_col' must be of type str or sa.Column, not " + str(type(index_col))) if not head_rows > 0: if meta is None: raise ValueError("Must provide 'meta' if 'head_rows' is 0") if divisions is None and npartitions is None: raise ValueError( "Must provide 'divisions' or 'npartitions' if 'head_rows' is 0" ) if divisions and npartitions: raise TypeError( "Must supply either 'divisions' or 'npartitions', not both") engine_kwargs = {} if engine_kwargs is None else engine_kwargs engine = sa.create_engine(con, **engine_kwargs) index = (sa.Column(index_col) if isinstance(index_col, str) else sa.Column( index_col.name, index_col.type)) kwargs["index_col"] = index.name if head_rows > 0: # derive metadata from first few rows q = sql.limit(head_rows) head = pd.read_sql(q, engine, **kwargs) if len(head) == 0: # no results at all return from_pandas(head, npartitions=1) bytes_per_row = (head.memory_usage(deep=True, index=True)).sum() / head_rows if meta is None: meta = head.iloc[:0] if divisions is None: if limits is None: # calculate max and min for given index q = sa.sql.select([sa.sql.func.max(index), sa.sql.func.min(index) ]).select_from(sql.subquery()) minmax = pd.read_sql(q, engine) maxi, mini = minmax.iloc[0] dtype = minmax.dtypes["max_1"] else: mini, maxi = limits dtype = pd.Series(limits).dtype if npartitions is None: q = sa.sql.select([sa.sql.func.count(index) ]).select_from(sql.subquery()) count = pd.read_sql(q, engine)["count_1"][0] npartitions = (int( round(count * bytes_per_row / parse_bytes(bytes_per_chunk))) or 1) if dtype.kind == "M": divisions = methods.tolist( pd.date_range( start=mini, end=maxi, freq="%iS" % ((maxi - mini).total_seconds() / npartitions), )) divisions[0] = mini divisions[-1] = maxi elif dtype.kind in ["i", "u", "f"]: divisions = np.linspace(mini, maxi, npartitions + 1, dtype=dtype).tolist() else: raise TypeError( 'Provided index column is of type "{}". If divisions is not provided the ' "index column type must be numeric or datetime.".format(dtype)) parts = [] lowers, uppers = divisions[:-1], divisions[1:] for i, (lower, upper) in enumerate(zip(lowers, uppers)): cond = index <= upper if i == len(lowers) - 1 else index < upper q = sql.where(sa.sql.and_(index >= lower, cond)) parts.append( delayed(_read_sql_chunk)(q, con, meta, engine_kwargs=engine_kwargs, **kwargs)) engine.dispose() return from_delayed(parts, meta, divisions=divisions)
def sort_values( df, by, max_branch=None, divisions=None, set_divisions=False, ignore_index=False, ascending=True, na_position="last", sort_function=None, sort_function_kwargs=None, ): """Sort by the given list/tuple of column names.""" if not isinstance(ascending, bool): raise ValueError("ascending must be either True or False") if na_position not in ("first", "last"): raise ValueError("na_position must be either 'first' or 'last'") npartitions = df.npartitions if isinstance(by, tuple): by = list(by) elif not isinstance(by, list): by = [by] # parse custom sort function / kwargs if provided sort_kwargs = { "by": by, "ascending": ascending, "na_position": na_position, } if sort_function is None: sort_function = M.sort_values if sort_function_kwargs is not None: sort_kwargs.update(sort_function_kwargs) # handle single partition case if npartitions == 1: return df.map_partitions(sort_function, **sort_kwargs) # Step 1 - Calculate new divisions (if necessary) if divisions is None: divisions = quantile_divisions(df, by, npartitions) # Step 2 - Perform repartitioning shuffle meta = df._meta._constructor_sliced([0]) if not isinstance(divisions, (gd.Series, gd.DataFrame)): dtype = df[by[0]].dtype divisions = df._meta._constructor_sliced(divisions, dtype=dtype) partitions = df[by].map_partitions( _set_partitions_pre, divisions=divisions, ascending=ascending, na_position=na_position, meta=meta, ) df2 = df.assign(_partitions=partitions) df3 = rearrange_by_column( df2, "_partitions", max_branch=max_branch, npartitions=len(divisions) - 1, shuffle="tasks", ignore_index=ignore_index, ).drop(columns=["_partitions"]) df3.divisions = (None, ) * (df3.npartitions + 1) # Step 3 - Return final sorted df df4 = df3.map_partitions(sort_function, **sort_kwargs) if not isinstance(divisions, gd.DataFrame) and set_divisions: # Can't have multi-column divisions elsewhere in dask (yet) df4.divisions = methods.tolist(divisions) return df4