1""" 2Dispatch in dask.dataframe. 3 4Also see extension.py 5""" 6 7import pandas as pd 8 9import dask.array as da 10import dask.dataframe as dd 11 12from ..utils import Dispatch 13 14make_meta_dispatch = Dispatch("make_meta_dispatch") 15make_meta_obj = Dispatch("make_meta_obj") 16meta_nonempty = Dispatch("meta_nonempty") 17hash_object_dispatch = Dispatch("hash_object_dispatch") 18group_split_dispatch = Dispatch("group_split_dispatch") 19get_parallel_type = Dispatch("get_parallel_type") 20categorical_dtype_dispatch = Dispatch("CategoricalDtype") 21concat_dispatch = Dispatch("concat") 22tolist_dispatch = Dispatch("tolist") 23is_categorical_dtype_dispatch = Dispatch("is_categorical_dtype") 24union_categoricals_dispatch = Dispatch("union_categoricals") 25 26 27def concat( 28 dfs, 29 axis=0, 30 join="outer", 31 uniform=False, 32 filter_warning=True, 33 ignore_index=False, 34 **kwargs, 35): 36 """Concatenate, handling some edge cases: 37 38 - Unions categoricals between partitions 39 - Ignores empty partitions 40 41 Parameters 42 ---------- 43 dfs : list of DataFrame, Series, or Index 44 axis : int or str, optional 45 join : str, optional 46 uniform : bool, optional 47 Whether to treat ``dfs[0]`` as representative of ``dfs[1:]``. Set to 48 True if all arguments have the same columns and dtypes (but not 49 necessarily categories). Default is False. 50 ignore_index : bool, optional 51 Whether to allow index values to be ignored/dropped during 52 concatenation. Default is False. 53 ignore_order : bool, optional 54 Whether to ignore the order when doing the union of categoricals. 55 Default is False. 56 """ 57 if len(dfs) == 1: 58 return dfs[0] 59 else: 60 func = concat_dispatch.dispatch(type(dfs[0])) 61 return func( 62 dfs, 63 axis=axis, 64 join=join, 65 uniform=uniform, 66 filter_warning=filter_warning, 67 ignore_index=ignore_index, 68 **kwargs, 69 ) 70 71 72def is_categorical_dtype(obj): 73 obj = getattr(obj, "dtype", obj) 74 func = is_categorical_dtype_dispatch.dispatch(type(obj)) 75 return func(obj) 76 77 78def categorical_dtype(meta, categories=None, ordered=False): 79 func = categorical_dtype_dispatch.dispatch(type(meta)) 80 return func(categories=categories, ordered=ordered) 81 82 83def tolist(obj): 84 func = tolist_dispatch.dispatch(type(obj)) 85 return func(obj) 86 87 88def make_meta(x, index=None, parent_meta=None): 89 """ 90 This method creates meta-data based on the type of ``x``, 91 and ``parent_meta`` if supplied. 92 93 Parameters 94 ---------- 95 x : Object of any type. 96 Object to construct meta-data from. 97 index : Index, optional 98 Any index to use in the metadata. This is a pass-through 99 parameter to dispatches registered. 100 parent_meta : Object, default None 101 If ``x`` is of arbitrary types and thus Dask cannot determine 102 which back-end to be used to generate the meta-data for this 103 object type, in which case ``parent_meta`` will be used to 104 determine which back-end to select and dispatch to. To use 105 utilize this parameter ``make_meta_obj`` has be dispatched. 106 If ``parent_meta`` is ``None``, a pandas DataFrame is used for 107 ``parent_meta`` thats chooses pandas as the backend. 108 109 Returns 110 ------- 111 A valid meta-data 112 """ 113 114 if isinstance( 115 x, 116 ( 117 dd._Frame, 118 dd.core.Scalar, 119 dd.groupby._GroupBy, 120 dd.accessor.Accessor, 121 da.Array, 122 ), 123 ): 124 return x._meta 125 126 try: 127 return make_meta_dispatch(x, index=index) 128 except TypeError: 129 if parent_meta is not None: 130 func = make_meta_obj.dispatch(type(parent_meta)) 131 return func(x, index=index) 132 else: 133 # Default to using the pandas backend 134 # if ``parent_meta`` is not specified 135 func = make_meta_obj.dispatch(pd.DataFrame) 136 return func(x, index=index) 137 138 139def union_categoricals(to_union, sort_categories=False, ignore_order=False): 140 func = union_categoricals_dispatch.dispatch(type(to_union[0])) 141 return func(to_union, sort_categories=sort_categories, ignore_order=ignore_order) 142