def repartition(self, df: DataFrame, partition_spec: PartitionSpec) -> DataFrame: def _persist_and_count(df: DataFrame) -> int: df = self.persist(df) return df.count() df = self.to_df(df) num_funcs = {KEYWORD_ROWCOUNT: lambda: _persist_and_count(df)} num = partition_spec.get_num_partitions(**num_funcs) if partition_spec.algo == "hash": sdf = hash_repartition(self.spark_session, df.native, num, partition_spec.partition_by) elif partition_spec.algo == "rand": sdf = rand_repartition(self.spark_session, df.native, num, partition_spec.partition_by) elif partition_spec.algo == "even": df = self.persist(df) sdf = even_repartition(self.spark_session, df.native, num, partition_spec.partition_by) else: # pragma: no cover raise NotImplementedError(partition_spec.algo + " is not supported") sorts = partition_spec.get_sorts(df.schema) if len(sorts) > 0: sdf = sdf.sortWithinPartitions(*sorts.keys(), ascending=list(sorts.values())) return self.to_df(sdf, df.schema, df.metadata)
def test_map_with_special_values(self): def with_nat(cursor, data): df = data.as_pandas() df["nat"] = pd.NaT schema = data.schema + "nat:datetime" return PandasDataFrame(df, schema) e = self.engine # test datetime with nat dt = datetime.now() o = ArrayDataFrame( [[dt, 2], [None, 2], [None, 1], [dt, 5], [None, 4]], "a:datetime,b:int", dict(a=1), ) c = e.map(o, select_top, o.schema, PartitionSpec(by=["a"], presort="b DESC")) df_eq(c, [[None, 4], [dt, 5]], "a:datetime,b:int", throw=True) d = e.map(c, with_nat, "a:datetime,b:int,nat:datetime", PartitionSpec()) df_eq( d, [[None, 4, None], [dt, 5, None]], "a:datetime,b:int,nat:datetime", throw=True, ) # test list o = ArrayDataFrame([[dt, [1, 2]]], "a:datetime,b:[int]") c = e.map(o, select_top, o.schema, PartitionSpec(by=["a"])) df_eq(c, o, check_order=True, throw=True)
def test_worflow_dataframes(): dag1 = FugueWorkflow() df1 = dag1.df([[0]], "a:int") df2 = dag1.df([[0]], "b:int") dag2 = FugueWorkflow() df3 = dag2.df([[0]], "a:int") dfs1 = WorkflowDataFrames(a=df1, b=df2) assert dfs1["a"] is df1 assert dfs1["b"] is df2 dfs2 = WorkflowDataFrames(dfs1, aa=df1, bb=df2) assert 4 == len(dfs2) with raises(ValueError): WorkflowDataFrames(a=df1, b=df3) with raises(ValueError): WorkflowDataFrames(a=df1, b=ArrayDataFrame([[0]], "a:int")) dag = FugueWorkflow() df = dag.df([[0, 1], [1, 1]], "a:int,b:int") assert df.partition_spec.empty df2 = df.partition(by=["a"]) assert df.partition_spec.empty assert df2.partition_spec == PartitionSpec(by=["a"]) df3 = df.partition_by("a", "b") assert df.partition_spec.empty assert df3.partition_spec == PartitionSpec(by=["a", "b"]) df4 = df.per_partition_by("a", "b") assert df.partition_spec.empty assert df4.partition_spec == PartitionSpec(by=["a", "b"], algo="even") df4 = df.per_row() assert df.partition_spec.empty assert df4.partition_spec == PartitionSpec("per_row")
def _serialize_by_partition( self, df: DataFrame, partition_spec: PartitionSpec, df_name: str, temp_path: Optional[str] = None, to_file_threshold: Any = -1, has_name: bool = False, ) -> DataFrame: to_file_threshold = _get_file_threshold(to_file_threshold) on = list(filter(lambda k: k in df.schema, partition_spec.partition_by)) presort = list( filter(lambda p: p[0] in df.schema, partition_spec.presort.items()) ) col_name = _df_name_to_serialize_col(df_name) if len(on) == 0: partition_spec = PartitionSpec( partition_spec, num=1, by=[], presort=presort ) output_schema = Schema(f"{col_name}:str") else: partition_spec = PartitionSpec(partition_spec, by=on, presort=presort) output_schema = partition_spec.get_key_schema(df.schema) + f"{col_name}:str" s = _PartitionSerializer(output_schema, temp_path, to_file_threshold) metadata = dict( serialized=True, serialized_cols={df_name: col_name}, schemas={df_name: str(df.schema)}, serialized_has_name=has_name, ) return self.map(df, s.run, output_schema, partition_spec, metadata)
def test_comap_with_key(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") c = e.to_df([[6, 1]], "c:int,a:int") z1 = e.persist(e.zip(a, b, df1_name="x", df2_name="y")) z2 = e.persist(e.zip_all(DataFrames(x=a, y=b, z=b))) z3 = e.persist( e.zip_all(DataFrames(z=c), partition_spec=PartitionSpec(by=["a"])) ) def comap(cursor, dfs): assert dfs.has_key v = ",".join([k + str(v.count()) for k, v in dfs.items()]) keys = cursor.key_value_array # if len(keys) == 0: # return ArrayDataFrame([[v]], "v:str") return ArrayDataFrame([keys + [v]], cursor.key_schema + "v:str") def on_init(partition_no, dfs): assert dfs.has_key assert partition_no >= 0 assert len(dfs) > 0 res = e.comap( z1, comap, "a:int,v:str", PartitionSpec(), metadata=dict(a=1), on_init=on_init, ) df_eq(res, [[1, "x2,y1"]], "a:int,v:str", metadata=dict(a=1), throw=True) res = e.comap( z2, comap, "a:int,v:str", PartitionSpec(), metadata=dict(a=1), on_init=on_init, ) df_eq(res, [[1, "x2,y1,z1"]], "a:int,v:str", metadata=dict(a=1), throw=True) res = e.comap( z3, comap, "a:int,v:str", PartitionSpec(), metadata=dict(a=1), on_init=on_init, ) df_eq(res, [[1, "z1"]], "a:int,v:str", metadata=dict(a=1), throw=True)
def test_map(self): def noop(cursor, data): return data def on_init(partition_no, data): # TODO: this test is not sufficient assert partition_no >= 0 data.peek_array() e = self.engine o = ArrayDataFrame( [[1, 2], [None, 2], [None, 1], [3, 4], [None, 4]], "a:double,b:int", dict(a=1), ) a = e.to_df(o) # no partition c = e.map(a, noop, a.schema, PartitionSpec(), dict(a=1)) df_eq(c, o, throw=True) # with key partition c = e.map(a, noop, a.schema, PartitionSpec(by=["a"], presort="b"), dict(a=1)) df_eq(c, o, throw=True) # select top c = e.map(a, select_top, a.schema, PartitionSpec(by=["a"], presort="b")) df_eq(c, [[None, 1], [1, 2], [3, 4]], "a:double,b:int", throw=True) # select top with another order c = e.map( a, select_top, a.schema, PartitionSpec(partition_by=["a"], presort="b DESC"), metadata=dict(a=1), ) df_eq( c, [[None, 4], [1, 2], [3, 4]], "a:double,b:int", metadata=dict(a=1), throw=True, ) # add num_partitions, on_init should not matter c = e.map( a, select_top, a.schema, PartitionSpec(partition_by=["a"], presort="b DESC", num_partitions=3), on_init=on_init, ) df_eq(c, [[None, 4], [1, 2], [3, 4]], "a:double,b:int", throw=True)
def test__serialize_by_partition(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") s = e._serialize_by_partition( a, PartitionSpec(by=["a"], presort="b"), df_name="_0" ) assert s.count() == 2 s = e.persist(e._serialize_by_partition(a, PartitionSpec(), df_name="_0")) assert s.count() == 1 s = e.persist( e._serialize_by_partition(a, PartitionSpec(by=["x"]), df_name="_0") ) assert s.count() == 1
def test_save_with_partition(tmpdir, spark_session): si = SparkIO(spark_session, FileSystem()) df1 = _df([["1", 2, 3]], "a:str,b:int,c:long") path = os.path.join(tmpdir, "a.parquet") si.save_df(df1, path, partition_spec=PartitionSpec(num=2)) actual = si.load_df(path, columns=["b", "a"]) df_eq(actual, [[2, "1"]], "b:int,a:str") si.save_df(df1, path, partition_spec=PartitionSpec(by=["a"])) actual = si.load_df(path, columns=["b", "a"]) df_eq(actual, [[2, "1"]], "b:int,a:str") si.save_df(df1, path, partition_spec=PartitionSpec(by=["a"], num=2)) actual = si.load_df(path, columns=["b", "a"]) df_eq(actual, [[2, "1"]], "b:int,a:str")
def test_comap(self): ps = PartitionSpec(presort="b,c") e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") z1 = e.persist(e.zip(a, b)) z2 = e.persist(e.zip(a, b, partition_spec=ps, how="left_outer")) z3 = e.persist( e._serialize_by_partition(a, partition_spec=ps, df_name="_x") ) z4 = e.persist(e.zip(a, b, partition_spec=ps, how="cross")) def comap(cursor, dfs): assert not dfs.has_key v = ",".join([k + str(v.count()) for k, v in dfs.items()]) keys = cursor.key_value_array if len(keys) == 0: return ArrayDataFrame([[v]], "v:str") return ArrayDataFrame([keys + [v]], cursor.key_schema + "v:str") def on_init(partition_no, dfs): assert not dfs.has_key assert partition_no >= 0 assert len(dfs) > 0 res = e.comap( z1, comap, "a:int,v:str", PartitionSpec(), metadata=dict(a=1), on_init=on_init, ) df_eq(res, [[1, "_02,_11"]], "a:int,v:str", metadata=dict(a=1), throw=True) # for outer joins, the NULL will be filled with empty dataframe res = e.comap(z2, comap, "a:int,v:str", PartitionSpec(), metadata=dict(a=1)) df_eq( res, [[1, "_02,_11"], [3, "_01,_10"]], "a:int,v:str", metadata=dict(a=1), throw=True, ) res = e.comap(z3, comap, "v:str", PartitionSpec(), metadata=dict(a=1)) df_eq(res, [["_03"]], "v:str", metadata=dict(a=1), throw=True) res = e.comap(z4, comap, "v:str", PartitionSpec(), metadata=dict(a=1)) df_eq(res, [["_03,_12"]], "v:str", metadata=dict(a=1), throw=True)
def test_partition_cursor(): p = PartitionSpec(dict(partition_by=["b", "a"])) s = Schema("a:int,b:int,c:int,d:int") c = p.get_cursor(s, 2) pt = p.get_partitioner(s) # this part is well covered in spark section assert c.row_schema == s assert c.key_schema == "b:int,a:int" c.set([1, 2, 2, 2], 5, 6) assert [2, 1] == c.key_value_array assert dict(a=1, b=2) == c.key_value_dict assert 2 == c["c"] assert [1, 2, 2, 2] == c.row assert 5 == c.partition_no assert 2 == c.physical_partition_no assert 6 == c.slice_no
def test_map_in_pandas(self): if not hasattr(ps.DataFrame, "mapInPandas"): return def add(cursor, data): assert isinstance(data, LocalDataFrameIterableDataFrame) def get_dfs(): for df in data.native: pdf = df.as_pandas() pdf["zz"] = pdf["xx"] + pdf["yy"] yield PandasDataFrame(pdf) return LocalDataFrameIterableDataFrame(get_dfs()) e = self.engine np.random.seed(0) df = pd.DataFrame(np.random.randint(0, 5, (100000, 2)), columns=["xx", "yy"]) expected = PandasDataFrame(df.assign(zz=df.xx + df.yy), "xx:int,yy:int,zz:int") a = e.to_df(df) # no partition c = e.map(a, add, "xx:int,yy:int,zz:int", PartitionSpec(num=16)) df_eq(c, expected, throw=True)
def test_zip(self): ps = PartitionSpec(by=["a"], presort="b DESC,c DESC") e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") sa = e._serialize_by_partition(a, ps, df_name="_0") sb = e._serialize_by_partition(b, ps, df_name="_1") # test zip with serialized dfs z1 = e.persist(e.zip(sa, sb, how="inner", partition_spec=ps)) assert 1 == z1.count() assert not z1.metadata.get("serialized_has_name", False) z2 = e.persist(e.zip(sa, sb, how="left_outer", partition_spec=ps)) assert 2 == z2.count() # can't have duplicated keys raises(ValueError, lambda: e.zip(sa, sa, how="inner", partition_spec=ps)) # not support semi or anti raises( InvalidOperationError, lambda: e.zip(sa, sa, how="anti", partition_spec=ps), ) raises( InvalidOperationError, lambda: e.zip(sa, sa, how="leftsemi", partition_spec=ps), ) raises( InvalidOperationError, lambda: e.zip(sa, sa, how="LEFT SEMI", partition_spec=ps), ) # can't specify keys for cross join raises( InvalidOperationError, lambda: e.zip(sa, sa, how="cross", partition_spec=ps), ) # test zip with unserialized dfs z3 = e.persist(e.zip(a, b, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) z3 = e.persist(e.zip(a, sb, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) z3 = e.persist(e.zip(sa, b, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) z4 = e.persist(e.zip(a, b, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) z4 = e.persist(e.zip(a, sb, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) z4 = e.persist(e.zip(sa, b, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) z5 = e.persist(e.zip(a, b, how="cross")) assert z5.count() == 1 assert len(z5.schema) == 2 z6 = e.persist(e.zip(sa, b, how="cross")) assert z6.count() == 2 assert len(z6.schema) == 3 z7 = e.zip(a, b, df1_name="x", df2_name="y") z7.show() assert z7.metadata.get("serialized_has_name", False)
def save( self, path: str, fmt: str = "", mode: str = "overwrite", partition: Any = None, single: bool = False, **kwargs: Any, ) -> None: """Save this dataframe to a persistent storage :param path: output path :param fmt: format hint can accept ``parquet``, ``csv``, ``json``, defaults to None, meaning to infer :param mode: can accept ``overwrite``, ``append``, ``error``, defaults to "overwrite" :param partition: |PartitionLikeObject|, how to partition the dataframe before saving, defaults to empty :param single: force the output as a single file, defaults to False :param kwargs: parameters to pass to the underlying framework For more details and examples, read :ref:`Save & Load <tutorial:/tutorials/dag.ipynb#save-&-load>`. """ if partition is None: partition = self._metadata.get("pre_partition", PartitionSpec()) self.workflow.output( self, using=Save, pre_partition=partition, params=dict(path=path, fmt=fmt, mode=mode, single=single, params=kwargs), )
def to_validation_rules(data: Dict[str, Any]) -> Dict[str, Any]: res: Dict[str, Any] = {} for k, v in data.items(): if k in ["partitionby_has", "partitionby_is"]: if isinstance(v, str): v = [x.strip() for x in v.split(",")] res[k] = PartitionSpec(by=v).partition_by elif k in ["presort_has", "presort_is"]: res[k] = list(parse_presort_exp(v).items()) elif k in ["input_has"]: if isinstance(v, str): res[k] = v.replace(" ", "").split(",") else: assert_or_throw( isinstance(v, list), lambda: SyntaxError(f"{v} is neither a string or a list"), ) res[k] = [x.replace(" ", "") for x in v] elif k in ["input_is"]: try: res[k] = str(Schema(v)) except SyntaxError: raise SyntaxError( # pylint: disable=W0707 f"for input_is, the input must be a schema expression {v}") else: raise NotImplementedError(k) return res
def process( self: TDF, using: Any, schema: Any = None, params: Any = None, pre_partition: Any = None, ) -> TDF: """Run a processor on this dataframe. It's a simple wrapper of :meth:`fugue.workflow.workflow.FugueWorkflow.process` Please read the :ref:`Processor Tutorial <tutorial:/tutorials/processor.ipynb>` :param using: processor-like object :param schema: |SchemaLikeObject|, defaults to None. The processor will be able to access this value from :meth:`~fugue.extensions.context.ExtensionContext.output_schema` :param params: |ParamsLikeObject| to run the processor, defaults to None. The processor will be able to access this value from :meth:`~fugue.extensions.context.ExtensionContext.params` :param pre_partition: |PartitionLikeObject|, defaults to None. The processor will be able to access this value from :meth:`~fugue.extensions.context.ExtensionContext.partition_spec` :return: result dataframe :rtype: :class:`~.WorkflowDataFrame` """ if pre_partition is None: pre_partition = self._metadata.get("pre_partition", PartitionSpec()) df = self.workflow.process( self, using=using, schema=schema, params=params, pre_partition=pre_partition ) return self._to_self_type(df)
def __init__( self, file_id: str, deterministic: bool, permanent: bool, lazy: bool = False, partition: Any = None, single: bool = False, namespace: Any = None, **save_kwargs: Any, ): super().__init__( to_file=True, deterministic=deterministic, permanent=permanent, lazy=lazy, fmt="", partition=PartitionSpec(partition), single=single, namespace=namespace, save_kwargs=dict(save_kwargs), ) self._yield_func: Any = None self._file_id = to_uuid(file_id, namespace) self._yielded = YieldedFile(self._file_id)
def visitFugueZipTask(self, ctx: fp.FugueZipTaskContext) -> WorkflowDataFrame: data = self.get_dict(ctx, "dfs", "how") partition_spec = PartitionSpec(**self.get_dict(ctx, "by", "presort")) # TODO: currently SQL does not support cache to file on ZIP return self.workflow.zip( data["dfs"], how=data.get("how", "inner"), partition=partition_spec )
def test_map_with_dict_col(self): e = self.engine dt = datetime.now() # test dict o = ArrayDataFrame([[dt, dict(a=1)]], "a:datetime,b:{a:int}") c = e.map(o, select_top, o.schema, PartitionSpec(by=["a"])) df_eq(c, o, no_pandas=True, check_order=True, throw=True)
def test_pre_partition(): def assert_eq(expr, expected): sql = FugueSQL(expr, "fuguePrepartition", ignore_case=True) v = _VisitorBase(sql) obj = json.dumps(v.visit(sql.tree).jsondict) assert json.dumps(expected.jsondict) == obj assert_eq("prepartition 100", PartitionSpec(num=100)) assert_eq( "prepartition ROWCOUNT*\n3+ (12.5-CONCUrrency/2)", PartitionSpec(num="ROWCOUNT*3+(12.5-CONCURRENCY/2)"), ) assert_eq("prepartition by a, b", PartitionSpec(by=["a", "b"])) assert_eq( "HASH PrePARTITION 100 BY a,b", PartitionSpec(algo="hash", num=100, by=["a", "b"]), ) assert_eq( "EVEN prepartition 100 BY a,b", PartitionSpec(algo="even", num=100, by=["a", "b"]), ) assert_eq( "rand prepartition 100 BY a,b", PartitionSpec(algo="rand", num=100, by=["a", "b"]), ) assert_eq( "prepartition 100 presort a,\nb\ndesc", PartitionSpec(num=100, presort="a asc, b desc"), )
def comap( self, df: DataFrame, map_func: Callable[[PartitionCursor, DataFrames], LocalDataFrame], output_schema: Any, partition_spec: PartitionSpec, metadata: Any = None, on_init: Optional[Callable[[int, DataFrames], Any]] = None, ): """Apply a function to each zipped partition on the zipped dataframe. :param df: input dataframe, it must be a zipped dataframe (it has to be a dataframe output from :meth:`~.zip` or :meth:`~.zip_all`) :param map_func: the function to apply on every zipped partition :param output_schema: |SchemaLikeObject| that can't be None. Please also understand :ref:`why we need this <tutorial:/tutorials/cotransformer.ipynb#why-explicit-on-output-schema?>` :param partition_spec: partition specification for processing the zipped zipped dataframe. :param metadata: dict-like metadata object to add to the dataframe after the map operation, defaults to None :param on_init: callback function when the physical partition is initializaing, defaults to None :return: the dataframe after the comap operation :Notice: * The input of this method must be an output of :meth:`~.zip` or :meth:`~.zip_all` * The ``partition_spec`` here is NOT related with how you zipped the dataframe and however you set it, will only affect the processing speed, actually the partition keys will be overriden to the zipped dataframe partition keys. You may use it in this way to improve the efficiency: ``PartitionSpec(algo="even", num="ROWCOUNT")``, this tells the execution engine to put each zipped partition into a physical partition so it can achieve the best possible load balance. * If input dataframe has keys, the dataframes you get in ``map_func`` and ``on_init`` will have keys, otherwise you will get list-like dataframes * on_init function will get a DataFrames object that has the same structure, but has all empty dataframes, you can use the schemas but not the data. For more details and examples, read :ref:`Zip & Comap <tutorial:/tutorials/execution_engine.ipynb#zip-&-comap>`. """ assert_or_throw(df.metadata["serialized"], ValueError("df is not serilaized")) cs = _Comap(df, map_func, on_init) key_schema = df.schema - list(df.metadata["serialized_cols"].values()) partition_spec = PartitionSpec(partition_spec, by=list(key_schema.keys())) return self.map(df, cs.run, output_schema, partition_spec, metadata, on_init=cs.on_init)
def test_zip_all(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") z = e.persist(e.zip_all(DataFrames(a))) assert 1 == z.count() assert z.metadata.get("serialized", False) assert not z.metadata.get("serialized_has_name", False) z = e.persist(e.zip_all(DataFrames(x=a))) assert 1 == z.count() assert z.metadata.get("serialized", False) assert z.metadata.get("serialized_has_name", False) z = e.persist( e.zip_all(DataFrames(x=a), partition_spec=PartitionSpec(by=["a"]))) assert 2 == z.count() assert z.metadata.get("serialized", False) assert z.metadata.get("serialized_has_name", False) b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") c = e.to_df([[6, 1], [2, 7]], "d:int,a:int") z = e.persist(e.zip_all(DataFrames(a, b, c))) assert 1 == z.count() assert not z.metadata.get("serialized_has_name", False) z = e.persist(e.zip_all(DataFrames(x=a, y=b, z=c))) assert 1 == z.count() assert z.metadata.get("serialized_has_name", False) z = e.persist(e.zip_all(DataFrames(b, b))) assert 2 == z.count() assert not z.metadata.get("serialized_has_name", False) assert ["a", "c"] in z.schema z = e.persist(e.zip_all(DataFrames(x=b, y=b))) assert 2 == z.count() assert z.metadata.get("serialized_has_name", False) assert ["a", "c"] in z.schema z = e.persist( e.zip_all(DataFrames(b, b), partition_spec=PartitionSpec(by=["a"]))) assert 2 == z.count() assert not z.metadata.get("serialized_has_name", False) assert "c" not in z.schema
def map( self, df: DataFrame, map_func: Callable[[PartitionCursor, LocalDataFrame], LocalDataFrame], output_schema: Any, partition_spec: PartitionSpec, metadata: Any = None, on_init: Optional[Callable[[int, DataFrame], Any]] = None, ) -> DataFrame: if partition_spec.num_partitions != "0": self.log.warning( "%s doesn't respect num_partitions %s", self, partition_spec.num_partitions, ) cursor = partition_spec.get_cursor(df.schema, 0) if on_init is not None: on_init(0, df) if len(partition_spec.partition_by) == 0: # no partition df = to_local_df(df) cursor.set(df.peek_array(), 0, 0) output_df = map_func(cursor, df) if (isinstance(output_df, PandasDataFrame) and output_df.schema != output_schema): output_df = PandasDataFrame(output_df.native, output_schema) assert_or_throw( output_df.schema == output_schema, lambda: f"map output {output_df.schema} " f"mismatches given {output_schema}", ) output_df._metadata = ParamDict(metadata, deep=True) output_df._metadata.set_readonly() return self.to_df(output_df) presort = partition_spec.presort presort_keys = list(presort.keys()) presort_asc = list(presort.values()) output_schema = Schema(output_schema) def _map(pdf: pd.DataFrame) -> pd.DataFrame: if len(presort_keys) > 0: pdf = pdf.sort_values(presort_keys, ascending=presort_asc) input_df = PandasDataFrame(pdf.reset_index(drop=True), df.schema, pandas_df_wrapper=True) cursor.set(input_df.peek_array(), cursor.partition_no + 1, 0) output_df = map_func(cursor, input_df) return output_df.as_pandas() result = self.pl_utils.safe_groupby_apply(df.as_pandas(), partition_spec.partition_by, _map) return PandasDataFrame(result, output_schema, metadata)
def map( self, df: DataFrame, map_func: Callable[[PartitionCursor, LocalDataFrame], LocalDataFrame], output_schema: Any, partition_spec: PartitionSpec, metadata: Any = None, on_init: Optional[Callable[[int, DataFrame], Any]] = None, ) -> DataFrame: presort = partition_spec.presort presort_keys = list(presort.keys()) presort_asc = list(presort.values()) output_schema = Schema(output_schema) input_schema = df.schema on_init_once: Any = ( None if on_init is None else RunOnce( on_init, lambda *args, **kwargs: to_uuid(id(on_init), id(args[0])) ) ) def _map(pdf: Any) -> pd.DataFrame: if pdf.shape[0] == 0: return PandasDataFrame([], output_schema).as_pandas() if len(presort_keys) > 0: pdf = pdf.sort_values(presort_keys, ascending=presort_asc) input_df = PandasDataFrame( pdf.reset_index(drop=True), input_schema, pandas_df_wrapper=True ) if on_init_once is not None: on_init_once(0, input_df) cursor = partition_spec.get_cursor(input_schema, 0) cursor.set(input_df.peek_array(), 0, 0) output_df = map_func(cursor, input_df) return output_df.as_pandas() df = self.to_df(df) if len(partition_spec.partition_by) == 0: pdf = self.repartition(df, partition_spec) result = pdf.native.map_partitions(_map, meta=output_schema.pandas_dtype) else: df = self.repartition(df, PartitionSpec(num=partition_spec.num_partitions)) result = self.pl_utils.safe_groupby_apply( df.native, partition_spec.partition_by, _map, meta=output_schema.pandas_dtype, ) return DaskDataFrame(result, output_schema, metadata)
def test_map_with_binary(self): e = self.engine o = ArrayDataFrame( [[pickle.dumps(BinaryObject("a"))], [pickle.dumps(BinaryObject("b"))]], "a:bytes", ) c = e.map(o, binary_map, o.schema, PartitionSpec()) expected = ArrayDataFrame( [ [pickle.dumps(BinaryObject("ax"))], [pickle.dumps(BinaryObject("bx"))], ], "a:bytes", ) df_eq(expected, c, no_pandas=True, check_order=True, throw=True)
def test_determinism(): a = PartitionSpec(num=0) b = PartitionSpec() assert to_uuid(a) == to_uuid(b) a = PartitionSpec(by=["a"], num=2) b = PartitionSpec(num="2", by=["a"]) assert to_uuid(a) == to_uuid(b) a = PartitionSpec(by=["a", "b"]) b = PartitionSpec(by=["b", "a"]) assert to_uuid(a) != to_uuid(b)
def zip( self: TDF, *dfs: Any, how: str = "inner", partition: Any = None, temp_path: Optional[str] = None, to_file_threshold: Any = -1, ) -> TDF: """Zip this data frame with multiple dataframes together with given partition specifications. It's a wrapper of :meth:`fugue.workflow.workflow.FugueWorkflow.zip`. :param dfs: |DataFramesLikeObject| :param how: can accept ``inner``, ``left_outer``, ``right_outer``, ``full_outer``, ``cross``, defaults to ``inner`` :param partition: |PartitionLikeObject|, defaults to None. :param temp_path: file path to store the data (used only if the serialized data is larger than ``to_file_threshold``), defaults to None :param to_file_threshold: file byte size threshold, defaults to -1 :return: a zipped dataframe :rtype: :class:`~.WorkflowDataFrame` :Notice: * ``dfs`` must be list like, the zipped dataframe will be list like * ``dfs`` is fine to be empty * If you want dict-like zip, use :meth:`fugue.workflow.workflow.FugueWorkflow.zip` Read :ref:`CoTransformer <tutorial:/tutorials/dag.ipynb#cotransformer>` and :ref:`Zip & Comap <tutorial:/tutorials/execution_engine.ipynb#zip-&-comap>` for details """ if partition is None: partition = self._metadata.get("pre_partition", PartitionSpec()) df = self.workflow.zip( self, *dfs, how=how, partition=partition, temp_path=temp_path, to_file_threshold=to_file_threshold, ) return self._to_self_type(df)
def test_repartition(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [5, 6], [7, 8], [9, 10]], "a:int,b:int") b = e.repartition(a, PartitionSpec()) assert a is b b = e.repartition(a, PartitionSpec(num=3)) assert 3 == b.num_partitions b = e.repartition(a, PartitionSpec(num="0")) assert a is b b = e.repartition(a, PartitionSpec(num="ROWCOUNT")) assert 5 == b.num_partitions b = e.repartition(a, PartitionSpec(num="ROWCOUNT/2")) assert 2 == b.num_partitions b = e.repartition(a, PartitionSpec(num="ROWCOUNT-ROWCOUNT")) assert a is b b = e.repartition(a, PartitionSpec(by=["a"], num=3)) assert a.num_partitions == b.num_partitions
def output(self, using: Any, params: Any = None, pre_partition: Any = None) -> None: """Run a outputter on this dataframe. It's a simple wrapper of :meth:`fugue.workflow.workflow.FugueWorkflow.output` Please read the :ref:`Outputter Tutorial <tutorial:/tutorials/outputter.ipynb>` :param using: outputter-like object :param params: |ParamsLikeObject| to run the outputter, defaults to None. The outputter will be able to access this value from :meth:`~fugue.extensions.context.ExtensionContext.params` :param pre_partition: |PartitionLikeObject|, defaults to None. The outputter will be able to access this value from :meth:`~fugue.extensions.context.ExtensionContext.partition_spec` """ if pre_partition is None: pre_partition = self._metadata.get("pre_partition", PartitionSpec()) self.workflow.output( self, using=using, params=params, pre_partition=pre_partition )
def partition(self: TDF, *args, **kwargs) -> TDF: """Partition the current dataframe. Please read |PartitionTutorial| :param args: |PartitionLikeObject| :param kwargs: |PartitionLikeObject| :return: dataframe with the partition hint :rtype: :class:`~.WorkflowDataFrame` :Notice: Normally this step is fast because it's to add a partition hint for the next step. """ return self._to_self_type( WorkflowDataFrame( self.workflow, self._task, {"pre_partition": PartitionSpec(*args, **kwargs)}, ) )
def repartition(self, df: DataFrame, partition_spec: PartitionSpec) -> DaskDataFrame: df = self.to_df(df) if partition_spec.empty: return df if len(partition_spec.partition_by) > 0: return df p = partition_spec.get_num_partitions( **{ KEYWORD_ROWCOUNT: lambda: df.persist().count(), # type: ignore KEYWORD_CORECOUNT: lambda: 2, # TODO: remove this hard code }) if p > 0: return DaskDataFrame( df.native.repartition(npartitions=p), schema=df.schema, metadata=df.metadata, type_safe=False, ) return df