def test_runonce(): def con(df, arr, n=1): arr[0] += n return df arr = [0] sc = RunOnce(con, lambda *args, **kwargs: id(args[0])) sc = cloudpickle.loads(cloudpickle.dumps(sc)) df1 = [[0]] df2 = [[0]] assert sc(df1, arr) is df1 assert 1 == arr[0] assert sc(df1, arr) is df1 assert 1 == arr[0] assert sc(df2, arr, n=2) is df2 assert 3 == arr[0] assert sc(df2, arr, n=3) is df2 assert 3 == arr[0] arr[0] = 0 # multi thread test sc = RunOnce(con, lambda *args, **kwargs: id(args[0])) test = [df1, df2] * 10 with concurrent.futures.ThreadPoolExecutor(max_workers=10) as executor: executor.map(lambda df: assert_or_throw(sc(df, arr) == df, "failed"), test) assert 2 == arr[0]
def __init__(self, spark_session: Optional[SparkSession] = None, conf: Any = None): if spark_session is None: spark_session = SparkSession.builder.getOrCreate() self._spark_session = spark_session cf = dict(FUGUE_SPARK_DEFAULT_CONF) cf.update({x[0]: x[1] for x in spark_session.sparkContext.getConf().getAll()}) cf.update(ParamDict(conf)) super().__init__(cf) self._fs = FileSystem() self._log = logging.getLogger() self._broadcast_func = RunOnce( self._broadcast, lambda *args, **kwargs: id(args[0]) ) self._persist_func = RunOnce(self._persist, lambda *args, **kwargs: id(args[0])) self._register_func = RunOnce( self._register, lambda *args, **kwargs: id(args[0]) ) self._io = SparkIO(self.spark_session, self.fs)
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 _map_by_pandas_udf( 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: df = self.to_df(self.repartition(df, partition_spec)) 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 _udf( dfs: Iterable[pd.DataFrame], ) -> Iterable[pd.DataFrame]: # pragma: no cover def get_dfs() -> Iterable[LocalDataFrame]: for df in dfs: if df.shape[0] > 0: yield PandasDataFrame( df.reset_index(drop=True), input_schema, pandas_df_wrapper=True, ) input_df = LocalDataFrameIterableDataFrame(get_dfs(), input_schema) if input_df.empty: return PandasDataFrame([], output_schema).as_pandas() 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) if isinstance(output_df, LocalDataFrameIterableDataFrame): for res in output_df.native: yield res.as_pandas() else: yield output_df.as_pandas() df = self.to_df(df) sdf = df.native.mapInPandas(_udf, schema=to_spark_schema(output_schema)) return SparkDataFrame(sdf, metadata=metadata)
def _group_map_by_pandas_udf( 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 _udf(pdf: Any) -> pd.DataFrame: # pragma: no cover 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) gdf = df.native.groupBy(*partition_spec.partition_by) sdf = gdf.applyInPandas(_udf, schema=to_spark_schema(output_schema)) return SparkDataFrame(sdf, metadata=metadata)