From fb8c2e0f9fcdbcc807438e9acfe63ccefa0fffc1 Mon Sep 17 00:00:00 2001 From: Tom <56171752+Flying-Tom@users.noreply.github.com> Date: Tue, 12 Sep 2023 10:41:23 +0800 Subject: [PATCH 01/27] ENH: Add a status monitor to trace the running stage of subtask (#677) Signed-off-by: Tom <56171752+Flying-Tom@users.noreply.github.com> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: aresnow1 <109642806+aresnow1@users.noreply.github.com> --- .../_mars/services/scheduling/api/oscar.py | 7 ++ .../services/scheduling/worker/__init__.py | 2 +- .../services/scheduling/worker/execution.py | 64 ++++++++++++++++++- .../_mars/services/scheduling/worker/quota.py | 8 ++- .../services/scheduling/worker/service.py | 14 +++- .../scheduling/worker/tests/test_execution.py | 44 ++++++++++++- .../scheduling/worker/tests/test_quota.py | 7 ++ .../_mars/services/subtask/__init__.py | 2 +- python/xorbits/_mars/services/subtask/core.py | 9 +++ 9 files changed, 147 insertions(+), 10 deletions(-) diff --git a/python/xorbits/_mars/services/scheduling/api/oscar.py b/python/xorbits/_mars/services/scheduling/api/oscar.py index cb9541e35..9e2e7cac1 100644 --- a/python/xorbits/_mars/services/scheduling/api/oscar.py +++ b/python/xorbits/_mars/services/scheduling/api/oscar.py @@ -174,11 +174,18 @@ async def create(cls: Type[APIType], session_id: str, address: str) -> APIType: from .... import resource as mars_resource from ..worker import ( + StageMonitorActor, SubtaskExecutionActor, WorkerQuotaManagerActor, WorkerSlotManagerActor, ) + await mo.create_actor( + StageMonitorActor, + uid=StageMonitorActor.default_uid(), + address=address, + ) + await mo.create_actor( SubtaskExecutionActor, subtask_max_retries=0, diff --git a/python/xorbits/_mars/services/scheduling/worker/__init__.py b/python/xorbits/_mars/services/scheduling/worker/__init__.py index f43167c94..ea0785d2e 100644 --- a/python/xorbits/_mars/services/scheduling/worker/__init__.py +++ b/python/xorbits/_mars/services/scheduling/worker/__init__.py @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -from .execution import SubtaskExecutionActor +from .execution import StageMonitorActor, SubtaskExecutionActor from .quota import MemQuotaActor, QuotaActor, WorkerQuotaManagerActor from .service import SchedulingWorkerService from .workerslot import ( diff --git a/python/xorbits/_mars/services/scheduling/worker/execution.py b/python/xorbits/_mars/services/scheduling/worker/execution.py index dfeceb80b..05b8177ba 100644 --- a/python/xorbits/_mars/services/scheduling/worker/execution.py +++ b/python/xorbits/_mars/services/scheduling/worker/execution.py @@ -19,9 +19,10 @@ import operator import pprint import sys +import time from collections import defaultdict from dataclasses import dataclass, field -from typing import Dict, List, Optional +from typing import Dict, List, Optional, Tuple import xoscar as mo from xoscar.errors import ServerClosed, XoscarError @@ -37,13 +38,43 @@ from ...cluster import ClusterAPI from ...meta import MetaAPI from ...storage import StorageAPI -from ...subtask import Subtask, SubtaskAPI, SubtaskResult, SubtaskStatus +from ...subtask import Subtask, SubtaskAPI, SubtaskResult, SubtaskStage, SubtaskStatus from ...task.task_info_collector import TaskInfoCollector from .quota import QuotaActor from .workerslot import BandSlotManagerActor logger = logging.getLogger(__name__) + +class StageMonitorActor(mo.Actor): + def __init__(self): + self._records = dict() + + def report_stage(self, keys: Tuple[str, str], stage: SubtaskStage): + if keys not in self._records: + self._records[keys] = { + "history": [], + } + if stage == SubtaskStage.FINISH: + self._records.pop(keys) + return + self._records[keys]["history"].append((time.time(), stage)) + + async def get_stale_tasks(self, status: SubtaskStage, timeout: int = 5): + cur_timestamp = time.time() + stale_tasks_keys = [] + for k, v in self._records.items(): + if ( + cur_timestamp - v["history"][-1][0] >= timeout + and v["history"][-1][1] == status + ): + stale_tasks_keys.append(k) + return stale_tasks_keys + + async def get_records(self): + return self._records + + # the default times to run subtask. DEFAULT_SUBTASK_MAX_RETRIES = 0 @@ -168,9 +199,16 @@ def __init__( "The count of finished subtasks of the current band.", ("band",), ) + self._stat_monitor_ref = None async def __post_create__(self): self._cluster_api = await ClusterAPI.create(self.address) + self._stat_monitor_ref = await mo.actor_ref( + uid=StageMonitorActor.default_uid(), address=self.address + ) + + async def _get_stat_monitor_ref(self) -> mo.ActorRefType[StageMonitorActor]: + return await mo.actor_ref(StageMonitorActor.default_uid(), address=self.address) @alru_cache(cache_exceptions=False) async def _get_slot_manager_ref( @@ -376,6 +414,9 @@ async def internal_run_subtask(self, subtask: Subtask, band_name: str): band_name, ) ) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.PREPARE_DATA + ) await asyncio.wait_for( prepare_data_task, timeout=self._data_prepare_timeout ) @@ -405,6 +446,9 @@ async def internal_run_subtask(self, subtask: Subtask, band_name: str): except: # noqa: E722 # pylint: disable=bare-except _fill_subtask_result_with_exception(subtask, subtask_info) finally: + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.RELEASE_SLOT + ) # make sure new slot usages are uploaded in time try: slot_manager_ref = await self._get_slot_manager_ref(band_name) @@ -429,9 +473,14 @@ async def _run_subtask_once(): aiotask = None slot_id = None try: + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.REQUEST_QUOTA + ) await quota_ref.request_batch_quota(batch_quota_req) self._check_cancelling(subtask_info) - + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.ACQUIRE_SLOT + ) slot_id = await slot_manager_ref.acquire_free_slot( (subtask.session_id, subtask.subtask_id) ) @@ -442,6 +491,9 @@ async def _run_subtask_once(): aiotask = asyncio.create_task( subtask_api.run_subtask_in_slot(band_name, slot_id, subtask) ) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.EXECUTE + ) return await asyncio.shield(aiotask) except asyncio.CancelledError as ex: try: @@ -541,6 +593,9 @@ async def run_subtask( logger.debug( "Start to schedule subtask %s on %s.", subtask.subtask_id, self.address ) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), "subtask start" + ) self._submitted_subtask_count.record(1, {"band": self.address}) with mo.debug.no_message_trace(): task = asyncio.create_task( @@ -564,6 +619,9 @@ async def run_subtask( self._subtask_info.pop(subtask.subtask_id, None) self._finished_subtask_count.record(1, {"band": self.address}) logger.debug("Subtask %s finished with result %s", subtask.subtask_id, result) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.FINISH + ) return result async def cancel_subtask(self, subtask_id: str, kill_timeout: Optional[int] = 5): diff --git a/python/xorbits/_mars/services/scheduling/worker/quota.py b/python/xorbits/_mars/services/scheduling/worker/quota.py index 430afe133..c6ab7853a 100644 --- a/python/xorbits/_mars/services/scheduling/worker/quota.py +++ b/python/xorbits/_mars/services/scheduling/worker/quota.py @@ -307,9 +307,15 @@ def __init__( self._stat_refresh_task = None self._slot_manager_ref = None + self._stat_monitor_ref = None async def __post_create__(self): await super().__post_create__() + from .execution import StageMonitorActor + + self._stat_monitor_ref = await mo.actor_ref( + uid=StageMonitorActor.default_uid(), address=self.address + ) self._stat_refresh_task = self.ref().update_mem_stats.tell_delay( delay=self._refresh_time ) @@ -332,7 +338,7 @@ async def update_mem_stats(self): """ cur_mem_available = mars_resource.virtual_memory().available if cur_mem_available > self._last_memory_available: - # memory usage reduced: try reallocate existing requests + # memory usage reduced: try to reallocate existing requests await self._process_requests() self._last_memory_available = cur_mem_available self._report_quota_info() diff --git a/python/xorbits/_mars/services/scheduling/worker/service.py b/python/xorbits/_mars/services/scheduling/worker/service.py index a5fad5cc1..1049bc69b 100644 --- a/python/xorbits/_mars/services/scheduling/worker/service.py +++ b/python/xorbits/_mars/services/scheduling/worker/service.py @@ -17,7 +17,11 @@ from ....utils import calc_size_by_str from ...core import AbstractService -from .execution import DEFAULT_SUBTASK_MAX_RETRIES, SubtaskExecutionActor +from .execution import ( + DEFAULT_SUBTASK_MAX_RETRIES, + StageMonitorActor, + SubtaskExecutionActor, +) from .quota import WorkerQuotaManagerActor from .workerslot import WorkerSlotManagerActor @@ -58,6 +62,11 @@ async def start(self): ) data_prepare_timeout = scheduling_config.get("data_prepare_timeout", 600) + await mo.create_actor( + StageMonitorActor, + uid=StageMonitorActor.default_uid(), + address=address, + ) await mo.create_actor( WorkerSlotManagerActor, uid=WorkerSlotManagerActor.default_uid(), @@ -100,3 +109,6 @@ async def stop(self): uid=WorkerSlotManagerActor.default_uid(), address=address ) ) + await mo.destroy_actor( + mo.create_actor_ref(uid=StageMonitorActor.default_uid(), address=address) + ) diff --git a/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py b/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py index 53f05f2ce..26742cae0 100644 --- a/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py +++ b/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py @@ -50,11 +50,16 @@ from ....session import MockSessionAPI from ....storage import MockStorageAPI from ....storage.handler import StorageHandlerActor -from ....subtask import MockSubtaskAPI, Subtask, SubtaskStatus +from ....subtask import MockSubtaskAPI, Subtask, SubtaskStage, SubtaskStatus from ....task.supervisor.manager import TaskManagerActor from ....task.task_info_collector import TaskInfoCollectorActor from ...supervisor import GlobalResourceManagerActor -from ...worker import BandSlotManagerActor, QuotaActor, SubtaskExecutionActor +from ...worker import ( + BandSlotManagerActor, + QuotaActor, + StageMonitorActor, + SubtaskExecutionActor, +) class CancelDetectActorMixin: @@ -181,7 +186,12 @@ async def actor_pool(request): pool.external_address, storage_handler_cls=MockStorageHandlerActor, ) - + # create monitor actor + monitor_ref = await mo.create_actor( + StageMonitorActor, + uid=StageMonitorActor.default_uid(), + address=pool.external_address, + ) # create assigner actor execution_ref = await mo.create_actor( SubtaskExecutionActor, @@ -230,6 +240,7 @@ async def actor_pool(request): try: yield pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref finally: + await mo.destroy_actor(monitor_ref) await mo.destroy_actor(task_manager_ref) await mo.destroy_actor(band_slot_ref) await mo.destroy_actor(global_resource_ref) @@ -611,3 +622,30 @@ def test_fetch_data_from_both_cpu_and_gpu(data_type, chunked, setup_gpu): pd.testing.assert_frame_equal(expected, actual.execute().fetch(to_cpu=True)) else: pd.testing.assert_series_equal(expected, actual.execute().fetch(to_cpu=True)) + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, True)], indirect=True) +async def test_status_monitor_actor(actor_pool): + pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool + subtask_id = f"test_subtask_{uuid.uuid4()}" + subtask = Subtask( + subtask_id=subtask_id, + session_id=session_id, + task_id=f"test_task_{uuid.uuid4()}", + # chunk_graph=chunk_graph, + ) + + monitor_ref = await mo.actor_ref( + StageMonitorActor.default_uid(), address=pool.external_address + ) + await asyncio.wait_for( + execution_ref.run_subtask(subtask, "numa-0", pool.external_address), timeout=30 + ) + for stage in SubtaskStage: + stale_tasks = await monitor_ref.get_stale_tasks(stage) + assert len(stale_tasks) == 0 + + # task has been finished + records = await monitor_ref.get_records() + assert len(records) == 0 diff --git a/python/xorbits/_mars/services/scheduling/worker/tests/test_quota.py b/python/xorbits/_mars/services/scheduling/worker/tests/test_quota.py index 99a406f91..bd8b6c5e7 100644 --- a/python/xorbits/_mars/services/scheduling/worker/tests/test_quota.py +++ b/python/xorbits/_mars/services/scheduling/worker/tests/test_quota.py @@ -25,6 +25,7 @@ from .....tests.core import mock from .....utils import get_next_port from ...worker import BandSlotManagerActor, MemQuotaActor, QuotaActor +from .. import StageMonitorActor class MockBandSlotManagerActor(mo.Actor): @@ -40,11 +41,17 @@ async def actor_pool(): start_method = ( os.environ.get("POOL_START_METHOD", "fork") if sys.platform != "win32" else None ) + # create monitor actor pool = await create_actor_pool( f"127.0.0.1:{get_next_port()}", n_process=0, subprocess_start_method=start_method, ) + await mo.create_actor( + StageMonitorActor, + uid=StageMonitorActor.default_uid(), + address=pool.external_address, + ) await pool.start() try: yield pool diff --git a/python/xorbits/_mars/services/subtask/__init__.py b/python/xorbits/_mars/services/subtask/__init__.py index 0b6fda518..0f81469c1 100644 --- a/python/xorbits/_mars/services/subtask/__init__.py +++ b/python/xorbits/_mars/services/subtask/__init__.py @@ -14,5 +14,5 @@ # limitations under the License. from .api import MockSubtaskAPI, SubtaskAPI -from .core import Subtask, SubtaskGraph, SubtaskResult, SubtaskStatus +from .core import Subtask, SubtaskGraph, SubtaskResult, SubtaskStage, SubtaskStatus from .errors import SlotOccupiedAlready, SubtaskNotExist diff --git a/python/xorbits/_mars/services/subtask/core.py b/python/xorbits/_mars/services/subtask/core.py index ac562cfbc..9823d39c0 100644 --- a/python/xorbits/_mars/services/subtask/core.py +++ b/python/xorbits/_mars/services/subtask/core.py @@ -36,6 +36,15 @@ from ...typing import BandType, ChunkType +class SubtaskStage(Enum): + PREPARE_DATA = 0 + REQUEST_QUOTA = 1 + ACQUIRE_SLOT = 2 + EXECUTE = 3 + RELEASE_SLOT = 4 + FINISH = 5 + + class SubtaskStatus(Enum): pending = 0 running = 1 From ca8e734a522d914b199bb710a1815eb2003553c5 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Tue, 12 Sep 2023 16:28:43 +0800 Subject: [PATCH 02/27] CHORE: Deprecate ``use_inf_as_na`` option (#689) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- python/xorbits/_mars/config.py | 3 - .../_mars/dataframe/groupby/aggregation.py | 25 +++---- .../_mars/dataframe/missing/checkna.py | 42 +++-------- .../xorbits/_mars/dataframe/missing/dropna.py | 70 +++++++----------- .../xorbits/_mars/dataframe/missing/fillna.py | 58 ++++++--------- .../dataframe/missing/tests/test_missing.py | 51 +------------ .../_mars/dataframe/reduction/aggregation.py | 71 +++++++++---------- .../xorbits/_mars/dataframe/reduction/all.py | 8 +-- .../xorbits/_mars/dataframe/reduction/any.py | 8 +-- .../xorbits/_mars/dataframe/reduction/core.py | 26 ++----- .../_mars/dataframe/reduction/count.py | 5 -- .../_mars/dataframe/reduction/cummax.py | 3 - .../_mars/dataframe/reduction/cummin.py | 3 - .../_mars/dataframe/reduction/cumprod.py | 3 - .../_mars/dataframe/reduction/cumsum.py | 3 - .../dataframe/reduction/custom_reduction.py | 3 - .../_mars/dataframe/reduction/kurtosis.py | 5 -- .../xorbits/_mars/dataframe/reduction/max.py | 7 -- .../xorbits/_mars/dataframe/reduction/mean.py | 5 -- .../xorbits/_mars/dataframe/reduction/min.py | 7 -- .../xorbits/_mars/dataframe/reduction/prod.py | 5 -- .../xorbits/_mars/dataframe/reduction/sem.py | 5 -- .../xorbits/_mars/dataframe/reduction/skew.py | 5 -- .../xorbits/_mars/dataframe/reduction/sum.py | 5 -- .../xorbits/_mars/dataframe/reduction/var.py | 5 -- 25 files changed, 102 insertions(+), 329 deletions(-) diff --git a/python/xorbits/_mars/config.py b/python/xorbits/_mars/config.py index 505f7bed2..3dabfbe5c 100644 --- a/python/xorbits/_mars/config.py +++ b/python/xorbits/_mars/config.py @@ -342,9 +342,6 @@ def validate(x): default_options.register_option("serialize_method", "pickle") # dataframe-related options -default_options.register_option( - "dataframe.mode.use_inf_as_na", False, validator=is_bool -) default_options.register_option( "dataframe.use_arrow_dtype", None, validator=any_validator(is_null, is_bool) ) diff --git a/python/xorbits/_mars/dataframe/groupby/aggregation.py b/python/xorbits/_mars/dataframe/groupby/aggregation.py index 1d67e0305..12c3935e6 100644 --- a/python/xorbits/_mars/dataframe/groupby/aggregation.py +++ b/python/xorbits/_mars/dataframe/groupby/aggregation.py @@ -30,7 +30,6 @@ from ...core.operand import OperandStage from ...serialization.serializables import ( AnyField, - BoolField, DictField, Int32Field, Int64Field, @@ -170,7 +169,6 @@ class DataFrameGroupByAgg(DataFrameOperand, DataFrameOperandMixin): groupby_params = DictField("groupby_params") method = StringField("method") - use_inf_as_na = BoolField("use_inf_as_na") # for chunk combine_size = Int32Field("combine_size") @@ -1286,18 +1284,14 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): @redirect_custom_log @enter_current_session def execute(cls, ctx, op: "DataFrameGroupByAgg"): - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - if op.stage == OperandStage.map: - cls._execute_map(ctx, op) - elif op.stage == OperandStage.combine: - cls._execute_combine(ctx, op) - elif op.stage == OperandStage.agg: - cls._execute_agg(ctx, op) - else: # pragma: no cover - raise ValueError("Aggregation operand not executable") - finally: - pd.reset_option("mode.use_inf_as_na") + if op.stage == OperandStage.map: + cls._execute_map(ctx, op) + elif op.stage == OperandStage.combine: + cls._execute_combine(ctx, op) + elif op.stage == OperandStage.agg: + cls._execute_agg(ctx, op) + else: # pragma: no cover + raise ValueError("Aggregation operand not executable") def agg(groupby, func=None, method="auto", combine_size=None, *args, **kwargs): @@ -1355,8 +1349,6 @@ def agg(groupby, func=None, method="auto", combine_size=None, *args, **kwargs): func, *args, _call_agg=True, index=index_value, **kwargs ) - use_inf_as_na = kwargs.pop("_use_inf_as_na", options.dataframe.mode.use_inf_as_na) - agg_op = DataFrameGroupByAgg( raw_func=func, raw_func_kw=kwargs, @@ -1365,6 +1357,5 @@ def agg(groupby, func=None, method="auto", combine_size=None, *args, **kwargs): groupby_params=groupby.op.groupby_params, combine_size=combine_size or options.combine_size, chunk_store_limit=options.chunk_store_limit, - use_inf_as_na=use_inf_as_na, ) return agg_op(groupby) diff --git a/python/xorbits/_mars/dataframe/missing/checkna.py b/python/xorbits/_mars/dataframe/missing/checkna.py index 530c042d8..3ccc70545 100644 --- a/python/xorbits/_mars/dataframe/missing/checkna.py +++ b/python/xorbits/_mars/dataframe/missing/checkna.py @@ -21,7 +21,6 @@ from ... import dataframe as md from ... import opcodes from ... import tensor as mt -from ...config import options from ...core import OutputType from ...serialization.serializables import BoolField from ..operands import ( @@ -39,14 +38,10 @@ class DataFrameCheckNA(DataFrameOperand, DataFrameOperandMixin): _op_type_ = opcodes.CHECK_NA _positive = BoolField("positive") - _use_inf_as_na = BoolField("use_inf_as_na") - def __init__( - self, positive=None, use_inf_as_na=None, sparse=None, output_types=None, **kw - ): + def __init__(self, positive=None, sparse=None, output_types=None, **kw): super().__init__( _positive=positive, - _use_inf_as_na=use_inf_as_na, _output_types=output_types, sparse=sparse, **kw, @@ -56,10 +51,6 @@ def __init__( def positive(self) -> bool: return self._positive - @property - def use_inf_as_na(self) -> bool: - return self._use_inf_as_na - def __call__(self, df): if isinstance(df, DATAFRAME_TYPE): self.output_types = [OutputType.dataframe] @@ -107,15 +98,10 @@ def tile(cls, op: "DataFrameCheckNA"): @classmethod def execute(cls, ctx, op: "DataFrameCheckNA"): in_data = ctx[op.inputs[0].key] - old_use_inf_as_na = pd.get_option("mode.use_inf_as_na") - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - if op.positive: - ctx[op.outputs[0].key] = in_data.isna() - else: - ctx[op.outputs[0].key] = in_data.notna() - finally: - pd.set_option("mode.use_inf_as_na", old_use_inf_as_na) + if op.positive: + ctx[op.outputs[0].key] = in_data.isna() + else: + ctx[op.outputs[0].key] = in_data.notna() def _from_pandas(obj: Any): @@ -200,14 +186,9 @@ def isna(obj): raise NotImplementedError("isna is not defined for MultiIndex") elif isinstance(obj, ENTITY_TYPE): if isinstance(obj, TENSOR_TYPE): - if options.dataframe.mode.use_inf_as_na: - return ~mt.isfinite(obj) - else: - return mt.isnan(obj) + return mt.isnan(obj) else: - op = DataFrameCheckNA( - positive=True, use_inf_as_na=options.dataframe.mode.use_inf_as_na - ) + op = DataFrameCheckNA(positive=True) return op(obj) else: return _from_pandas(pd.isna(obj)) @@ -279,14 +260,9 @@ def notna(obj): raise NotImplementedError("isna is not defined for MultiIndex") elif isinstance(obj, ENTITY_TYPE): if isinstance(obj, TENSOR_TYPE): - if options.dataframe.mode.use_inf_as_na: - return mt.isfinite(obj) - else: - return ~mt.isnan(obj) + return ~mt.isnan(obj) else: - op = DataFrameCheckNA( - positive=False, use_inf_as_na=options.dataframe.mode.use_inf_as_na - ) + op = DataFrameCheckNA(positive=False) return op(obj) else: return _from_pandas(pd.notna(obj)) diff --git a/python/xorbits/_mars/dataframe/missing/dropna.py b/python/xorbits/_mars/dataframe/missing/dropna.py index af71e3cd0..a3d50f466 100644 --- a/python/xorbits/_mars/dataframe/missing/dropna.py +++ b/python/xorbits/_mars/dataframe/missing/dropna.py @@ -19,7 +19,6 @@ import pandas as pd from ... import opcodes -from ...config import options from ...core import OutputType, recursive_tile from ...serialization.serializables import AnyField, BoolField, Int32Field, StringField from ...utils import no_default, pd_release_version @@ -37,7 +36,6 @@ class DataFrameDropNA(DataFrameOperand, DataFrameOperandMixin): _how = StringField("how") _thresh = Int32Field("thresh") _subset = AnyField("subset") - _use_inf_as_na = BoolField("use_inf_as_na") # when True, dropna will be called on the input, # otherwise non-nan counts will be used @@ -51,7 +49,6 @@ def __init__( how=None, thresh=None, subset=None, - use_inf_as_na=None, drop_directly=None, subset_size=None, sparse=None, @@ -63,7 +60,6 @@ def __init__( _how=how, _thresh=thresh, _subset=subset, - _use_inf_as_na=use_inf_as_na, _drop_directly=drop_directly, _subset_size=subset_size, _output_types=output_types, @@ -87,10 +83,6 @@ def thresh(self) -> int: def subset(self) -> list: return self._subset - @property - def use_inf_as_na(self) -> bool: - return self._use_inf_as_na - @property def drop_directly(self) -> bool: return self._drop_directly @@ -150,9 +142,7 @@ def tile(cls, op: "DataFrameDropNA"): subset_df = in_df if op.subset: subset_df = in_df[op.subset] - count_series = yield from recursive_tile( - subset_df.agg("count", axis=1, _use_inf_as_na=op.use_inf_as_na) - ) + count_series = yield from recursive_tile(subset_df.agg("count", axis=1)) nsplits, out_shape, left_chunks, right_chunks = align_dataframe_series( in_df, count_series, axis=0 @@ -185,35 +175,30 @@ def tile(cls, op: "DataFrameDropNA"): @classmethod def execute(cls, ctx, op: "DataFrameDropNA"): - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - - in_data = ctx[op.inputs[0].key] - if op.drop_directly: - if isinstance(in_data, pd.DataFrame): - result = in_data.dropna( - axis=op.axis, how=op.how, thresh=op.thresh, subset=op.subset - ) - elif isinstance(in_data, pd.Series): - result = in_data.dropna(axis=op.axis, how=op.how) - else: - result = in_data.dropna(how=op.how) - ctx[op.outputs[0].key] = result - return - - in_counts = ctx[op.inputs[1].key] - if op.how == "all": - in_counts = in_counts[in_counts > 0] + in_data = ctx[op.inputs[0].key] + if op.drop_directly: + if isinstance(in_data, pd.DataFrame): + result = in_data.dropna( + axis=op.axis, how=op.how, thresh=op.thresh, subset=op.subset + ) + elif isinstance(in_data, pd.Series): + result = in_data.dropna(axis=op.axis, how=op.how) else: - if op.thresh is None or op.thresh is no_default: - thresh = op.subset_size - else: # pragma: no cover - thresh = op.thresh - in_counts = in_counts[in_counts >= thresh] + result = in_data.dropna(how=op.how) + ctx[op.outputs[0].key] = result + return - ctx[op.outputs[0].key] = in_data.reindex(in_counts.index) - finally: - pd.reset_option("mode.use_inf_as_na") + in_counts = ctx[op.inputs[1].key] + if op.how == "all": + in_counts = in_counts[in_counts > 0] + else: + if op.thresh is None or op.thresh is no_default: + thresh = op.subset_size + else: # pragma: no cover + thresh = op.thresh + in_counts = in_counts[in_counts >= thresh] + + ctx[op.outputs[0].key] = in_data.reindex(in_counts.index) def df_dropna( @@ -328,14 +313,12 @@ def df_dropna( if thresh is no_default and how is no_default: how = "any" - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameDropNA( axis=axis, how=how, thresh=thresh, subset=subset, output_types=[OutputType.dataframe], - use_inf_as_na=use_inf_as_na, ) out_df = op(df) if inplace: @@ -417,12 +400,10 @@ def series_dropna(series, axis=0, inplace=False, how=None): dtype: object """ axis = validate_axis(axis, series) - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameDropNA( axis=axis, how=how, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, ) out_series = op(series) if inplace: @@ -445,8 +426,5 @@ def index_dropna(index, how="any"): ------- Index """ - use_inf_as_na = options.dataframe.mode.use_inf_as_na - op = DataFrameDropNA( - axis=0, how=how, output_types=[OutputType.index], use_inf_as_na=use_inf_as_na - ) + op = DataFrameDropNA(axis=0, how=how, output_types=[OutputType.index]) return op(index) diff --git a/python/xorbits/_mars/dataframe/missing/fillna.py b/python/xorbits/_mars/dataframe/missing/fillna.py index 15c33226a..a2a87615e 100644 --- a/python/xorbits/_mars/dataframe/missing/fillna.py +++ b/python/xorbits/_mars/dataframe/missing/fillna.py @@ -19,10 +19,9 @@ import pandas as pd from ... import opcodes -from ...config import options from ...core import ENTITY_TYPE, Entity, OutputType, get_output_types from ...core.operand import OperandStage -from ...serialization.serializables import AnyField, BoolField, Int64Field, StringField +from ...serialization.serializables import AnyField, Int64Field, StringField from ..align import ( align_dataframe_dataframe, align_dataframe_series, @@ -43,7 +42,6 @@ class FillNA(DataFrameOperand, DataFrameOperandMixin): _axis = AnyField("axis") _limit = Int64Field("limit") _downcast = AnyField("downcast") - _use_inf_as_na = BoolField("use_inf_as_na") _output_limit = Int64Field("output_limit") @@ -54,7 +52,6 @@ def __init__( axis=None, limit=None, downcast=None, - use_inf_as_na=None, output_types=None, output_limit=None, **kw @@ -65,7 +62,6 @@ def __init__( _axis=axis, _limit=limit, _downcast=downcast, - _use_inf_as_na=use_inf_as_na, _output_types=output_types, _output_limit=output_limit, **kw @@ -91,10 +87,6 @@ def limit(self): def downcast(self): return self._downcast - @property - def use_inf_as_na(self): - return self._use_inf_as_na - def _set_inputs(self, inputs): super()._set_inputs(inputs) if self._method is None and len(inputs) > 1: @@ -180,31 +172,27 @@ def _execute_combine(cls, ctx, op): @classmethod def execute(cls, ctx, op): - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - if op.stage == OperandStage.map: - cls._execute_map(ctx, op) - elif op.stage == OperandStage.combine: - cls._execute_combine(ctx, op) + if op.stage == OperandStage.map: + cls._execute_map(ctx, op) + elif op.stage == OperandStage.combine: + cls._execute_combine(ctx, op) + else: + input_data = ctx[op.inputs[0].key] + value = getattr(op, "value", None) + if isinstance(op.value, ENTITY_TYPE): + value = ctx[op.value.key] + if not isinstance(input_data, pd.Index): + ctx[op.outputs[0].key] = input_data.fillna( + value=value, + method=op.method, + axis=op.axis, + limit=op.limit, + downcast=op.downcast, + ) else: - input_data = ctx[op.inputs[0].key] - value = getattr(op, "value", None) - if isinstance(op.value, ENTITY_TYPE): - value = ctx[op.value.key] - if not isinstance(input_data, pd.Index): - ctx[op.outputs[0].key] = input_data.fillna( - value=value, - method=op.method, - axis=op.axis, - limit=op.limit, - downcast=op.downcast, - ) - else: - ctx[op.outputs[0].key] = input_data.fillna( - value=value, downcast=op.downcast - ) - finally: - pd.reset_option("mode.use_inf_as_na") + ctx[op.outputs[0].key] = input_data.fillna( + value=value, downcast=op.downcast + ) @classmethod def _tile_one_by_one(cls, op): @@ -607,14 +595,12 @@ def fillna( else: value_df = None - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = FillNA( value=value, method=method, axis=axis, limit=limit, downcast=downcast, - use_inf_as_na=use_inf_as_na, output_types=get_output_types(df), ) out_df = op(df, value_df=value_df) @@ -678,11 +664,9 @@ def index_fillna(index, value=None, downcast=None): if isinstance(value, (list, pd.Series, SERIES_TYPE)): raise ValueError("'value' must be a scalar, passed: %s" % type(value)) - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = FillNA( value=value, downcast=downcast, - use_inf_as_na=use_inf_as_na, output_types=get_output_types(index), ) return op(index) diff --git a/python/xorbits/_mars/dataframe/missing/tests/test_missing.py b/python/xorbits/_mars/dataframe/missing/tests/test_missing.py index cab62c383..0f33f15e9 100644 --- a/python/xorbits/_mars/dataframe/missing/tests/test_missing.py +++ b/python/xorbits/_mars/dataframe/missing/tests/test_missing.py @@ -226,19 +226,10 @@ def test_replace(): assert r.chunks[0].op.limit is None -@pytest.mark.parametrize("inf_as_na", [True, False]) -def test_isna(setup, inf_as_na): - from ....config import options +def test_isna(setup): from ..checkna import isna - old_mars_inf_as_na = options.dataframe.mode.use_inf_as_na - options.dataframe.mode.use_inf_as_na = inf_as_na - # this option could be changed by mars execution. - old_pd_inf_as_na = pd.get_option("mode.use_inf_as_na") - pd.options.mode.use_inf_as_na = inf_as_na - # scalars - assert pd.get_option("mode.use_inf_as_na") == inf_as_na assert isna("dog") == pd.isna("dog") assert isna(None) == pd.isna(None) assert isna(md.NA) == pd.isna(pd.NA) @@ -247,47 +238,39 @@ def test_isna(setup, inf_as_na): assert isna(type) == pd.isna(type) # multi index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na with pytest.raises(NotImplementedError): midx = md.MultiIndex() isna(midx) # list - assert pd.get_option("mode.use_inf_as_na") == inf_as_na l = [1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT] - assert pd.get_option("mode.use_inf_as_na") == inf_as_na actual = isna(l).execute().fetch() expected = pd.isna(l) np.testing.assert_array_equal(expected, actual) # tuple - assert pd.get_option("mode.use_inf_as_na") == inf_as_na t = (1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT) assert not isna(t) # numpy ndarray - assert pd.get_option("mode.use_inf_as_na") == inf_as_na narr = np.array((1, 2, 3, np.Inf, np.NaN)) actual = isna(narr).execute().fetch() expected = pd.isna(narr) np.testing.assert_array_equal(expected, actual) # pandas index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na pi = pd.Index((1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT)) actual = isna(pi).execute().fetch() expected = pd.isna(pi) np.testing.assert_array_equal(expected, actual) # pandas series - assert pd.get_option("mode.use_inf_as_na") == inf_as_na ps = pd.Series((1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT)) actual = isna(ps).execute().fetch() expected = pd.isna(ps) pd.testing.assert_series_equal(expected, actual) # pandas dataframe - assert pd.get_option("mode.use_inf_as_na") == inf_as_na pdf = pd.DataFrame( {"foo": (1, 2, 3, np.Inf, pd.NA), "bar": (4, 5, 6, np.NaN, pd.NaT)} ) @@ -296,7 +279,6 @@ def test_isna(setup, inf_as_na): pd.testing.assert_frame_equal(expected, actual) # mars tensor - assert pd.get_option("mode.use_inf_as_na") == inf_as_na marr = mt.tensor(narr) actual = isna(marr).execute().fetch() expected = pd.isna(narr) @@ -305,7 +287,6 @@ def test_isna(setup, inf_as_na): # mars index from ...datasource.index import from_pandas as from_pandas_index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na mi = from_pandas_index(pi) actual = isna(mi).execute().fetch() expected = pd.isna(pi) @@ -314,7 +295,6 @@ def test_isna(setup, inf_as_na): # mars series from ...datasource.series import from_pandas as from_pandas_series - assert pd.get_option("mode.use_inf_as_na") == inf_as_na ms = from_pandas_series(ps) actual = isna(ms).execute().fetch() expected = pd.isna(ps) @@ -323,29 +303,16 @@ def test_isna(setup, inf_as_na): # mars dataframe from ...datasource.dataframe import from_pandas as from_pandas_df - assert pd.get_option("mode.use_inf_as_na") == inf_as_na mdf = from_pandas_df(pdf) actual = isna(mdf).execute().fetch() expected = pd.isna(pdf) pd.testing.assert_frame_equal(expected, actual) - options.dataframe.mode.use_inf_as_na = old_mars_inf_as_na - pd.options.mode.use_inf_as_na = old_pd_inf_as_na - -@pytest.mark.parametrize("inf_as_na", [True, False]) -def test_notna(setup, inf_as_na): - from ....config import options +def test_notna(setup): from ..checkna import notna - old_mars_inf_as_na = options.dataframe.mode.use_inf_as_na - options.dataframe.mode.use_inf_as_na = inf_as_na - # this option could be changed by mars execution. - old_pd_inf_as_na = pd.get_option("mode.use_inf_as_na") - pd.options.mode.use_inf_as_na = inf_as_na - # scalars - assert pd.get_option("mode.use_inf_as_na") == inf_as_na assert notna("dog") == pd.notna("dog") assert notna(None) == pd.notna(None) assert notna(md.NA) == pd.notna(pd.NA) @@ -354,46 +321,39 @@ def test_notna(setup, inf_as_na): assert notna(type) == pd.notna(type) # multi index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na with pytest.raises(NotImplementedError): midx = md.MultiIndex() notna(midx) # list - assert pd.get_option("mode.use_inf_as_na") == inf_as_na l = [1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT] actual = notna(l).execute().fetch() expected = pd.notna(l) np.testing.assert_array_equal(expected, actual) # tuple - assert pd.get_option("mode.use_inf_as_na") == inf_as_na t = (1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT) assert notna(t) # numpy ndarray - assert pd.get_option("mode.use_inf_as_na") == inf_as_na narr = np.array((1, 2, 3, np.Inf, np.NaN)) actual = notna(narr).execute().fetch() expected = pd.notna(narr) np.testing.assert_array_equal(expected, actual) # pandas index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na pi = pd.Index((1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT)) actual = notna(pi).execute().fetch() expected = pd.notna(pi) np.testing.assert_array_equal(expected, actual) # pandas series - assert pd.get_option("mode.use_inf_as_na") == inf_as_na ps = pd.Series((1, 2, 3, np.Inf, np.NaN, pd.NA, pd.NaT)) actual = notna(ps).execute().fetch() expected = pd.notna(ps) pd.testing.assert_series_equal(expected, actual) # pandas dataframe - assert pd.get_option("mode.use_inf_as_na") == inf_as_na pdf = pd.DataFrame( {"foo": (1, 2, 3, np.Inf, pd.NA), "bar": (4, 5, 6, np.NaN, pd.NaT)} ) @@ -402,7 +362,6 @@ def test_notna(setup, inf_as_na): pd.testing.assert_frame_equal(expected, actual) # mars tensor - assert pd.get_option("mode.use_inf_as_na") == inf_as_na marr = mt.tensor(narr) actual = notna(marr).execute().fetch() expected = pd.notna(narr) @@ -411,7 +370,6 @@ def test_notna(setup, inf_as_na): # mars index from ...datasource.index import from_pandas as from_pandas_index - assert pd.get_option("mode.use_inf_as_na") == inf_as_na mi = from_pandas_index(pi) actual = notna(mi).execute().fetch() expected = pd.notna(pi) @@ -420,7 +378,6 @@ def test_notna(setup, inf_as_na): # mars series from ...datasource.series import from_pandas as from_pandas_series - assert pd.get_option("mode.use_inf_as_na") == inf_as_na ms = from_pandas_series(ps) actual = notna(ms).execute().fetch() expected = pd.notna(ps) @@ -429,11 +386,7 @@ def test_notna(setup, inf_as_na): # mars dataframe from ...datasource.dataframe import from_pandas as from_pandas_df - assert pd.get_option("mode.use_inf_as_na") == inf_as_na mdf = from_pandas_df(pdf) actual = notna(mdf).execute().fetch() expected = pd.notna(pdf) pd.testing.assert_frame_equal(expected, actual) - - options.dataframe.mode.use_inf_as_na = old_mars_inf_as_na - pd.options.mode.use_inf_as_na = old_pd_inf_as_na diff --git a/python/xorbits/_mars/dataframe/reduction/aggregation.py b/python/xorbits/_mars/dataframe/reduction/aggregation.py index dc748dfb3..404dd7d7f 100644 --- a/python/xorbits/_mars/dataframe/reduction/aggregation.py +++ b/python/xorbits/_mars/dataframe/reduction/aggregation.py @@ -98,7 +98,6 @@ class DataFrameAggregate(DataFrameOperand, DataFrameOperandMixin): axis = AnyField("axis") numeric_only = BoolField("numeric_only") bool_only = BoolField("bool_only") - use_inf_as_na = BoolField("use_inf_as_na") combine_size = Int32Field("combine_size") pre_funcs = ListField("pre_funcs") @@ -925,45 +924,41 @@ def _cudf_agg(cls, op: "DataFrameAggregate", in_data): @redirect_custom_log @enter_current_session def execute(cls, ctx, op: "DataFrameAggregate"): - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - if op.stage == OperandStage.map: - cls._execute_map(ctx, op) - elif op.stage == OperandStage.combine: - cls._execute_combine(ctx, op) - elif op.stage == OperandStage.agg: - cls._execute_agg(ctx, op) - elif not _agg_size_as_series and op.raw_func == "size": - xp = cp if op.gpu else np - ctx[op.outputs[0].key] = xp.array( - ctx[op.inputs[0].key].agg(op.raw_func, axis=op.axis) - ).reshape(op.outputs[0].shape) + if op.stage == OperandStage.map: + cls._execute_map(ctx, op) + elif op.stage == OperandStage.combine: + cls._execute_combine(ctx, op) + elif op.stage == OperandStage.agg: + cls._execute_agg(ctx, op) + elif not _agg_size_as_series and op.raw_func == "size": + xp = cp if op.gpu else np + ctx[op.outputs[0].key] = xp.array( + ctx[op.inputs[0].key].agg(op.raw_func, axis=op.axis) + ).reshape(op.outputs[0].shape) + else: + xp = cp if op.gpu else np + in_obj = op.inputs[0] + in_data = ctx[in_obj.key] + in_data = cls._select_dtypes(in_data, op) + if isinstance(in_obj, INDEX_CHUNK_TYPE): + result = op.func[0](in_data) + elif ( + op.output_types[0] == OutputType.scalar + and in_data.shape == (0,) + and callable(op.func[0]) + ): + result = op.func[0](in_data) else: - xp = cp if op.gpu else np - in_obj = op.inputs[0] - in_data = ctx[in_obj.key] - in_data = cls._select_dtypes(in_data, op) - if isinstance(in_obj, INDEX_CHUNK_TYPE): - result = op.func[0](in_data) - elif ( - op.output_types[0] == OutputType.scalar - and in_data.shape == (0,) - and callable(op.func[0]) - ): - result = op.func[0](in_data) + if is_cudf(in_data): + result = cls._cudf_agg(op, in_data) else: - if is_cudf(in_data): - result = cls._cudf_agg(op, in_data) - else: - result = in_data.agg(op.raw_func, axis=op.axis) - if op.outputs[0].ndim == 1: - result = result.astype(op.outputs[0].dtype, copy=False) + result = in_data.agg(op.raw_func, axis=op.axis) + if op.outputs[0].ndim == 1: + result = result.astype(op.outputs[0].dtype, copy=False) - if op.output_types[0] == OutputType.tensor: - result = xp.array(result) - ctx[op.outputs[0].key] = result - finally: - pd.reset_option("mode.use_inf_as_na") + if op.output_types[0] == OutputType.tensor: + result = xp.array(result) + ctx[op.outputs[0].key] = result def is_funcs_aggregate(func, func_kw=None, ndim=2): @@ -1068,7 +1063,6 @@ def normalize_reduction_funcs(op, ndim=None): def aggregate(df, func=None, axis=0, **kw): axis = validate_axis(axis, df) - use_inf_as_na = kw.pop("_use_inf_as_na", options.dataframe.mode.use_inf_as_na) if ( df.ndim == 2 and isinstance(func, dict) @@ -1107,7 +1101,6 @@ def aggregate(df, func=None, axis=0, **kw): combine_size=combine_size, numeric_only=numeric_only, bool_only=bool_only, - use_inf_as_na=use_inf_as_na, ) return op(df, output_type=output_type, dtypes=dtypes, index=index) diff --git a/python/xorbits/_mars/dataframe/reduction/all.py b/python/xorbits/_mars/dataframe/reduction/all.py index e7a3e2569..f8a5d1409 100644 --- a/python/xorbits/_mars/dataframe/reduction/all.py +++ b/python/xorbits/_mars/dataframe/reduction/all.py @@ -17,7 +17,6 @@ import pandas as pd from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import ( DATAFRAME_TYPE, @@ -86,7 +85,6 @@ def all_series( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameAll( axis=axis, skipna=skipna, @@ -94,7 +92,6 @@ def all_series( bool_only=bool_only, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(series) @@ -109,7 +106,6 @@ def all_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na output_types = [OutputType.series] if axis is not None else [OutputType.scalar] op = DataFrameAll( axis=axis, @@ -118,13 +114,11 @@ def all_dataframe( bool_only=bool_only, combine_size=combine_size, output_types=output_types, - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) def all_index(idx): - use_inf_as_na = options.dataframe.mode.use_inf_as_na - op = DataFrameAll(output_types=[OutputType.scalar], use_inf_as_na=use_inf_as_na) + op = DataFrameAll(output_types=[OutputType.scalar]) return op(idx) diff --git a/python/xorbits/_mars/dataframe/reduction/any.py b/python/xorbits/_mars/dataframe/reduction/any.py index d0bc85b30..b653a461e 100644 --- a/python/xorbits/_mars/dataframe/reduction/any.py +++ b/python/xorbits/_mars/dataframe/reduction/any.py @@ -17,7 +17,6 @@ import pandas as pd from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import ( DATAFRAME_TYPE, @@ -86,7 +85,6 @@ def any_series( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameAny( axis=axis, skipna=skipna, @@ -94,7 +92,6 @@ def any_series( bool_only=bool_only, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(series) @@ -109,7 +106,6 @@ def any_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na output_types = [OutputType.series] if axis is not None else [OutputType.scalar] op = DataFrameAny( axis=axis, @@ -118,13 +114,11 @@ def any_dataframe( bool_only=bool_only, combine_size=combine_size, output_types=output_types, - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) def any_index(index): - use_inf_as_na = options.dataframe.mode.use_inf_as_na - op = DataFrameAny(output_types=[OutputType.scalar], use_inf_as_na=use_inf_as_na) + op = DataFrameAny(output_types=[OutputType.scalar]) return op(index) diff --git a/python/xorbits/_mars/dataframe/reduction/core.py b/python/xorbits/_mars/dataframe/reduction/core.py index 21e33d662..c82b652cd 100644 --- a/python/xorbits/_mars/dataframe/reduction/core.py +++ b/python/xorbits/_mars/dataframe/reduction/core.py @@ -65,7 +65,6 @@ class DataFrameReductionOperand(DataFrameOperand): _numeric_only = BoolField("numeric_only") _bool_only = BoolField("bool_only") _min_count = Int32Field("min_count") - _use_inf_as_na = BoolField("use_inf_as_na") _method = StringField("method") _dtype = DataTypeField("dtype") @@ -84,7 +83,6 @@ def __init__( gpu=None, sparse=None, output_types=None, - use_inf_as_na=None, method=None, **kw, ): @@ -100,7 +98,6 @@ def __init__( gpu=gpu, sparse=sparse, _output_types=output_types, - _use_inf_as_na=use_inf_as_na, _method=method, **kw, ) @@ -137,10 +134,6 @@ def dtype(self): def combine_size(self): return self._combine_size - @property - def use_inf_as_na(self): - return self._use_inf_as_na - @property def is_atomic(self): return False @@ -163,7 +156,6 @@ def get_reduction_args(self, axis=None): class DataFrameCumReductionOperand(DataFrameOperand): _axis = AnyField("axis") _skipna = BoolField("skipna") - _use_inf_as_na = BoolField("use_inf_as_na") _dtype = DataTypeField("dtype") @@ -175,7 +167,6 @@ def __init__( gpu=None, sparse=None, output_types=None, - use_inf_as_na=None, **kw, ): super().__init__( @@ -185,7 +176,6 @@ def __init__( gpu=gpu, sparse=sparse, _output_types=output_types, - _use_inf_as_na=use_inf_as_na, **kw, ) @@ -201,10 +191,6 @@ def skipna(self): def dtype(self): return self._dtype - @property - def use_inf_as_na(self): - return self._use_inf_as_na - def _default_agg_fun(value, func_name=None, **kw): if value.ndim == 1: @@ -612,14 +598,10 @@ def _execute_combine(cls, ctx, op): @classmethod def execute(cls, ctx, op): - try: - pd.set_option("mode.use_inf_as_na", op.use_inf_as_na) - if op.stage == OperandStage.map: - return cls._execute_map(ctx, op) - else: - return cls._execute_combine(ctx, op) - finally: - pd.reset_option("mode.use_inf_as_na") + if op.stage == OperandStage.map: + return cls._execute_map(ctx, op) + else: + return cls._execute_combine(ctx, op) def _call_dataframe(self, df): axis = getattr(self, "axis", None) or 0 diff --git a/python/xorbits/_mars/dataframe/reduction/count.py b/python/xorbits/_mars/dataframe/reduction/count.py index 41b4047b4..71012db21 100644 --- a/python/xorbits/_mars/dataframe/reduction/count.py +++ b/python/xorbits/_mars/dataframe/reduction/count.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -40,13 +39,11 @@ def count(value): def count_series(series, level=None, combine_size=None, **kw): - use_inf_as_na = kw.pop("_use_inf_as_na", options.dataframe.mode.use_inf_as_na) method = kw.pop("method", None) op = DataFrameCount( level=level, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(series) @@ -55,7 +52,6 @@ def count_series(series, level=None, combine_size=None, **kw): def count_dataframe( df, axis=0, level=None, numeric_only=False, combine_size=None, **kw ): - use_inf_as_na = kw.pop("_use_inf_as_na", options.dataframe.mode.use_inf_as_na) method = kw.pop("method", None) op = DataFrameCount( axis=axis, @@ -63,7 +59,6 @@ def count_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/cummax.py b/python/xorbits/_mars/dataframe/reduction/cummax.py index 238d76196..18e248572 100644 --- a/python/xorbits/_mars/dataframe/reduction/cummax.py +++ b/python/xorbits/_mars/dataframe/reduction/cummax.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from .core import DataFrameCumReductionMixin, DataFrameCumReductionOperand @@ -24,11 +23,9 @@ class DataFrameCummax(DataFrameCumReductionOperand, DataFrameCumReductionMixin): def cummax(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameCummax( axis=axis, skipna=skipna, output_types=df.op.output_types, - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/cummin.py b/python/xorbits/_mars/dataframe/reduction/cummin.py index 2ee203770..559b258be 100644 --- a/python/xorbits/_mars/dataframe/reduction/cummin.py +++ b/python/xorbits/_mars/dataframe/reduction/cummin.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from .core import DataFrameCumReductionMixin, DataFrameCumReductionOperand @@ -24,11 +23,9 @@ class DataFrameCummin(DataFrameCumReductionOperand, DataFrameCumReductionMixin): def cummin(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameCummin( axis=axis, skipna=skipna, output_types=df.op.output_types, - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/cumprod.py b/python/xorbits/_mars/dataframe/reduction/cumprod.py index d251198ad..9e52f33e4 100644 --- a/python/xorbits/_mars/dataframe/reduction/cumprod.py +++ b/python/xorbits/_mars/dataframe/reduction/cumprod.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from .core import DataFrameCumReductionMixin, DataFrameCumReductionOperand @@ -24,11 +23,9 @@ class DataFrameCumprod(DataFrameCumReductionOperand, DataFrameCumReductionMixin) def cumprod(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameCumprod( axis=axis, skipna=skipna, output_types=df.op.output_types, - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/cumsum.py b/python/xorbits/_mars/dataframe/reduction/cumsum.py index 591c85302..f6e74ef4b 100644 --- a/python/xorbits/_mars/dataframe/reduction/cumsum.py +++ b/python/xorbits/_mars/dataframe/reduction/cumsum.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from .core import DataFrameCumReductionMixin, DataFrameCumReductionOperand @@ -24,11 +23,9 @@ class DataFrameCumsum(DataFrameCumReductionOperand, DataFrameCumReductionMixin): def cumsum(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameCumsum( axis=axis, skipna=skipna, output_types=df.op.output_types, - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/custom_reduction.py b/python/xorbits/_mars/dataframe/reduction/custom_reduction.py index 59ee88830..a89361df6 100644 --- a/python/xorbits/_mars/dataframe/reduction/custom_reduction.py +++ b/python/xorbits/_mars/dataframe/reduction/custom_reduction.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from ...serialization.serializables import AnyField from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -35,12 +34,10 @@ def get_reduction_args(self, axis=None): def build_custom_reduction_result(df, custom_reduction_obj, method=None): - use_inf_as_na = options.dataframe.mode.use_inf_as_na output_type = OutputType.series if df.ndim == 2 else OutputType.scalar op = DataFrameCustomReduction( custom_reduction=custom_reduction_obj, output_types=[output_type], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/kurtosis.py b/python/xorbits/_mars/dataframe/reduction/kurtosis.py index 5bb2702e9..b32055953 100644 --- a/python/xorbits/_mars/dataframe/reduction/kurtosis.py +++ b/python/xorbits/_mars/dataframe/reduction/kurtosis.py @@ -16,7 +16,6 @@ import numpy as np from ... import opcodes -from ...config import options from ...core import ENTITY_TYPE, OutputType from ...serialization.serializables import BoolField from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -83,7 +82,6 @@ def kurt_series( fisher=True, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameKurtosis( axis=axis, skipna=skipna, @@ -92,7 +90,6 @@ def kurt_series( bias=bias, fisher=fisher, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -109,7 +106,6 @@ def kurt_dataframe( fisher=True, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameKurtosis( axis=axis, skipna=skipna, @@ -119,7 +115,6 @@ def kurt_dataframe( fisher=fisher, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/max.py b/python/xorbits/_mars/dataframe/reduction/max.py index e04fa12cc..0df433075 100644 --- a/python/xorbits/_mars/dataframe/reduction/max.py +++ b/python/xorbits/_mars/dataframe/reduction/max.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -29,14 +28,12 @@ def is_atomic(self): def max_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMax( axis=axis, skipna=skipna, level=level, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -51,7 +48,6 @@ def max_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMax( axis=axis, skipna=skipna, @@ -59,18 +55,15 @@ def max_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) def max_index(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMax( axis=axis, skipna=skipna, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/mean.py b/python/xorbits/_mars/dataframe/reduction/mean.py index e471e66ae..c04f21095 100644 --- a/python/xorbits/_mars/dataframe/reduction/mean.py +++ b/python/xorbits/_mars/dataframe/reduction/mean.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -34,14 +33,12 @@ def mean(x): def mean_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMean( axis=axis, skipna=skipna, level=level, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -56,7 +53,6 @@ def mean_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMean( axis=axis, skipna=skipna, @@ -64,7 +60,6 @@ def mean_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/min.py b/python/xorbits/_mars/dataframe/reduction/min.py index d514c43b7..90b253330 100644 --- a/python/xorbits/_mars/dataframe/reduction/min.py +++ b/python/xorbits/_mars/dataframe/reduction/min.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -29,14 +28,12 @@ def is_atomic(self): def min_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMin( axis=axis, skipna=skipna, level=level, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -51,7 +48,6 @@ def min_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMin( axis=axis, skipna=skipna, @@ -59,18 +55,15 @@ def min_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) def min_index(df, axis=None, skipna=True): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameMin( axis=axis, skipna=skipna, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/prod.py b/python/xorbits/_mars/dataframe/reduction/prod.py index c84b1ce3b..217c34994 100644 --- a/python/xorbits/_mars/dataframe/reduction/prod.py +++ b/python/xorbits/_mars/dataframe/reduction/prod.py @@ -16,7 +16,6 @@ import numpy as np from ... import opcodes -from ...config import options from ...core import OutputType from .aggregation import where_function from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -48,7 +47,6 @@ def prod(value): def prod_series( df, axis=None, skipna=True, level=None, min_count=0, combine_size=None, method=None ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameProd( axis=axis, skipna=skipna, @@ -56,7 +54,6 @@ def prod_series( min_count=min_count, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -72,7 +69,6 @@ def prod_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameProd( axis=axis, skipna=skipna, @@ -81,7 +77,6 @@ def prod_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/sem.py b/python/xorbits/_mars/dataframe/reduction/sem.py index 5c6d274ed..f434c4e9b 100644 --- a/python/xorbits/_mars/dataframe/reduction/sem.py +++ b/python/xorbits/_mars/dataframe/reduction/sem.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from ...serialization.serializables import Int32Field from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -48,7 +47,6 @@ def sem(x): def sem_series( series, axis=None, skipna=True, level=None, ddof=1, combine_size=None, method=None ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSem( axis=axis, skipna=skipna, @@ -56,7 +54,6 @@ def sem_series( ddof=ddof, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(series) @@ -72,7 +69,6 @@ def sem_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSem( axis=axis, skipna=skipna, @@ -81,7 +77,6 @@ def sem_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/skew.py b/python/xorbits/_mars/dataframe/reduction/skew.py index c7e16a5a1..7b9ec6eae 100644 --- a/python/xorbits/_mars/dataframe/reduction/skew.py +++ b/python/xorbits/_mars/dataframe/reduction/skew.py @@ -16,7 +16,6 @@ import numpy as np from ... import opcodes -from ...config import options from ...core import ENTITY_TYPE, OutputType from ...serialization.serializables import BoolField from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -68,7 +67,6 @@ def skew(x): def skew_series( df, axis=None, skipna=True, level=None, combine_size=None, bias=False, method=None ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSkew( axis=axis, skipna=skipna, @@ -76,7 +74,6 @@ def skew_series( combine_size=combine_size, bias=bias, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -92,7 +89,6 @@ def skew_dataframe( bias=False, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSkew( axis=axis, skipna=skipna, @@ -101,7 +97,6 @@ def skew_dataframe( bias=bias, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/sum.py b/python/xorbits/_mars/dataframe/reduction/sum.py index 1b3a4dd7b..69cd4d534 100644 --- a/python/xorbits/_mars/dataframe/reduction/sum.py +++ b/python/xorbits/_mars/dataframe/reduction/sum.py @@ -16,7 +16,6 @@ import numpy as np from ... import opcodes -from ...config import options from ...core import OutputType from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -49,7 +48,6 @@ def sum_(value): def sum_series( df, axis=None, skipna=True, level=None, min_count=0, combine_size=None, method=None ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSum( axis=axis, skipna=skipna, @@ -57,7 +55,6 @@ def sum_series( min_count=min_count, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) @@ -73,7 +70,6 @@ def sum_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameSum( axis=axis, skipna=skipna, @@ -82,7 +78,6 @@ def sum_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) diff --git a/python/xorbits/_mars/dataframe/reduction/var.py b/python/xorbits/_mars/dataframe/reduction/var.py index e59d5fa23..f7e3acbf4 100644 --- a/python/xorbits/_mars/dataframe/reduction/var.py +++ b/python/xorbits/_mars/dataframe/reduction/var.py @@ -14,7 +14,6 @@ # limitations under the License. from ... import opcodes as OperandDef -from ...config import options from ...core import OutputType from ...serialization.serializables import Int32Field from .core import DataFrameReductionMixin, DataFrameReductionOperand @@ -51,7 +50,6 @@ def var(x): def var_series( series, axis=None, skipna=True, level=None, ddof=1, combine_size=None, method=None ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameVar( axis=axis, skipna=skipna, @@ -59,7 +57,6 @@ def var_series( ddof=ddof, combine_size=combine_size, output_types=[OutputType.scalar], - use_inf_as_na=use_inf_as_na, method=method, ) return op(series) @@ -75,7 +72,6 @@ def var_dataframe( combine_size=None, method=None, ): - use_inf_as_na = options.dataframe.mode.use_inf_as_na op = DataFrameVar( axis=axis, skipna=skipna, @@ -84,7 +80,6 @@ def var_dataframe( numeric_only=numeric_only, combine_size=combine_size, output_types=[OutputType.series], - use_inf_as_na=use_inf_as_na, method=method, ) return op(df) From e097bd93e4fdec3b9ac0f740f4847454932f25dd Mon Sep 17 00:00:00 2001 From: Uranus <109661872+UranusSeven@users.noreply.github.com> Date: Wed, 13 Sep 2023 15:59:03 +0800 Subject: [PATCH 03/27] ENH: impl array protocol for series and index (#531) Co-authored-by: ChengjieLi --- python/xorbits/_mars/dataframe/core.py | 20 ++++++++++++++++ .../_mars/dataframe/tests/test_core.py | 24 +++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/python/xorbits/_mars/dataframe/core.py b/python/xorbits/_mars/dataframe/core.py index c4facdcd7..342f0d9c0 100644 --- a/python/xorbits/_mars/dataframe/core.py +++ b/python/xorbits/_mars/dataframe/core.py @@ -954,6 +954,16 @@ def __str__(self): def __repr__(self): return self._to_str(representation=True) + def _to_arr(self): + if len(self._executed_sessions) == 0: # pragma: no cover + raise NotImplementedError + + data = self.fetch(session=self._executed_sessions[-1]) + return np.asarray(data) + + def __array__(self): + return self._to_arr() + def _to_mars_tensor(self, dtype=None, order="K", extract_multi_index=False): tensor = self.to_tensor(extract_multi_index=extract_multi_index) dtype = dtype if dtype is not None else tensor.dtype @@ -1414,6 +1424,16 @@ def __str__(self): def __repr__(self): return self._to_str(representation=False) + def _to_arr(self): + if len(self._executed_sessions) == 0: # pragma: no cover + raise NotImplementedError + + data = self.fetch(session=self._executed_sessions[-1]) + return np.asarray(data) + + def __array__(self): + return self._to_arr() + @property def dtype(self): return getattr(self, "_dtype", None) or getattr(self.op, "dtype", None) diff --git a/python/xorbits/_mars/dataframe/tests/test_core.py b/python/xorbits/_mars/dataframe/tests/test_core.py index af061ff4f..2ca4b0777 100644 --- a/python/xorbits/_mars/dataframe/tests/test_core.py +++ b/python/xorbits/_mars/dataframe/tests/test_core.py @@ -442,3 +442,27 @@ def test_mars_tensor_magic(setup): np.testing.assert_array_equal(expected, actual) with pytest.raises(ValueError, match="could not convert string to float"): DataFrame(expected).__mars_tensor__(dtype="float64").execute() + + +def test_series_and_index_array(setup): + data = np.random.rand(10) + series = Series(data).execute() + + array = np.array(series) + np.testing.assert_array_equal(array, data) + + df = pd.DataFrame({"a": [1, 2], "b": ["foo", "bar"]}) + xdf = DataFrame(df) + index = xdf.index.execute() + np.testing.assert_array_equal(np.array(df.index), np.array(index)) + + arrays = [ + ["bar", "bar", "baz", "baz", "foo", "foo", "qux", "qux"], + ["one", "two", "one", "two", "one", "two", "one", "two"], + ] + tuples = list(zip(*arrays)) + index = pd.MultiIndex.from_tuples(tuples, names=["first", "second"]) + s = pd.Series(np.random.randn(8), index=index) + xs = Series(s).index.execute() + + np.testing.assert_array_equal(np.array(s.index), np.array(xs)) From 755e12072d6a130a328aa8824d23aaee575528d6 Mon Sep 17 00:00:00 2001 From: aresnow1 <109642806+aresnow1@users.noreply.github.com> Date: Wed, 13 Sep 2023 22:49:25 +0800 Subject: [PATCH 04/27] CHORE: Fix asv CI (#654) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .github/workflows/asv.yaml | 3 +-- asv/asv.conf.json | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/asv.yaml b/.github/workflows/asv.yaml index 165912b38..1e1053965 100644 --- a/.github/workflows/asv.yaml +++ b/.github/workflows/asv.yaml @@ -37,8 +37,7 @@ jobs: id: build shell: bash -el {0} run: | - pip install -e "git+https://github.com/xorbitsai/xoscar.git@main#subdirectory=python&egg=xoscar" - pip install numpy scipy cython asv coverage + pip install numpy scipy cython asv==0.5.1 coverage cd python && pip install -e ".[dev,extra]" - name: Run ASV benchmarks diff --git a/asv/asv.conf.json b/asv/asv.conf.json index f751dc5c7..5e240b4f1 100644 --- a/asv/asv.conf.json +++ b/asv/asv.conf.json @@ -88,7 +88,7 @@ "numpy": [], "Cython": ["0.29.24"], "pandas": [], - "scipy": [], + "scipy": ["1.10.0"], "scikit-learn": [], "numexpr": [], "cloudpickle": [], From 1876bcfc154a72d90905a89a5386a8925187521e Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Thu, 14 Sep 2023 18:08:43 +0800 Subject: [PATCH 05/27] FEAT: Impl series __setitem__ (#694) --- .../_mars/dataframe/indexing/__init__.py | 3 +- .../_mars/dataframe/indexing/setitem.py | 91 ++++++++++++++++++- .../indexing/tests/test_indexing_execution.py | 30 ++++++ 3 files changed, 119 insertions(+), 5 deletions(-) diff --git a/python/xorbits/_mars/dataframe/indexing/__init__.py b/python/xorbits/_mars/dataframe/indexing/__init__.py index 45aabf4b2..7d30d7aaf 100644 --- a/python/xorbits/_mars/dataframe/indexing/__init__.py +++ b/python/xorbits/_mars/dataframe/indexing/__init__.py @@ -38,7 +38,7 @@ def _install(): from .sample import sample from .set_axis import df_set_axis, series_set_axis from .set_index import set_index - from .setitem import dataframe_setitem + from .setitem import dataframe_setitem, series_setitem from .where import mask, where for cls in DATAFRAME_TYPE + SERIES_TYPE: @@ -69,6 +69,7 @@ def _install(): for cls in SERIES_TYPE: setattr(cls, "__getitem__", series_getitem) + setattr(cls, "__setitem__", series_setitem) setattr(cls, "reset_index", series_reset_index) setattr(cls, "rename", series_rename) setattr(cls, "set_axis", series_set_axis) diff --git a/python/xorbits/_mars/dataframe/indexing/setitem.py b/python/xorbits/_mars/dataframe/indexing/setitem.py index 41190475d..103165137 100644 --- a/python/xorbits/_mars/dataframe/indexing/setitem.py +++ b/python/xorbits/_mars/dataframe/indexing/setitem.py @@ -14,6 +14,7 @@ # limitations under the License. import collections +from typing import Union import numpy as np import pandas as pd @@ -24,7 +25,7 @@ from ...serialization.serializables import AnyField, KeyField from ...tensor.core import TENSOR_TYPE from ...utils import pd_release_version -from ..core import DATAFRAME_TYPE, SERIES_TYPE, DataFrame +from ..core import DATAFRAME_TYPE, SERIES_TYPE, DataFrame, Series from ..initializer import DataFrame as asframe from ..initializer import Series as asseries from ..operands import DataFrameOperand, DataFrameOperandMixin @@ -49,8 +50,6 @@ def __init__(self, target=None, indexes=None, value=None, output_types=None, **k _output_types=output_types, **kw, ) - if self.output_types is None: - self.output_types = [OutputType.dataframe] @property def target(self): @@ -74,7 +73,7 @@ def _set_inputs(self, inputs): def _is_scalar_tensor(t): return isinstance(t, TENSOR_TYPE) and t.ndim == 0 - def __call__(self, target: DataFrame, value): + def _call_dataframe(self, target: DataFrame, value): raw_target = target inputs = [target] @@ -145,8 +144,54 @@ def __call__(self, target: DataFrame, value): ) raw_target.data = ret.data + def _call_series(self, target: Series, value): + inputs = [target] + dtype = target.dtype + shape = target.shape + index_value = target.index_value + + target.data = self.new_series( + inputs, shape=shape, dtype=dtype, index_value=index_value, name=target.name + ).data + + def __call__(self, target: Union[DataFrame, Series], value): + if target.ndim == 2: + self._call_dataframe(target, value) + else: + self._call_series(target, value) + @classmethod def tile(cls, op: "DataFrameSetitem"): + if op.target.ndim == 2: + res = yield from cls._tile_dataframe(op) + return res + else: + return cls._tile_series(op) + + @classmethod + def _tile_series(cls, op: "DataFrameSetitem"): + in_df = op.inputs[0] + result_chunks = [] + + for chk in in_df.chunks: + new_op = op.copy().reset_key() + new_op.output_types = [OutputType.series] + params = dict( + shape=chk.shape, + index=chk.index, + dtype=chk.dtype, + index_value=chk.index_value, + ) + result_chunks.append(new_op.new_chunk([chk], **params)) + + _new_op = op.copy() + params = op.outputs[0].params.copy() + params["nsplits"] = in_df.nsplits + params["chunks"] = result_chunks + return _new_op.new_seriess(op.inputs, **params) + + @classmethod + def _tile_dataframe(cls, op: "DataFrameSetitem"): from ..merge.concat import DataFrameConcat out = op.outputs[0] @@ -307,6 +352,36 @@ def estimate_size(cls, ctx: dict, op: "DataFrameSetitem"): @classmethod def execute(cls, ctx, op: "DataFrameSetitem"): + target = ctx[op.target.key] + if target.ndim == 2: + cls._execute_dataframe(ctx, op) + else: + cls._execute_series(ctx, op) + + @classmethod + def _execute_series(cls, ctx, op: "DataFrameSetitem"): + target = ctx[op.target.key] + + indexes = op.indexes + value = op.value + + try: + _ = target[indexes] + indexed = True + except KeyError: + indexed = False + + if indexed: + try: + target[indexes] = value + except ValueError: + target = target.copy(deep=True) + target[indexes] = value + + ctx[op.outputs[0].key] = target + + @classmethod + def _execute_dataframe(cls, ctx, op: "DataFrameSetitem"): target = ctx[op.target.key] # only deep copy when updating indexes = ( @@ -336,3 +411,11 @@ def execute(cls, ctx, op: "DataFrameSetitem"): def dataframe_setitem(df, col, value): op = DataFrameSetitem(target=df, indexes=col, value=value) return op(df, value) + + +def series_setitem(series, index, value): + """ + Currently only supports series whose indexes contain `index` + """ + op = DataFrameSetitem(target=series, indexes=index, value=value) + return op(series, value) diff --git a/python/xorbits/_mars/dataframe/indexing/tests/test_indexing_execution.py b/python/xorbits/_mars/dataframe/indexing/tests/test_indexing_execution.py index 67b231eca..c63ba124b 100644 --- a/python/xorbits/_mars/dataframe/indexing/tests/test_indexing_execution.py +++ b/python/xorbits/_mars/dataframe/indexing/tests/test_indexing_execution.py @@ -194,6 +194,36 @@ def test_iloc_getitem(setup_gpu, gpu): pd.testing.assert_index_equal(index.execute().fetch(), data[selection]) +def test_series_setitem(setup): + data1 = pd.Series(np.arange(10)) + series = md.Series(data1, chunk_size=3) + series[2] = 777 + real = series.execute().fetch() + data1[2] = 777 + pd.testing.assert_series_equal(real, data1) + + arrays = [ + ["bar", "bar", "baz", "baz", "foo", "foo", "qux", "qux"], + ["one", "two", "one", "two", "one", "two", "one", "two"], + ] + tuples = list(zip(*arrays)) + index = pd.MultiIndex.from_tuples(tuples, names=["first", "second"]) + data2 = pd.Series(np.random.randn(8), index=index) + series = md.Series(data2, chunk_size=3) + series["bar", "two"] = 0.888888 + real = series.execute().fetch() + data2["bar", "two"] = 0.888888 + pd.testing.assert_series_equal(real, data2) + + data3 = [9, 99, 999, 9999] + series = md.Series(data3, chunk_size=1) + series[1] = 88 + real = series.execute().fetch() + expected = pd.Series(data3) + expected[1] = 88 + pd.testing.assert_series_equal(real, expected) + + def test_iloc_setitem(setup): df1 = pd.DataFrame( [[1, 3, 3], [4, 2, 6], [7, 8, 9]], From 72b1e368ae375454e64009bf5de0ba86943969a6 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Fri, 15 Sep 2023 10:14:29 +0800 Subject: [PATCH 06/27] BLD: ADLFS import error and docker build failed (#698) --- python/xorbits/deploy/docker/Dockerfile.base | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/xorbits/deploy/docker/Dockerfile.base b/python/xorbits/deploy/docker/Dockerfile.base index b2ba05971..adb1f6008 100644 --- a/python/xorbits/deploy/docker/Dockerfile.base +++ b/python/xorbits/deploy/docker/Dockerfile.base @@ -48,9 +48,11 @@ RUN /opt/conda/bin/conda install \ jaxlib \ uvloop \ libnuma \ + && pip install -U pip \ && pip install -U \ xoscar \ cloudpickle \ + azure-storage-blob>=12.18.1 \ adlfs \ fsspec>=2022.7.1,!=2022.8.0 \ s3fs \ From dd999f0908e4dba9dfaad4caec8e42830800dd6c Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Fri, 15 Sep 2023 13:58:34 +0800 Subject: [PATCH 07/27] BUG: Tensor map chunk error when func is `tolist` (#697) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- python/xorbits/_mars/tensor/core.py | 3 ++ .../_mars/tensor/tests/test_core_execution.py | 8 +++++ .../tests/test_numpy_adapters.py | 30 ++++++++++--------- 3 files changed, 27 insertions(+), 14 deletions(-) diff --git a/python/xorbits/_mars/tensor/core.py b/python/xorbits/_mars/tensor/core.py index b16e3c03e..c92fe6f62 100644 --- a/python/xorbits/_mars/tensor/core.py +++ b/python/xorbits/_mars/tensor/core.py @@ -415,6 +415,9 @@ def imag(self, new_imag): def __array__(self, dtype=None): return np.asarray(self.to_numpy(), dtype=dtype) + def tolist(self): + return self.to_numpy().tolist() + def __array_function__(self, func, types, args, kwargs): from .. import tensor as module diff --git a/python/xorbits/_mars/tensor/tests/test_core_execution.py b/python/xorbits/_mars/tensor/tests/test_core_execution.py index 99d803925..0c59a4e81 100644 --- a/python/xorbits/_mars/tensor/tests/test_core_execution.py +++ b/python/xorbits/_mars/tensor/tests/test_core_execution.py @@ -14,6 +14,7 @@ # limitations under the License. import numpy as np +import pytest from .. import ( add, @@ -281,3 +282,10 @@ def test_flat(setup): np.testing.assert_array_equal(b.execute(), npb) np.testing.assert_array_equal(a.execute(), npa) + + +@pytest.mark.parametrize("chunk_size", [None, 1, 4]) +def test_tolist(setup, chunk_size): + data = np.random.rand(10, 20) + a = tensor(data, chunk_size=chunk_size) + assert a.tolist() == data.tolist() diff --git a/python/xorbits/numpy/numpy_adapters/tests/test_numpy_adapters.py b/python/xorbits/numpy/numpy_adapters/tests/test_numpy_adapters.py index c03f4ded0..5b62ec55c 100644 --- a/python/xorbits/numpy/numpy_adapters/tests/test_numpy_adapters.py +++ b/python/xorbits/numpy/numpy_adapters/tests/test_numpy_adapters.py @@ -13,9 +13,11 @@ # limitations under the License. import numpy as np +import pandas as pd import pytest from .... import numpy as xnp +from .... import pandas as xpd @pytest.mark.parametrize( @@ -145,20 +147,6 @@ def test_tensorinv_fallback(setup): assert np.equal(xnp_output.all(), np_output.all()) -def test_ndarray_fallback(setup): - with pytest.warns(Warning) as w: - a = np.array([1, 2, 3]) - b = xnp.array([1, 2, 3]) - xnp_output = b.tolist().fetch() - np_output = a.tolist() - - assert f"Tensor.tolist will fallback to Numpy" == str(w[0].message) - assert isinstance(xnp_output, list) - for i in range(0, len(b)): - assert np_output[i] == xnp_output[i] - assert xnp_output[i] == i + 1 - - def test_busday_offset(setup): with pytest.warns(Warning) as w: xnp_output = xnp.busday_offset("2011-10", 0, roll="forward").execute().fetch() @@ -216,3 +204,17 @@ def test_docstring(): assert docstring is not None and docstring.endswith( "This docstring was copied from numpy.ndarray." ) + + +def test_tensor_tolist(setup): + data = np.random.rand(15, 25) + tensor = xnp.array(data) + assert data.tolist() == tensor.tolist() + + expected = pd.unique(pd.Series([i for i in range(100)])).tolist() + result = xpd.unique(xpd.Series([i for i in range(100)])).tolist() + assert expected == result + + data = np.array([1, 2, 3, 4]) + tensor = xnp.array([1, 2, 3, 4]) + assert data.tolist() == tensor.tolist() From 14d64ae37aabb32948b67d24f561e8f3ea774adb Mon Sep 17 00:00:00 2001 From: "Xuye (Chris) Qin" Date: Mon, 18 Sep 2023 11:16:25 +0800 Subject: [PATCH 08/27] ENH: added auto merge for cartesian_chunk (#699) --- .../_mars/dataframe/base/cartesian_chunk.py | 109 +++++++++--------- python/xorbits/_mars/dataframe/base/core.py | 98 ++++++++++++++++ python/xorbits/_mars/dataframe/merge/merge.py | 77 ++----------- 3 files changed, 162 insertions(+), 122 deletions(-) diff --git a/python/xorbits/_mars/dataframe/base/cartesian_chunk.py b/python/xorbits/_mars/dataframe/base/cartesian_chunk.py index 6da9acd21..30db54e1d 100644 --- a/python/xorbits/_mars/dataframe/base/cartesian_chunk.py +++ b/python/xorbits/_mars/dataframe/base/cartesian_chunk.py @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging + import numpy as np import pandas as pd @@ -22,11 +24,13 @@ from ...serialization.serializables import ( DictField, FunctionField, + Int32Field, KeyField, + StringField, TupleField, ) from ...utils import enter_current_session, has_unknown_shape, quiet_stdio -from ..operands import DataFrameOperand, DataFrameOperandMixin, OutputType +from ..operands import DataFrameOperand, OutputType from ..utils import ( build_df, build_empty_df, @@ -34,63 +38,31 @@ parse_index, validate_output_types, ) +from .core import DataFrameAutoMergeMixin + +logger = logging.getLogger(__name__) -class DataFrameCartesianChunk(DataFrameOperand, DataFrameOperandMixin): +class DataFrameCartesianChunk(DataFrameOperand, DataFrameAutoMergeMixin): _op_type_ = opcodes.CARTESIAN_CHUNK - _left = KeyField("left") - _right = KeyField("right") - _func = FunctionField("func") - _args = TupleField("args") - _kwargs = DictField("kwargs") + left = KeyField("left") + right = KeyField("right") + func = FunctionField("func") + args = TupleField("args") + kwargs = DictField("kwargs") + auto_merge = StringField("auto_merge") + auto_merge_threshold = Int32Field("auto_merge_threshold") - def __init__( - self, - left=None, - right=None, - func=None, - args=None, - kwargs=None, - output_types=None, - **kw - ): - super().__init__( - _left=left, - _right=right, - _func=func, - _args=args, - _kwargs=kwargs, - _output_types=output_types, - **kw - ) + def __init__(self, output_types=None, **kw): + super().__init__(_output_types=output_types, **kw) if self.memory_scale is None: self.memory_scale = 2.0 - @property - def left(self): - return self._left - - @property - def right(self): - return self._right - - @property - def func(self): - return self._func - - @property - def args(self): - return self._args - - @property - def kwargs(self): - return self._kwargs - def _set_inputs(self, inputs): super()._set_inputs(inputs) - self._left = self._inputs[0] - self._right = self._inputs[1] + self.left = self.inputs[0] + self.right = self.inputs[1] @staticmethod def _build_test_obj(obj): @@ -103,7 +75,7 @@ def _build_test_obj(obj): def __call__(self, left, right, index=None, dtypes=None): test_left = self._build_test_obj(left) test_right = self._build_test_obj(right) - output_type = self._output_types[0] if self._output_types else None + output_type = self.output_types[0] if self.output_types else None if output_type == OutputType.df_or_series: return self.new_df_or_series([left, right]) @@ -111,7 +83,7 @@ def __call__(self, left, right, index=None, dtypes=None): # try run to infer meta try: with np.errstate(all="ignore"), quiet_stdio(): - obj = self._func(test_left, test_right, *self._args, **self._kwargs) + obj = self.func(test_left, test_right, *self.args, **self.kwargs) except: # noqa: E722 # nosec # pylint: disable=bare-except if output_type == OutputType.series: obj = pd.Series([], dtype=np.dtype(object)) @@ -126,11 +98,11 @@ def __call__(self, left, right, index=None, dtypes=None): ) if getattr(obj, "ndim", 0) == 1 or output_type == OutputType.series: - shape = self._kwargs.pop("shape", (np.nan,)) + shape = self.kwargs.pop("shape", (np.nan,)) if index is None: index = obj.index index_value = parse_index( - index, left, right, self._func, self._args, self._kwargs + index, left, right, self.func, self.args, self.kwargs ) return self.new_series( [left, right], @@ -147,7 +119,7 @@ def __call__(self, left, right, index=None, dtypes=None): if index is None: index = obj.index index_value = parse_index( - index, left, right, self._func, self._args, self._kwargs + index, left, right, self.func, self.args, self.kwargs ) return self.new_dataframe( [left, right], @@ -164,6 +136,13 @@ def tile(cls, op: "DataFrameCartesianChunk"): out = op.outputs[0] out_type = op.output_types[0] + auto_merge_threshold = op.auto_merge_threshold + auto_merge_before, auto_merge_after = cls._get_auto_merge_options(op.auto_merge) + + yield from cls._merge_before( + op, auto_merge_before, auto_merge_threshold, left, right, logger + ) + if left.ndim == 2 and left.chunk_shape[1] > 1: if has_unknown_shape(left): yield @@ -240,7 +219,12 @@ def tile(cls, op: "DataFrameCartesianChunk"): params["nsplits"] = tuple(tuple(ns) for ns in nsplits) if nsplits else nsplits params["chunks"] = out_chunks new_op = op.copy() - return new_op.new_tileables(op.inputs, kws=[params]) + ret = new_op.new_tileables(op.inputs, kws=[params]) + + ret = yield from cls._merge_after( + op, auto_merge_after, auto_merge_threshold, ret, logger + ) + return ret @classmethod @redirect_custom_log @@ -250,7 +234,16 @@ def execute(cls, ctx, op: "DataFrameCartesianChunk"): ctx[op.outputs[0].key] = op.func(left, right, *op.args, **(op.kwargs or dict())) -def cartesian_chunk(left, right, func, skip_infer=False, args=(), **kwargs): +def cartesian_chunk( + left, + right, + func, + skip_infer=False, + args=(), + auto_merge: str = "both", + auto_merge_threshold: int = 8, + **kwargs, +): output_type = kwargs.pop("output_type", None) output_types = kwargs.pop("output_types", None) object_type = kwargs.pop("object_type", None) @@ -265,6 +258,10 @@ def cartesian_chunk(left, right, func, skip_infer=False, args=(), **kwargs): index = kwargs.pop("index", None) dtypes = kwargs.pop("dtypes", None) memory_scale = kwargs.pop("memory_scale", None) + if auto_merge not in ["both", "none", "before", "after"]: # pragma: no cover + raise ValueError( + f"auto_merge can only be `both`, `none`, `before` or `after`, got {auto_merge}" + ) op = DataFrameCartesianChunk( left=left, @@ -274,5 +271,7 @@ def cartesian_chunk(left, right, func, skip_infer=False, args=(), **kwargs): kwargs=kwargs, output_types=output_types, memory_scale=memory_scale, + auto_merge=auto_merge, + auto_merge_threshold=auto_merge_threshold, ) return op(left, right, index=index, dtypes=dtypes) diff --git a/python/xorbits/_mars/dataframe/base/core.py b/python/xorbits/_mars/dataframe/base/core.py index b7f529dc1..5f444c439 100644 --- a/python/xorbits/_mars/dataframe/base/core.py +++ b/python/xorbits/_mars/dataframe/base/core.py @@ -13,9 +13,17 @@ # See the License for the specific language governing permissions and # limitations under the License. +from __future__ import annotations + +import logging + +from ...core import TileStatus +from ...core.context import get_context from ...serialization.serializables import KeyField +from ...typing import OperandType, TileableType from ..core import DATAFRAME_TYPE, SERIES_TYPE from ..operands import DataFrameOperand, DataFrameOperandMixin +from ..utils import auto_merge_chunks class DataFrameDeviceConversionBase(DataFrameOperand, DataFrameOperandMixin): @@ -63,3 +71,93 @@ def tile(cls, op): return new_op.new_tileables( op.inputs, chunks=out_chunks, nsplits=op.inputs[0].nsplits, **out.params ) + + +class DataFrameAutoMergeMixin(DataFrameOperandMixin): + @classmethod + def _get_auto_merge_options(cls, auto_merge: str) -> tuple[bool, bool]: + if auto_merge == "both": + return True, True + elif auto_merge == "none": + return False, False + elif auto_merge == "before": + return True, False + else: + assert auto_merge == "after" + return False, True + + @classmethod + def _merge_before( + cls, + op: OperandType, + auto_merge_before: bool, + auto_merge_threshold: int, + left: TileableType, + right: TileableType, + logger: logging.Logger, + ): + ctx = get_context() + + if ( + auto_merge_before + and len(left.chunks) + len(right.chunks) > auto_merge_threshold + ): + yield TileStatus([left, right] + left.chunks + right.chunks, progress=0.2) + left_chunk_size = len(left.chunks) + right_chunk_size = len(right.chunks) + left = auto_merge_chunks(ctx, left) + right = auto_merge_chunks(ctx, right) + logger.info( + "Auto merge before %s, left data shape: %s, chunk count: %s -> %s, " + "right data shape: %s, chunk count: %s -> %s.", + op, + left.shape, + left_chunk_size, + len(left.chunks), + right.shape, + right_chunk_size, + len(right.chunks), + ) + else: + logger.info( + "Skip auto merge before %s, left data shape: %s, chunk count: %d, " + "right data shape: %s, chunk count: %d.", + op, + left.shape, + len(left.chunks), + right.shape, + len(right.chunks), + ) + + @classmethod + def _merge_after( + cls, + op: OperandType, + auto_merge_after: bool, + auto_merge_threshold: int, + ret: TileableType, + logger: logging.Logger, + ): + if auto_merge_after and len(ret[0].chunks) > auto_merge_threshold: + # if how=="inner", output data size will reduce greatly with high probability, + # use auto_merge_chunks to combine small chunks. + yield TileStatus( + ret[0].chunks, progress=0.8 + ) # trigger execution for chunks + merged = auto_merge_chunks(get_context(), ret[0]) + logger.info( + "Auto merge after %s, data shape: %s, chunk count: %s -> %s.", + op, + merged.shape, + len(ret[0].chunks), + len(merged.chunks), + ) + return [merged] + else: + logger.info( + "Skip auto merge after %s, data shape: %s, chunk count: %d.", + op, + ret[0].shape, + len(ret[0].chunks), + ) + return ret diff --git a/python/xorbits/_mars/dataframe/merge/merge.py b/python/xorbits/_mars/dataframe/merge/merge.py index 6882be58c..8f1baf5e5 100644 --- a/python/xorbits/_mars/dataframe/merge/merge.py +++ b/python/xorbits/_mars/dataframe/merge/merge.py @@ -39,6 +39,7 @@ from ...typing import TileableType from ...utils import has_unknown_shape, lazy_import from ..base.bloom_filter import filter_by_bloom_filter +from ..base.core import DataFrameAutoMergeMixin from ..core import DataFrame, DataFrameChunk, Series from ..operands import DataFrameOperand, DataFrameOperandMixin, DataFrameShuffleProxy from ..utils import ( @@ -173,7 +174,7 @@ class MergeMethod(Enum): shuffle = 2 -class DataFrameMerge(DataFrameOperand, DataFrameOperandMixin): +class DataFrameMerge(DataFrameOperand, DataFrameAutoMergeMixin): _op_type_ = OperandDef.DATAFRAME_MERGE how = StringField("how") @@ -668,18 +669,6 @@ def _can_merge_with_broadcast( ) -> bool: return how in [big_side, "inner"] and np.log2(big_chunk_size) > small_chunk_size - @classmethod - def _get_auto_merge_options(cls, auto_merge: str) -> Tuple[bool, bool]: - if auto_merge == "both": - return True, True - elif auto_merge == "none": - return False, False - elif auto_merge == "before": - return True, False - else: - assert auto_merge == "after" - return False, True - @classmethod def _choose_merge_method( cls, op: "DataFrameMerge", left: TileableType, right: TileableType @@ -755,36 +744,9 @@ def tile(cls, op: "DataFrameMerge"): auto_merge_threshold = op.auto_merge_threshold auto_merge_before, auto_merge_after = cls._get_auto_merge_options(op.auto_merge) - if ( - auto_merge_before - and len(left.chunks) + len(right.chunks) > auto_merge_threshold - ): - yield TileStatus([left, right] + left.chunks + right.chunks, progress=0.2) - left_chunk_size = len(left.chunks) - right_chunk_size = len(right.chunks) - left = auto_merge_chunks(ctx, left) - right = auto_merge_chunks(ctx, right) - logger.info( - "Auto merge before %s, left data shape: %s, chunk count: %s -> %s, " - "right data shape: %s, chunk count: %s -> %s.", - op, - left.shape, - left_chunk_size, - len(left.chunks), - right.shape, - right_chunk_size, - len(right.chunks), - ) - else: - logger.info( - "Skip auto merge before %s, left data shape: %s, chunk count: %d, " - "right data shape: %s, chunk count: %d.", - op, - left.shape, - len(left.chunks), - right.shape, - len(right.chunks), - ) + yield from cls._merge_before( + op, auto_merge_before, auto_merge_threshold, left, right, logger + ) method = cls._choose_merge_method(op, left, right) if cls._if_apply_bloom_filter(method, op, left, right): @@ -818,33 +780,14 @@ def tile(cls, op: "DataFrameMerge"): assert method == MergeMethod.shuffle ret = cls._tile_shuffle(op, left, right) - if ( - op.how == "inner" - and auto_merge_after - and len(ret[0].chunks) > auto_merge_threshold - ): + if op.how == "inner": # if how=="inner", output data size will reduce greatly with high probability, # use auto_merge_chunks to combine small chunks. - yield TileStatus( - ret[0].chunks, progress=0.8 - ) # trigger execution for chunks - merged = auto_merge_chunks(get_context(), ret[0]) - logger.info( - "Auto merge after %s, data shape: %s, chunk count: %s -> %s.", - op, - merged.shape, - len(ret[0].chunks), - len(merged.chunks), + ret = yield from cls._merge_after( + op, auto_merge_after, auto_merge_threshold, ret, logger ) - return [merged] - else: - logger.info( - "Skip auto merge after %s, data shape: %s, chunk count: %d.", - op, - ret[0].shape, - len(ret[0].chunks), - ) - return ret + + return ret @classmethod def execute(cls, ctx, op): From bb4c746cbf69c035a3a92eababd5386690732677 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Tue, 19 Sep 2023 15:16:07 +0800 Subject: [PATCH 09/27] CHORE: Fix hadoop CI (#706) --- .github/workflows/python.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python.yaml b/.github/workflows/python.yaml index b1c3cb8f6..f0f93a408 100644 --- a/.github/workflows/python.yaml +++ b/.github/workflows/python.yaml @@ -172,7 +172,7 @@ jobs: ../CI/install-hadoop.sh echo "import coverage; coverage.process_startup()" > \ $(python -c "import site; print(site.getsitepackages()[-1])")/coverage.pth - conda install --quiet --yes -c conda-forge skein libffi conda-pack + conda install --quiet --yes -c conda-forge skein libffi conda-pack grpcio=1.42.0 fi if [[ "$MODULE" == "vineyard" ]]; then pip install "vineyard<0.16.1" -i https://pypi.org/simple From f16df9da2b790f55f10be58ffcc36ac0733d277c Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Tue, 19 Sep 2023 18:44:49 +0800 Subject: [PATCH 10/27] BUG: `df.agg` with kwargs (#707) --- .../_mars/dataframe/reduction/aggregation.py | 6 +++- .../tests/test_reduction_execution.py | 31 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/python/xorbits/_mars/dataframe/reduction/aggregation.py b/python/xorbits/_mars/dataframe/reduction/aggregation.py index 404dd7d7f..d9aa5768f 100644 --- a/python/xorbits/_mars/dataframe/reduction/aggregation.py +++ b/python/xorbits/_mars/dataframe/reduction/aggregation.py @@ -952,7 +952,11 @@ def execute(cls, ctx, op: "DataFrameAggregate"): if is_cudf(in_data): result = cls._cudf_agg(op, in_data) else: - result = in_data.agg(op.raw_func, axis=op.axis) + result = ( + in_data.agg(op.raw_func, axis=op.axis) + if op.raw_func is not None + else in_data.agg(**op.raw_func_kw, axis=op.axis) + ) if op.outputs[0].ndim == 1: result = result.astype(op.outputs[0].dtype, copy=False) diff --git a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py index cfcd70192..de3127bf3 100644 --- a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py +++ b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py @@ -1138,3 +1138,34 @@ def g3(x): s.agg((g1, g2, g3)), ms.agg((g1, g2, g3)).execute().fetch() ) pd.testing.assert_series_equal(s.agg((g1, g1)), ms.agg((g1, g1)).execute().fetch()) + + +@pytest.mark.parametrize("chunk_size", [None, 1, 5, 10]) +def test_agg_with_kwargs(setup, chunk_size): + rs = np.random.RandomState(0) + df = pd.DataFrame( + { + "a": rs.choice([1, 3, 8], size=100), + "b": rs.choice([201.8, 155.7, 95.7], size=100), + "c": rs.choice([1, np.nan, 3], size=100), + }, + ) + mdf = md.DataFrame(df, chunk_size=chunk_size) + res = mdf.agg(a=("a", "sum")) + pd.testing.assert_frame_equal(res.execute().fetch(), df.agg(a=("a", "sum"))) + + res = mdf.agg(x=("a", "sum"), y=("b", "mean")) + pd.testing.assert_frame_equal( + res.execute().fetch(), df.agg(x=("a", "sum"), y=("b", "mean")) + ) + + res = mdf.agg(x=("a", "mean"), y=("c", sum)) + pd.testing.assert_frame_equal( + res.execute().fetch(), df.agg(x=("a", "mean"), y=("c", sum)) + ) + + def g(x): + return x.sum() - (x * 3).sum() + + res = mdf.agg(g=("b", g)) + pd.testing.assert_frame_equal(res.execute().fetch(), df.agg(g=("b", g))) From fe7cabad9a4ddc45f516c1bdb9f694fb065438d2 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Wed, 20 Sep 2023 17:19:40 +0800 Subject: [PATCH 11/27] BUG: Column pruning failed when `groupby` by multi series (#708) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../tileable/column_pruning/input_column_selector.py | 5 +++-- .../tests/test_input_column_selector.py | 12 ++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/input_column_selector.py b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/input_column_selector.py index dda97f7f0..97e866e95 100644 --- a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/input_column_selector.py +++ b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/input_column_selector.py @@ -164,9 +164,10 @@ def df_groupby_agg_select_function( ret = {} # group by a series groupby_series = False - if isinstance(by, list) and len(by) == 1 and isinstance(by[0], BaseSeriesData): + if isinstance(by, list) and all([isinstance(_by, BaseSeriesData) for _by in by]): groupby_series = True - ret[by[0]] = {by[0].name} + for _by in by: + ret[_by] = {_by.name} if isinstance(inp, BaseSeriesData): ret[inp] = {inp.name} diff --git a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_input_column_selector.py b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_input_column_selector.py index e351e339e..82cf899e8 100644 --- a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_input_column_selector.py +++ b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_input_column_selector.py @@ -116,6 +116,18 @@ def test_df_groupby_agg(): assert labels.data in input_columns assert input_columns[labels.data] == {"label"} + label1 = Series([1, 1, 1, 1], name="label1") + label2 = Series([2, 2, 3, 3], name="label2") + s = df.groupby(by=[label1, label2]).sum() + input_columns = InputColumnSelector.select(s.data, {"foo"}) + assert len(input_columns) == 3 + assert df.data in input_columns + assert input_columns[df.data] == {"foo"} + assert label1.data in input_columns + assert input_columns[label1.data] == {"label1"} + assert label2.data in input_columns + assert input_columns[label2.data] == {"label2"} + @pytest.mark.skip(reason="group by index is not supported yet") def test_df_groupby_index_agg(): From bc0c60c5989567bd6748c56a393da0b03a4e9c28 Mon Sep 17 00:00:00 2001 From: Tom <56171752+Flying-Tom@users.noreply.github.com> Date: Thu, 21 Sep 2023 09:59:25 +0800 Subject: [PATCH 12/27] ENH: Add a check whether scheduling is stuck (#695) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../_mars/deploy/oscar/base_config.yml | 9 ++ .../services/scheduling/worker/execution.py | 109 +++++++++++++----- .../services/scheduling/worker/service.py | 1 + .../scheduling/worker/tests/test_execution.py | 66 +++++++++-- 4 files changed, 146 insertions(+), 39 deletions(-) diff --git a/python/xorbits/_mars/deploy/oscar/base_config.yml b/python/xorbits/_mars/deploy/oscar/base_config.yml index 99754f1bf..51305f766 100644 --- a/python/xorbits/_mars/deploy/oscar/base_config.yml +++ b/python/xorbits/_mars/deploy/oscar/base_config.yml @@ -56,6 +56,15 @@ scheduling: # Max number of concurrent speculative run for a subtask. max_concurrent_run: 3 subtask_cancel_timeout: 5 + stage_monitor: + enable_check: false + refresh_time: 3 + prepare_data_timeout: 300 + request_quota_timeout: 300 + acquire_slot_timeout: 300 + execution_timeout: null + release_slot_timeout: 300 + finish_timeout: 300 metrics: backend: console # If backend is prometheus, then we can add prometheus config as follows: diff --git a/python/xorbits/_mars/services/scheduling/worker/execution.py b/python/xorbits/_mars/services/scheduling/worker/execution.py index 05b8177ba..b90da08fc 100644 --- a/python/xorbits/_mars/services/scheduling/worker/execution.py +++ b/python/xorbits/_mars/services/scheduling/worker/execution.py @@ -47,29 +47,79 @@ class StageMonitorActor(mo.Actor): - def __init__(self): + def __init__( + self, + monitoring_config: Dict = {}, + ): self._records = dict() - def report_stage(self, keys: Tuple[str, str], stage: SubtaskStage): - if keys not in self._records: - self._records[keys] = { - "history": [], - } - if stage == SubtaskStage.FINISH: - self._records.pop(keys) - return - self._records[keys]["history"].append((time.time(), stage)) + self._enable_check = monitoring_config.get("enable_check", False) + self._refresh_time = monitoring_config.get("refresh_time", 3) + self._kill_timeout = { + SubtaskStage.PREPARE_DATA: monitoring_config.get("prepare_data_timeout"), + SubtaskStage.REQUEST_QUOTA: monitoring_config.get("request_quota_timeout"), + SubtaskStage.ACQUIRE_SLOT: monitoring_config.get("acquire_slot_timeout"), + SubtaskStage.EXECUTE: monitoring_config.get("execution_timeout"), + SubtaskStage.RELEASE_SLOT: monitoring_config.get("release_slot_timeout"), + SubtaskStage.FINISH: monitoring_config.get("finish_timeout"), + } + self._check_task = None + + async def __post_create__(self): + await super().__post_create__() + if self._enable_check: + self._check_task = self.ref().check_subtasks.tell_delay( + delay=self._refresh_time + ) - async def get_stale_tasks(self, status: SubtaskStage, timeout: int = 5): + async def __pre_destroy__(self): + if self._enable_check: + self._check_task.cancel() + await super().__pre_destroy__() + + async def check_subtasks(self): + stale_tasks = await self.get_all_stale_tasks() + for task_key, stage in stale_tasks: + session_id, subtask_id = task_key + try: + logger.warning( + "Subtask[session_id: %s, subtask_id: %s] is timeout at stage %s", + session_id, + subtask_id, + stage, + ) + except Exception as e: + logger.error(e) + + self._check_task = self.ref().check_subtasks.tell_delay( + delay=self._refresh_time + ) + + async def get_all_stale_tasks(self): cur_timestamp = time.time() - stale_tasks_keys = [] + stale_tasks = [] for k, v in self._records.items(): + pre_timestamp, cur_stage = v["history"][-1][0], v["history"][-1][1] if ( - cur_timestamp - v["history"][-1][0] >= timeout - and v["history"][-1][1] == status + self._kill_timeout[cur_stage] is not None + and cur_timestamp - pre_timestamp >= self._kill_timeout[cur_stage] ): - stale_tasks_keys.append(k) - return stale_tasks_keys + stale_tasks.append((k, cur_stage)) + return stale_tasks + + async def register_subtask(self, subtask: Subtask, supervisor_address: str): + keys = (subtask.session_id, subtask.subtask_id) + self._records[keys] = { + "subtask": subtask, + "history": [], + "supervisor_address": supervisor_address, + } + + async def report_stage(self, keys: Tuple[str, str], stage: SubtaskStage): + if stage == SubtaskStage.FINISH: + self._records.pop(keys) + return + self._records[keys]["history"].append((time.time(), stage)) async def get_records(self): return self._records @@ -404,6 +454,9 @@ async def internal_run_subtask(self, subtask: Subtask, band_name: str): ) try: logger.debug("Preparing data for subtask %s", subtask.subtask_id) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.PREPARE_DATA + ) with Timer() as timer: prepare_data_task = asyncio.create_task( _retry_run( @@ -414,9 +467,7 @@ async def internal_run_subtask(self, subtask: Subtask, band_name: str): band_name, ) ) - await self._stat_monitor_ref.report_stage( - (subtask.session_id, subtask.subtask_id), SubtaskStage.PREPARE_DATA - ) + await asyncio.wait_for( prepare_data_task, timeout=self._data_prepare_timeout ) @@ -446,9 +497,6 @@ async def internal_run_subtask(self, subtask: Subtask, band_name: str): except: # noqa: E722 # pylint: disable=bare-except _fill_subtask_result_with_exception(subtask, subtask_info) finally: - await self._stat_monitor_ref.report_stage( - (subtask.session_id, subtask.subtask_id), SubtaskStage.RELEASE_SLOT - ) # make sure new slot usages are uploaded in time try: slot_manager_ref = await self._get_slot_manager_ref(band_name) @@ -487,13 +535,14 @@ async def _run_subtask_once(): subtask_info.slot_id = slot_id self._check_cancelling(subtask_info) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), SubtaskStage.EXECUTE + ) subtask_info.result.status = SubtaskStatus.running aiotask = asyncio.create_task( subtask_api.run_subtask_in_slot(band_name, slot_id, subtask) ) - await self._stat_monitor_ref.report_stage( - (subtask.session_id, subtask.subtask_id), SubtaskStage.EXECUTE - ) + return await asyncio.shield(aiotask) except asyncio.CancelledError as ex: try: @@ -554,6 +603,10 @@ async def _run_subtask_once(): await slot_manager_ref.release_free_slot( slot_id, (subtask.session_id, subtask.subtask_id) ) + await self._stat_monitor_ref.report_stage( + (subtask.session_id, subtask.subtask_id), + SubtaskStage.RELEASE_SLOT, + ) logger.debug( "Released slot %d for subtask %s", slot_id, subtask.subtask_id ) @@ -593,9 +646,9 @@ async def run_subtask( logger.debug( "Start to schedule subtask %s on %s.", subtask.subtask_id, self.address ) - await self._stat_monitor_ref.report_stage( - (subtask.session_id, subtask.subtask_id), "subtask start" - ) + + await self._stat_monitor_ref.register_subtask(subtask, supervisor_address) + self._submitted_subtask_count.record(1, {"band": self.address}) with mo.debug.no_message_trace(): task = asyncio.create_task( diff --git a/python/xorbits/_mars/services/scheduling/worker/service.py b/python/xorbits/_mars/services/scheduling/worker/service.py index 1049bc69b..0d12fab05 100644 --- a/python/xorbits/_mars/services/scheduling/worker/service.py +++ b/python/xorbits/_mars/services/scheduling/worker/service.py @@ -64,6 +64,7 @@ async def start(self): await mo.create_actor( StageMonitorActor, + monitoring_config=scheduling_config.get("stage_monitor", {}), uid=StageMonitorActor.default_uid(), address=address, ) diff --git a/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py b/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py index 26742cae0..e174598c7 100644 --- a/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py +++ b/python/xorbits/_mars/services/scheduling/worker/tests/test_execution.py @@ -50,7 +50,7 @@ from ....session import MockSessionAPI from ....storage import MockStorageAPI from ....storage.handler import StorageHandlerActor -from ....subtask import MockSubtaskAPI, Subtask, SubtaskStage, SubtaskStatus +from ....subtask import MockSubtaskAPI, Subtask, SubtaskStatus from ....task.supervisor.manager import TaskManagerActor from ....task.task_info_collector import TaskInfoCollectorActor from ...supervisor import GlobalResourceManagerActor @@ -163,7 +163,7 @@ def collect_task_info_enabled(self): @pytest.fixture async def actor_pool(request): - n_slots, enable_kill = request.param + n_slots, enable_kill, enable_stage_check = request.param pool = await create_actor_pool( "127.0.0.1", labels=[None] + ["numa-0"] * n_slots, n_process=n_slots ) @@ -189,6 +189,12 @@ async def actor_pool(request): # create monitor actor monitor_ref = await mo.create_actor( StageMonitorActor, + monitoring_config={ + "enable_check": True, + "execution_timeout": 5, + } + if enable_stage_check + else {}, uid=StageMonitorActor.default_uid(), address=pool.external_address, ) @@ -253,7 +259,7 @@ async def actor_pool(request): @pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, True)], indirect=True) +@pytest.mark.parametrize("actor_pool", [(1, True, False)], indirect=True) async def test_execute_tensor(actor_pool): pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool @@ -334,7 +340,7 @@ async def test_execute_tensor(actor_pool): @pytest.mark.asyncio @pytest.mark.parametrize( "actor_pool,cancel_phase", - [((1, True), phase) for phase in _cancel_phases], + [((1, True, False), phase) for phase in _cancel_phases], indirect=["actor_pool"], ) async def test_execute_with_cancel(actor_pool, cancel_phase): @@ -438,7 +444,7 @@ def delay_fun(delay, _inp1): @pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, True)], indirect=True) +@pytest.mark.parametrize("actor_pool", [(1, True, False)], indirect=True) async def test_execute_with_pure_deps(actor_pool): pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool @@ -519,7 +525,7 @@ def test_estimate_size(): @pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, False)], indirect=True) +@pytest.mark.parametrize("actor_pool", [(1, False, False)], indirect=True) async def test_cancel_without_kill(actor_pool): pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool executed_file = os.path.join( @@ -625,8 +631,8 @@ def test_fetch_data_from_both_cpu_and_gpu(data_type, chunked, setup_gpu): @pytest.mark.asyncio -@pytest.mark.parametrize("actor_pool", [(1, True)], indirect=True) -async def test_status_monitor_actor(actor_pool): +@pytest.mark.parametrize("actor_pool", [(1, True, False)], indirect=True) +async def test_stage_monitor_actor(actor_pool): pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool subtask_id = f"test_subtask_{uuid.uuid4()}" subtask = Subtask( @@ -642,10 +648,48 @@ async def test_status_monitor_actor(actor_pool): await asyncio.wait_for( execution_ref.run_subtask(subtask, "numa-0", pool.external_address), timeout=30 ) - for stage in SubtaskStage: - stale_tasks = await monitor_ref.get_stale_tasks(stage) - assert len(stale_tasks) == 0 + + stale_tasks = await monitor_ref.get_all_stale_tasks() + assert len(stale_tasks) == 0 # task has been finished records = await monitor_ref.get_records() assert len(records) == 0 + + +@pytest.mark.asyncio +@pytest.mark.parametrize("actor_pool", [(1, True, True)], indirect=True) +async def test_terminate_stale_tasks(actor_pool, caplog): + pool, session_id, meta_api, worker_meta_api, storage_api, execution_ref = actor_pool + + def delay_fun(delay): + time.sleep(delay) + return delay + + remote_result = RemoteFunction( + function=delay_fun, function_args=[10], function_kwargs={} + ).new_chunk([]) + chunk_graph = ChunkGraph([remote_result]) + chunk_graph.add_node(remote_result) + + subtask = Subtask( + f"test_subtask_{uuid.uuid4()}", + session_id=session_id, + task_id=f"test_task_{uuid.uuid4()}", + chunk_graph=chunk_graph, + ) + + with Timer() as timer: + aiotask = asyncio.create_task( + execution_ref.run_subtask(subtask, "numa-0", pool.external_address) + ) + + r = await asyncio.wait_for(aiotask, timeout=20) + assert r.status == SubtaskStatus.succeeded + + assert 5 < timer.duration < 20 + + import re + + match = re.search(r"Subtask\[.*?\].*stage.*", caplog.text) + assert match is not None From 4132deb3cd3d2e7c214b189c50d46b63438d28d3 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Thu, 21 Sep 2023 19:53:29 +0800 Subject: [PATCH 13/27] CHORE: Deprecate `downcast` in `fillna` (#705) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../base/tests/test_base_execution.py | 19 +++---- .../_mars/dataframe/base/value_counts.py | 3 + .../xorbits/_mars/dataframe/groupby/fill.py | 12 +--- python/xorbits/_mars/dataframe/hash_utils.py | 5 +- .../xorbits/_mars/dataframe/missing/fillna.py | 57 ++++++------------- .../dataframe/missing/tests/test_missing.py | 2 - 6 files changed, 33 insertions(+), 65 deletions(-) diff --git a/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py b/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py index 23c19e122..bb997d233 100644 --- a/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py +++ b/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py @@ -1729,12 +1729,10 @@ def test_value_counts_execution(setup): r = series.value_counts() pd.testing.assert_series_equal(r.execute().fetch(), s.value_counts()) - # pandas issue: https://github.com/pandas-dev/pandas/issues/54857 - if pd.__version__ != "2.1.0": - r = series.value_counts(bins=5, normalize=True) - pd.testing.assert_series_equal( - r.execute().fetch(), s.value_counts(bins=5, normalize=True) - ) + r = series.value_counts(bins=5, normalize=True) + pd.testing.assert_series_equal( + r.execute().fetch(), s.value_counts(bins=5, normalize=True) + ) # test multi chunks series = from_pandas_series(s, chunk_size=30) @@ -1746,11 +1744,10 @@ def test_value_counts_execution(setup): pd.testing.assert_series_equal(r.execute().fetch(), s.value_counts(normalize=True)) # test bins and normalize - if pd.__version__ != "2.1.0": - r = series.value_counts(method="tree", bins=5, normalize=True) - pd.testing.assert_series_equal( - r.execute().fetch(), s.value_counts(bins=5, normalize=True) - ) + r = series.value_counts(method="tree", bins=5, normalize=True) + pd.testing.assert_series_equal( + r.execute().fetch(), s.value_counts(bins=5, normalize=True) + ) def test_astype(setup): diff --git a/python/xorbits/_mars/dataframe/base/value_counts.py b/python/xorbits/_mars/dataframe/base/value_counts.py index 457250014..ae80f3db3 100644 --- a/python/xorbits/_mars/dataframe/base/value_counts.py +++ b/python/xorbits/_mars/dataframe/base/value_counts.py @@ -193,6 +193,9 @@ def execute(cls, ctx, op: "DataFrameValueCounts"): # convert CategoricalDtype which generated in `cut` # to IntervalDtype result.index = result.index.astype("interval") + # index name changed since pandas 2.1.1 + if pd_release_version >= (2, 1, 1): + result.index.name = None if op.nrows: result = result.head(op.nrows) result.name = op.outputs[0].name diff --git a/python/xorbits/_mars/dataframe/groupby/fill.py b/python/xorbits/_mars/dataframe/groupby/fill.py index 7d71a89de..4baa08ea1 100644 --- a/python/xorbits/_mars/dataframe/groupby/fill.py +++ b/python/xorbits/_mars/dataframe/groupby/fill.py @@ -18,7 +18,7 @@ from ... import opcodes from ...core import OutputType -from ...serialization.serializables import AnyField, DictField, Int64Field, StringField +from ...serialization.serializables import AnyField, Int64Field, StringField from ..operands import DataFrameOperand, DataFrameOperandMixin from ..utils import build_empty_df, build_empty_series, parse_index @@ -29,7 +29,6 @@ class GroupByFillOperand(DataFrameOperand, DataFrameOperandMixin): value = AnyField("value", default=None) method = StringField("method", default=None) limit = Int64Field("limit", default=None) - downcast = DictField("downcast", default=None) def _calc_out_dtypes(self, in_groupby): mock_groupby = in_groupby.op.build_mock_groupby() @@ -40,7 +39,6 @@ def _calc_out_dtypes(self, in_groupby): value=self.value, method=self.method, limit=self.limit, - downcast=self.downcast, ) else: result_df = getattr(mock_groupby, func_name)(limit=self.limit) @@ -133,7 +131,6 @@ def execute(cls, ctx, op: "GroupByFillOperand"): value=op.value, method=op.method, limit=op.limit, - downcast=op.downcast, ) else: result = getattr(in_data, func_name)(limit=op.limit) @@ -184,7 +181,7 @@ def bfill(groupby, limit=None): return op(groupby) -def fillna(groupby, value=None, method=None, limit=None, downcast=None): +def fillna(groupby, value=None, method=None, limit=None): """ Fill NA/NaN values using the specified method @@ -197,11 +194,8 @@ def fillna(groupby, value=None, method=None, limit=None, downcast=None): limit: int, default None If method is specified, this is the maximum number of consecutive NaN values to forward/backward fill - downcast: dict, default None - A dict of item->dtype of what to downcast if possible, - or the string ‘infer’ which will try to downcast to an appropriate equal type return: DataFrame or None """ - op = GroupByFillNa(value=value, method=method, limit=limit, downcast=downcast) + op = GroupByFillNa(value=value, method=method, limit=limit) return op(groupby) diff --git a/python/xorbits/_mars/dataframe/hash_utils.py b/python/xorbits/_mars/dataframe/hash_utils.py index 348c4b16e..0d4211e2d 100644 --- a/python/xorbits/_mars/dataframe/hash_utils.py +++ b/python/xorbits/_mars/dataframe/hash_utils.py @@ -9,10 +9,11 @@ from typing import TYPE_CHECKING, Hashable, Iterable, Iterator, cast import numpy as np +import pandas as pd from pandas._libs import lib from pandas._libs.hashing import hash_object_array from pandas._typing import ArrayLike, npt -from pandas.core.dtypes.common import is_categorical_dtype, is_list_like +from pandas.core.dtypes.common import is_list_like from pandas.core.dtypes.generic import ( ABCDataFrame, ABCExtensionArray, @@ -272,7 +273,7 @@ def hash_array( # For categoricals, we hash the categories, then remap the codes to the # hash values. (This check is above the complex check so that we don't ask # numpy if categorical is a subdtype of complex, as it will choke). - if is_categorical_dtype(dtype): + if isinstance(dtype, pd.CategoricalDtype): vals = cast("Categorical", vals) return _hash_categorical(vals, encoding, hash_key) diff --git a/python/xorbits/_mars/dataframe/missing/fillna.py b/python/xorbits/_mars/dataframe/missing/fillna.py index a2a87615e..65bca253b 100644 --- a/python/xorbits/_mars/dataframe/missing/fillna.py +++ b/python/xorbits/_mars/dataframe/missing/fillna.py @@ -41,7 +41,6 @@ class FillNA(DataFrameOperand, DataFrameOperandMixin): _method = StringField("method") _axis = AnyField("axis") _limit = Int64Field("limit") - _downcast = AnyField("downcast") _output_limit = Int64Field("output_limit") @@ -51,7 +50,6 @@ def __init__( method=None, axis=None, limit=None, - downcast=None, output_types=None, output_limit=None, **kw @@ -61,7 +59,6 @@ def __init__( _method=method, _axis=axis, _limit=limit, - _downcast=downcast, _output_types=output_types, _output_limit=output_limit, **kw @@ -83,10 +80,6 @@ def axis(self): def limit(self): return self._limit - @property - def downcast(self): - return self._downcast - def _set_inputs(self, inputs): super()._set_inputs(inputs) if self._method is None and len(inputs) > 1: @@ -123,7 +116,9 @@ def _execute_map(cls, ctx, op): method = op.method filled = input_data.fillna( - method=method, axis=axis, limit=limit, downcast=op.downcast + method=method, + axis=axis, + limit=limit, ) ctx[op.outputs[0].key] = cls._get_first_slice(op, filled, 1) del filled @@ -143,7 +138,9 @@ def _execute_combine(cls, ctx, op): if not summaries: ctx[op.outputs[0].key] = input_data.fillna( - method=method, axis=axis, limit=limit, downcast=op.downcast + method=method, + axis=axis, + limit=limit, ) return @@ -158,7 +155,9 @@ def _execute_combine(cls, ctx, op): if is_pandas_2(): concat_df = concat_df.fillna( - method=method, axis=axis, limit=limit, downcast=op.downcast + method=method, + axis=axis, + limit=limit, ) else: concat_df.fillna( @@ -166,7 +165,6 @@ def _execute_combine(cls, ctx, op): axis=axis, inplace=True, limit=limit, - downcast=op.downcast, ) ctx[op.outputs[0].key] = cls._get_first_slice(op, concat_df, -1) @@ -187,12 +185,9 @@ def execute(cls, ctx, op): method=op.method, axis=op.axis, limit=op.limit, - downcast=op.downcast, ) else: - ctx[op.outputs[0].key] = input_data.fillna( - value=value, downcast=op.downcast - ) + ctx[op.outputs[0].key] = input_data.fillna(value=value) @classmethod def _tile_one_by_one(cls, op): @@ -478,9 +473,7 @@ def __call__(self, a, value_df=None): ) -def fillna( - df, value=None, method=None, axis=None, inplace=False, limit=None, downcast=None -): +def fillna(df, value=None, method=None, axis=None, inplace=False, limit=None): """ Fill NA/NaN values using the specified method. @@ -509,10 +502,6 @@ def fillna( be partially filled. If method is not specified, this is the maximum number of entries along the entire axis where NaNs will be filled. Must be greater than 0 if not None. - downcast : dict, default is None - A dict of item->dtype of what to downcast if possible, - or the string 'infer' which will try to downcast to an appropriate - equal type (e.g. float64 to int64 if possible). Returns ------- @@ -583,10 +572,6 @@ def fillna( % type(value).__name__ ) - if downcast is not None: - raise NotImplementedError( - 'Currently argument "downcast" is not implemented yet' - ) if limit is not None: raise NotImplementedError('Currently argument "limit" is not implemented yet') @@ -600,7 +585,6 @@ def fillna( method=method, axis=axis, limit=limit, - downcast=downcast, output_types=get_output_types(df), ) out_df = op(df, value_df=value_df) @@ -610,7 +594,7 @@ def fillna( return out_df -def ffill(df, axis=None, inplace=False, limit=None, downcast=None): +def ffill(df, axis=None, inplace=False, limit=None): """ Synonym for :meth:`DataFrame.fillna` with ``method='ffill'``. @@ -619,12 +603,10 @@ def ffill(df, axis=None, inplace=False, limit=None, downcast=None): {klass} or None Object with missing values filled or None if ``inplace=True``. """ - return fillna( - df, method="ffill", axis=axis, inplace=inplace, limit=limit, downcast=downcast - ) + return fillna(df, method="ffill", axis=axis, inplace=inplace, limit=limit) -def bfill(df, axis=None, inplace=False, limit=None, downcast=None): +def bfill(df, axis=None, inplace=False, limit=None): """ Synonym for :meth:`DataFrame.fillna` with ``method='bfill'``. @@ -633,12 +615,10 @@ def bfill(df, axis=None, inplace=False, limit=None, downcast=None): {klass} or None Object with missing values filled or None if ``inplace=True``. """ - return fillna( - df, method="bfill", axis=axis, inplace=inplace, limit=limit, downcast=downcast - ) + return fillna(df, method="bfill", axis=axis, inplace=inplace, limit=limit) -def index_fillna(index, value=None, downcast=None): +def index_fillna(index, value=None): """ Fill NA/NaN values with the specified value. @@ -647,10 +627,6 @@ def index_fillna(index, value=None, downcast=None): value : scalar Scalar value to use to fill holes (e.g. 0). This value cannot be a list-likes. - downcast : dict, default is None - A dict of item->dtype of what to downcast if possible, - or the string 'infer' which will try to downcast to an appropriate - equal type (e.g. float64 to int64 if possible). Returns ------- @@ -666,7 +642,6 @@ def index_fillna(index, value=None, downcast=None): op = FillNA( value=value, - downcast=downcast, output_types=get_output_types(index), ) return op(index) diff --git a/python/xorbits/_mars/dataframe/missing/tests/test_missing.py b/python/xorbits/_mars/dataframe/missing/tests/test_missing.py index 0f33f15e9..50a7a7578 100644 --- a/python/xorbits/_mars/dataframe/missing/tests/test_missing.py +++ b/python/xorbits/_mars/dataframe/missing/tests/test_missing.py @@ -56,8 +56,6 @@ def test_fill_na(): series.fillna(value=df) with pytest.raises(ValueError): series.fillna(value=df_raw) - with pytest.raises(NotImplementedError): - series.fillna(value=series_raw, downcast="infer") with pytest.raises(NotImplementedError): series.ffill(limit=1) From 384e45339dfdfd3b30d71f50210eac82db73898f Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Fri, 22 Sep 2023 15:44:37 +0800 Subject: [PATCH 14/27] ENH: Compatible with scikit-learn 1.3.1 (#715) --- python/xorbits/_mars/learn/neighbors/base.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/python/xorbits/_mars/learn/neighbors/base.py b/python/xorbits/_mars/learn/neighbors/base.py index b1f1b21f4..408dd9c8e 100644 --- a/python/xorbits/_mars/learn/neighbors/base.py +++ b/python/xorbits/_mars/learn/neighbors/base.py @@ -24,7 +24,6 @@ from ..metrics import pairwise_distances_topk from ..metrics.pairwise import PAIRWISE_DISTANCE_FUNCTIONS from ..utils import check_array -from ..utils.core import sklearn_version from ..utils.validation import check_is_fitted from ._ball_tree import SklearnBallTree, ball_tree_query, create_ball_tree from ._faiss import METRIC_TO_FAISS_METRIC_TYPE, build_faiss_index, faiss_query @@ -33,12 +32,8 @@ from ._proxima import METRIC_TO_PROXIMA_METRIC_TYPE, build_proxima_index, proxima_query VALID_METRICS = dict( - ball_tree=SklearnBallTree.valid_metrics() - if sklearn_version() >= "1.3.0" - else SklearnBallTree.valid_metrics, - kd_tree=SklearnKDTree.valid_metrics() - if sklearn_version() >= "1.3.0" - else SklearnKDTree.valid_metrics, + ball_tree=SklearnBallTree.valid_metrics, + kd_tree=SklearnKDTree.valid_metrics, # The following list comes from the # sklearn.metrics.pairwise doc string brute=( From ebc391fe0fa55599c3197c52408bd43a4bd9476f Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Sat, 23 Sep 2023 00:48:55 +0800 Subject: [PATCH 15/27] BUG: fix df deep copy (#709) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- python/xorbits/_mars/core/base.py | 2 +- python/xorbits/_mars/core/entity/tileables.py | 27 +++++++++- .../base/tests/test_base_execution.py | 50 +++++++++++++++++++ python/xorbits/_mars/dataframe/core.py | 43 ++++++++++++++-- python/xorbits/_mars/deploy/oscar/session.py | 33 +++++++++++- python/xorbits/core/adapter.py | 6 +++ .../tests/test_pandas_adapters.py | 34 +++++++++++++ 7 files changed, 187 insertions(+), 8 deletions(-) diff --git a/python/xorbits/_mars/core/base.py b/python/xorbits/_mars/core/base.py index 1513a4cc8..dbc71959c 100644 --- a/python/xorbits/_mars/core/base.py +++ b/python/xorbits/_mars/core/base.py @@ -94,7 +94,7 @@ def __copy__(self): return self.copy() def copy(self): - return self.copy_to(type(self)(_key=self.key)) + return self.copy_to(type(self)()) def copy_to(self, target: "Base"): target_fields = target._FIELDS diff --git a/python/xorbits/_mars/core/entity/tileables.py b/python/xorbits/_mars/core/entity/tileables.py index 0a4feda6d..b43ade6ca 100644 --- a/python/xorbits/_mars/core/entity/tileables.py +++ b/python/xorbits/_mars/core/entity/tileables.py @@ -364,7 +364,14 @@ def __copy__(self): def _view(self): return super().copy() - def copy(self: TileableType) -> TileableType: + def copy(self: TileableType, **kw) -> TileableType: + from ...dataframe import Index + from ...deploy.oscar.session import SyncSession + + new_name = None + if isinstance(self, Index): + new_name = kw.pop("name", None) + new_op = self.op.copy() if new_op.create_view: # if the operand is a view, make it a copy @@ -378,6 +385,24 @@ def copy(self: TileableType) -> TileableType: new_outs = new_op.new_tileables( self.op.inputs, kws=params, output_limit=len(params) ) + + sess = self._executed_sessions[-1] if self._executed_sessions else None + to_incref_keys = [] + for _out in new_outs: + if sess: + _out._attach_session(sess) + to_incref_keys.append(_out.key) + if self.data in sess._tileable_to_fetch: + sess._tileable_to_fetch[_out.data] = sess._tileable_to_fetch[ + self.data + ] + if new_name: + _out.name = new_name + + if to_incref_keys: + assert sess is not None + SyncSession.from_isolated_session(sess).incref(*to_incref_keys) + pos = -1 for i, out in enumerate(self.op.outputs): # create a ref to copied one diff --git a/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py b/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py index bb997d233..529812ea5 100644 --- a/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py +++ b/python/xorbits/_mars/dataframe/base/tests/test_base_execution.py @@ -3182,3 +3182,53 @@ def test_nunique(setup, method, chunked, axis): raw_df.nunique(axis=axis), mdf.nunique(axis=axis, method=method).execute().fetch(), ) + + +@pytest.mark.parametrize("chunk_size", [None, 10]) +def test_copy_deep(setup, chunk_size): + ns = np.random.RandomState(0) + df = pd.DataFrame(ns.rand(100, 10), columns=["a" + str(i) for i in range(10)]) + mdf = from_pandas_df(df, chunk_size=chunk_size) + + # test case that there is no other result between copy and origin data + res = mdf.copy() + res["a0"] = res["a0"] + 1 + dfc = df.copy(deep=True) + dfc["a0"] = dfc["a0"] + 1 + pd.testing.assert_frame_equal(res.execute().fetch(), dfc) + pd.testing.assert_frame_equal(mdf.execute().fetch(), df) + + s = pd.Series(ns.randint(0, 100, size=(100,))) + ms = from_pandas_series(s, chunk_size=chunk_size) + + res = ms.copy() + res.iloc[0] = 111.0 + sc = s.copy(deep=True) + sc.iloc[0] = 111.0 + pd.testing.assert_series_equal(res.execute().fetch(), sc) + pd.testing.assert_series_equal(ms.execute().fetch(), s) + + index = pd.Index([i for i in range(100)], name="test") + m_index = from_pandas_index(index, chunk_size=chunk_size) + + res = m_index.copy() + assert res is not m_index + pd.testing.assert_index_equal(res.execute().fetch(), index.copy()) + pd.testing.assert_index_equal(m_index.execute().fetch(), index) + + res = m_index.copy(name="abc") + pd.testing.assert_index_equal(res.execute().fetch(), index.copy(name="abc")) + pd.testing.assert_index_equal(m_index.execute().fetch(), index) + + # test case that there is other ops between copy and origin data + xdf = (mdf + 1) * 2 / 7 + expected = (df + 1) * 2 / 7 + pd.testing.assert_frame_equal(xdf.execute().fetch(), expected) + + xdf_c = xdf.copy() + expected_c = expected.copy(deep=True) + pd.testing.assert_frame_equal(xdf_c.execute().fetch(), expected) + xdf_c["a1"] = xdf_c["a1"] + 0.8 + expected_c["a1"] = expected_c["a1"] + 0.8 + pd.testing.assert_frame_equal(xdf_c.execute().fetch(), expected_c) + pd.testing.assert_frame_equal(xdf.execute().fetch(), expected) diff --git a/python/xorbits/_mars/dataframe/core.py b/python/xorbits/_mars/dataframe/core.py index 342f0d9c0..ff0b6d9fc 100644 --- a/python/xorbits/_mars/dataframe/core.py +++ b/python/xorbits/_mars/dataframe/core.py @@ -1167,6 +1167,37 @@ def to_series(self, index=None, name=None): return series_from_index(self, index=index, name=name) + def copy(self, name=None, deep=False): + """ + Make a copy of this object. + + Name is set on the new object. + + Parameters + ---------- + name : Label, optional + Set name for new object. + deep : bool, default False + + Returns + ------- + Index + Index refer to new object which is a copy of this object. + + Notes + ----- + In most cases, there should be no functional difference from using + ``deep``, but if ``deep`` is passed it will attempt to deepcopy. + + Examples + -------- + >>> idx = pd.Index(['a', 'b', 'c']) + >>> new_idx = idx.copy() + >>> idx is new_idx + False + """ + return super().copy(name=name) + class RangeIndex(Index): __slots__ = () @@ -1591,10 +1622,9 @@ def copy(self, deep=True): # pylint: disable=arguments-differ copy : Series or DataFrame Object type matches caller. """ - if deep: - return super().copy() - else: - return super()._view() + if deep is False: + raise NotImplementedError("Not support `deep=False` for now") + return super().copy() def __len__(self): return len(self._data) @@ -2618,6 +2648,11 @@ def apply_if_callable(maybe_callable, obj, **kwargs): data[k] = apply_if_callable(v, data) return data + def copy(self, deep=True): + if deep is False: + raise NotImplementedError("Not support `deep=False` for now") + return super().copy() + class DataFrameGroupByChunkData(BaseDataFrameChunkData): type_name = "DataFrameGroupBy" diff --git a/python/xorbits/_mars/deploy/oscar/session.py b/python/xorbits/_mars/deploy/oscar/session.py index 2fe2d10dd..8e1ac5b09 100644 --- a/python/xorbits/_mars/deploy/oscar/session.py +++ b/python/xorbits/_mars/deploy/oscar/session.py @@ -502,6 +502,17 @@ def decref(self, *tileables_keys): Tileables' keys """ + @abstractmethod + def incref(self, *tileables_keys): + """ + Incref tileables. + + Parameters + ---------- + tileables_keys : list + Tileables' keys + """ + @abstractmethod def _get_ref_counts(self) -> Dict[str, int]: """ @@ -960,10 +971,19 @@ async def execute(self, *tileables, **kwargs) -> ExecutionInfo: def _get_to_fetch_tileable( self, tileable: TileableType ) -> Tuple[TileableType, List[Union[slice, Integral]]]: - from ...dataframe.indexing.iloc import DataFrameIlocGetItem, SeriesIlocGetItem + from ...dataframe.indexing.iloc import ( + DataFrameIlocGetItem, + IndexIlocGetItem, + SeriesIlocGetItem, + ) from ...tensor.indexing import TensorIndex - slice_op_types = TensorIndex, DataFrameIlocGetItem, SeriesIlocGetItem + slice_op_types = ( + TensorIndex, + DataFrameIlocGetItem, + SeriesIlocGetItem, + IndexIlocGetItem, + ) if hasattr(tileable, "data"): tileable = tileable.data @@ -1200,6 +1220,10 @@ async def decref(self, *tileable_keys): logger.debug("Decref tileables on client: %s", tileable_keys) return await self._lifecycle_api.decref_tileables(list(tileable_keys)) + async def incref(self, *tileable_keys): + logger.debug("Incref tileables on client: %s", tileable_keys) + return await self._lifecycle_api.incref_tileables(list(tileable_keys)) + async def _get_ref_counts(self) -> Dict[str, int]: return await self._lifecycle_api.get_all_chunk_ref_counts() @@ -1623,6 +1647,11 @@ def fetch_infos(self, *tileables, fields, **kwargs) -> list: def decref(self, *tileables_keys): pass # pragma: no cover + @implements(AbstractSyncSession.incref) + @_delegate_to_isolated_session + def incref(self, *tileables_keys): + pass # pragma: no cover + @implements(AbstractSyncSession._get_ref_counts) @_delegate_to_isolated_session def _get_ref_counts(self) -> Dict[str, int]: diff --git a/python/xorbits/core/adapter.py b/python/xorbits/core/adapter.py index bd3d93317..92e1dbdc5 100644 --- a/python/xorbits/core/adapter.py +++ b/python/xorbits/core/adapter.py @@ -495,6 +495,12 @@ def collect_cls_members( ) -> Dict[str, Any]: cls_members: Dict[str, Any] = {} for name, cls_member in inspect.getmembers(cls): + # Tileable and TileableData object may have functions that have the same names. + # For example, Index and IndexData both have `copy` function, but they have completely different semantics. + # Therefore, when the Index's `copy` method has been collected, + # the method of the same name on IndexData cannot be collected again. + if cls.__name__.endswith("Data") and name in DATA_MEMBERS[data_type]: # type: ignore + continue if inspect.isfunction(cls_member) and not name.startswith("_"): cls_members[name] = wrap_mars_callable( cls_member, diff --git a/python/xorbits/pandas/pandas_adapters/tests/test_pandas_adapters.py b/python/xorbits/pandas/pandas_adapters/tests/test_pandas_adapters.py index cd2ce21f8..14f2f5595 100644 --- a/python/xorbits/pandas/pandas_adapters/tests/test_pandas_adapters.py +++ b/python/xorbits/pandas/pandas_adapters/tests/test_pandas_adapters.py @@ -22,6 +22,7 @@ from .... import pandas as xpd from ....core.data import DataRef +from ....core.execution import need_to_execute def test_pandas_dataframe_methods(setup): @@ -499,3 +500,36 @@ def test_read_pickle(setup): assert (x == y).all() finally: shutil.rmtree(tempdir) + + +def test_copy(setup): + index = xpd.Index([i for i in range(100)], name="test") + index_iloc = index[:20] + assert need_to_execute(index_iloc) is True + repr(index_iloc) + + index_copy = index_iloc.copy() + assert need_to_execute(index_copy) is False + pd.testing.assert_index_equal(index_copy.to_pandas(), index_iloc.to_pandas()) + + index_copy = index_iloc.copy(name="abc") + assert need_to_execute(index_copy) is True + pd.testing.assert_index_equal( + index_copy.to_pandas(), index_iloc.to_pandas().copy(name="abc") + ) + + series = xpd.Series([1, 2, 3, 4, np.nan, 6]) + series = series + 1 + assert need_to_execute(series) is True + repr(series) + + sc = series.copy() + assert need_to_execute(sc) is False + expected = series.to_pandas() + pd.testing.assert_series_equal(sc.to_pandas(), expected) + + sc[0] = np.nan + assert need_to_execute(sc) is True + ec = expected.copy() + ec[0] = np.nan + pd.testing.assert_series_equal(sc.to_pandas(), ec) From b4d70ccd68ee99bcf978bf456f619e91c5294f85 Mon Sep 17 00:00:00 2001 From: leojia <59967634+JiaYaobo@users.noreply.github.com> Date: Mon, 25 Sep 2023 14:26:22 +0800 Subject: [PATCH 16/27] FEAT: Add `Dataframe.Groupby.nth` (#684) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../_mars/dataframe/groupby/__init__.py | 2 + python/xorbits/_mars/dataframe/groupby/nth.py | 232 ++++++++++++++++++ .../groupby/tests/test_groupby_execution.py | 98 ++++++++ 3 files changed, 332 insertions(+) create mode 100644 python/xorbits/_mars/dataframe/groupby/nth.py diff --git a/python/xorbits/_mars/dataframe/groupby/__init__.py b/python/xorbits/_mars/dataframe/groupby/__init__.py index 4b76ca6c4..c4369be4e 100644 --- a/python/xorbits/_mars/dataframe/groupby/__init__.py +++ b/python/xorbits/_mars/dataframe/groupby/__init__.py @@ -26,6 +26,7 @@ def _install(): from .fill import bfill, ffill, fillna from .getitem import df_groupby_getitem from .head import head + from .nth import nth # Just for enabling custom agg function registration. # Therefore, del this immediately after import. @@ -73,6 +74,7 @@ def _install(): setattr(cls, "cumsum", cumsum) setattr(cls, "head", head) + setattr(cls, "nth", nth) setattr(cls, "rolling", rolling) diff --git a/python/xorbits/_mars/dataframe/groupby/nth.py b/python/xorbits/_mars/dataframe/groupby/nth.py new file mode 100644 index 000000000..da2cb14a0 --- /dev/null +++ b/python/xorbits/_mars/dataframe/groupby/nth.py @@ -0,0 +1,232 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import numpy as np +import pandas as pd + +from ...core import OutputType, get_output_types, recursive_tile +from ...serialization.serializables import DictField, IndexField, StringField +from ..core import IndexValue +from ..operands import DataFrameOperand, DataFrameOperandMixin +from ..utils import build_concatenated_rows_frame, parse_index + + +class GroupByNthSelector(DataFrameOperand, DataFrameOperandMixin): + _op_module_ = "dataframe.groupby" + + groupby_params = DictField("groupby_params") + n = IndexField("n") + dropna = StringField("dropna", default=None) + + def __call__(self, groupby): + df = groupby + while df.op.output_types[0] not in (OutputType.dataframe, OutputType.series): + df = df.inputs[0] + selection = groupby.op.groupby_params.pop("selection", None) + if df.ndim > 1 and selection: + if isinstance(selection, tuple) and selection not in df.dtypes: + selection = list(selection) + + result_df = df[selection] + else: + result_df = df + + self._output_types = ( + [OutputType.dataframe] if result_df.ndim == 2 else [OutputType.series] + ) + params = result_df.params + params["shape"] = (np.nan,) + result_df.shape[1:] + if isinstance(df.index_value.value, IndexValue.RangeIndex): + params["index_value"] = parse_index(pd.RangeIndex(-1), df.key) + + return self.new_tileable([df], **params) + + @classmethod + def tile(cls, op: "GroupByNthSelector"): + in_df = op.inputs[0] + groupby_params = op.groupby_params.copy() + selection = groupby_params.pop("selection", None) + if len(in_df.shape) > 1: + in_df = build_concatenated_rows_frame(in_df) + out_df = op.outputs[0] + # if there is only one chunk, tile with a single chunk + if len(in_df.chunks) <= 1: + new_shape = (np.nan,) + new_nsplits = ((np.nan,),) + if out_df.ndim > 1: + new_shape += (out_df.shape[1],) + new_nsplits += ((out_df.shape[1],),) + c = in_df.chunks[0] + chunk_op = op.copy().reset_key() + params = out_df.params + params["shape"] = new_shape + params["index"] = (0,) * out_df.ndim + out_chunk = chunk_op.new_chunk([c], **params) + + tileable_op = op.copy().reset_key() + return tileable_op.new_tileables( + [in_df], nsplits=new_nsplits, chunks=[out_chunk], **params + ) + + if in_df.ndim > 1 and selection: + if isinstance(selection, tuple) and selection not in in_df.dtypes: + selection = list(selection) + + if not isinstance(selection, list): + pre_selection = [selection] + else: + pre_selection = list(selection) + + if isinstance(groupby_params.get("by"), list): + pre_selection += [ + el for el in groupby_params["by"] if el not in pre_selection + ] + + if len(pre_selection) != in_df.shape[1]: + in_df = yield from recursive_tile(in_df[pre_selection]) + + # pre chunks + pre_chunks = [] + for c in in_df.chunks: + pre_op = op.copy().reset_key() + pre_op._output_types = get_output_types(c) + pre_op.groupby_params = op.groupby_params.copy() + pre_op.groupby_params.pop("selection", None) + params = c.params + params["shape"] = (np.nan,) + c.shape[1:] + pre_chunks.append(pre_op.new_chunk([c], **params)) + + new_op = op.copy().reset_key() + new_op._output_types = get_output_types(in_df) + new_nsplits = ((np.nan,) * len(in_df.nsplits[0]),) + in_df.nsplits[1:] + pre_tiled = new_op.new_tileable( + [in_df], chunks=pre_chunks, nsplits=new_nsplits, **in_df.params + ) + # generate groupby + grouped = yield from recursive_tile(pre_tiled.groupby(**groupby_params)) + if selection: + grouped = yield from recursive_tile(grouped[selection]) + + # generate post chunks + post_chunks = [] + for c in grouped.chunks: + post_op = op.copy().reset_key() + post_op.groupby_params = op.groupby_params.copy() + post_op.groupby_params.pop("selection", None) + if op.output_types[0] == OutputType.dataframe: + index = c.index + else: + index = (c.index[0],) + params = out_df.params + params["index"] = index + post_chunks.append(post_op.new_chunk([c], **params)) + + new_op = op.copy().reset_key() + new_nsplits = ((np.nan,) * len(in_df.nsplits[0]),) + if out_df.ndim > 1: + new_nsplits += ((out_df.shape[1],),) + return new_op.new_tileables( + [in_df], chunks=post_chunks, nsplits=new_nsplits, **out_df.params + ) + + @classmethod + def execute(cls, ctx, op: "GroupByNthSelector"): + in_data = ctx[op.inputs[0].key] + params = op.groupby_params.copy() + selection = params.pop("selection", None) + + if hasattr(in_data, "groupby"): + grouped = in_data.groupby(**params) + else: + grouped = in_data + if selection: + grouped = grouped[selection] + result = grouped.nth(op.n, op.dropna) + ctx[op.outputs[0].key] = result + + +def nth(groupby, n, dropna=None): + """ + Take the nth row from each group if n is an int, or a subset of rows + if n is a list of ints. + + If dropna, will take the nth non-null row, dropna is either + Truthy (if a Series) or 'all', 'any' (if a DataFrame); + this is equivalent to calling dropna(how=dropna) before the + groupby. + + Parameters + ---------- + n : int or list of ints + a single nth value for the row or a list of nth values + dropna : None or str, optional + apply the specified dropna operation before counting which row is + the nth row. Needs to be None, 'any' or 'all' + + Examples + -------- + >>> import mars.dataframe as md + >>> df = md.DataFrame({'A': [1, 1, 2, 1, 2], + ... 'B': [np.nan, 2, 3, 4, 5]}, columns=['A', 'B']) + >>> g = df.groupby('A') + >>> g.nth(0).execute() + B + A + 1 NaN + 2 3.0 + >>> g.nth(1).execute() + B + A + 1 2.0 + 2 5.0 + >>> g.nth(-1).execute() + B + A + 1 4.0 + 2 5.0 + >>> g.nth([0, 1]).execute() + B + A + 1 NaN + 1 2.0 + 2 3.0 + 2 5.0 + + Specifying ``dropna`` allows count ignoring NaN + + >>> g.nth(0, dropna='any').execute() + B + A + 1 2.0 + 2 3.0 + + NaNs denote group exhausted when using dropna + + >>> g.nth(3, dropna='any').execute() + B + A + 1 NaN + 2 NaN + + Specifying ``as_index=False`` in ``groupby`` keeps the original index. + + >>> df.groupby('A', as_index=False).nth(1).execute() + A B + 1 1 2.0 + 4 2 5.0 + """ + groupby_params = groupby.op.groupby_params.copy() + groupby_params.pop("as_index", None) + op = GroupByNthSelector(n=n, dropna=dropna, groupby_params=groupby_params) + return op(groupby) diff --git a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_execution.py b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_execution.py index 003870aed..f3ebbc56d 100644 --- a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_execution.py @@ -1878,3 +1878,101 @@ def test_series_groupby_rolling_agg(setup, window, min_periods, center, closed, mresult = mresult.execute().fetch() pd.testing.assert_series_equal(presult, mresult.sort_index()) + + +@pytest.mark.skipif(pd.__version__ <= "1.5.3", reason="pandas version is too low") +@pytest.mark.parametrize( + "chunk_size, dropna", list(product([None, 3], [None, "any", "all"])) +) +def test_groupby_nth(setup, chunk_size, dropna): + df1 = pd.DataFrame( + { + "a": np.random.randint(0, 5, size=20), + "b": np.random.randint(0, 5, size=20), + "c": np.random.randint(0, 5, size=20), + "d": np.random.randint(0, 5, size=20), + } + ) + mdf = md.DataFrame(df1, chunk_size=chunk_size) + + r = mdf.groupby("b").nth(0) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b").nth(0) + ) + r = mdf.groupby("b").nth(-1) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b").nth(-1) + ) + r = mdf.groupby("b")[["a", "c"]].nth(0) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b")[["a", "c"]].nth(0) + ) + + # test nth with list index + r = mdf.groupby("b").nth([0, 1]) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b").nth([0, 1]) + ) + + # test nth with slice + r = mdf.groupby("b").nth(slice(None, 1)) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b").nth(slice(None, 1)) + ) + + # test nth with selection + r = mdf.groupby("b")[["a", "d"]].nth(0) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b")[["a", "d"]].nth(0) + ) + r = mdf.groupby("b")[["c", "a", "d"]].nth(0) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df1.groupby("b")[["c", "a", "d"]].nth(0) + ) + r = mdf.groupby("b")["c"].nth(0) + pd.testing.assert_series_equal( + r.execute().fetch().sort_index(), df1.groupby("b")["c"].nth(0) + ) + + series1 = pd.Series([3, 4, 5, 3, 5, 4, 1, 2, 3]) + ms = md.Series(series1, chunk_size=chunk_size) + + r = ms.groupby(lambda x: x % 2).nth(0) + pd.testing.assert_series_equal( + r.execute().fetch().sort_index(), series1.groupby(lambda x: x % 2).nth(0) + ) + + # test with special index + series1 = pd.Series([3, 4, 5, 3, 5, 4, 1, 2, 3], index=[4, 1, 2, 3, 5, 8, 6, 7, 9]) + ms = md.Series(series1, chunk_size=chunk_size) + + r = ms.groupby(lambda x: x % 2).nth(0) + pd.testing.assert_series_equal( + r.execute().fetch().sort_index(), + series1.groupby(lambda x: x % 2).nth(0).sort_index(), + ) + + df2 = pd.DataFrame( + { + "a": [3, 5, 2, np.nan, 1, 2, 4, 6, 2, 4], + "b": [8, 3, 4, 1, 8, np.nan, 2, 2, 2, 3], + "c": [1, 8, 8, np.nan, 3, 5, 0, 0, 5, 4], + "d": [np.nan, 7, 6, 3, 6, 3, 2, 1, 5, 8], + } + ) + + mdf = md.DataFrame(df2) + + r = mdf.groupby("b").nth(0, dropna=dropna) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df2.groupby("b").nth(0, dropna=dropna) + ) + r = mdf.groupby("b").nth(-1, dropna=dropna) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), df2.groupby("b").nth(-1, dropna=dropna) + ) + r = mdf.groupby("b")[["a", "c"]].nth(0, dropna=dropna) + pd.testing.assert_frame_equal( + r.execute().fetch().sort_index(), + df2.groupby("b")[["a", "c"]].nth(0, dropna=dropna), + ) From 9bff550d42a1243afabad2b2366d1da82b2a3c63 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Tue, 26 Sep 2023 18:36:45 +0800 Subject: [PATCH 17/27] BUG: df.groupby.agg error when func is nunique with tuple kwargs (#722) --- .github/workflows/python.yaml | 2 +- .../_mars/dataframe/groupby/aggregation.py | 6 ++ .../_mars/dataframe/groupby/nunique.py | 5 ++ .../tests/test_groupby_nunique_execution.py | 73 +++++++++++++++++++ 4 files changed, 85 insertions(+), 1 deletion(-) diff --git a/.github/workflows/python.yaml b/.github/workflows/python.yaml index f0f93a408..a508a6f9a 100644 --- a/.github/workflows/python.yaml +++ b/.github/workflows/python.yaml @@ -250,7 +250,7 @@ jobs: - name: Install on GPU if: ${{ matrix.module == 'gpu' }} run: | - pip install -e "git+https://github.com/xorbitsai/xoscar.git@main#subdirectory=python&egg=xoscar" + pip install -U xoscar python setup.py build_ext -i working-directory: ./python diff --git a/python/xorbits/_mars/dataframe/groupby/aggregation.py b/python/xorbits/_mars/dataframe/groupby/aggregation.py index 12c3935e6..a46a002ba 100644 --- a/python/xorbits/_mars/dataframe/groupby/aggregation.py +++ b/python/xorbits/_mars/dataframe/groupby/aggregation.py @@ -988,6 +988,12 @@ def _pack_inputs(agg_funcs: List[ReductionAggStep], in_data): def _do_custom_agg( func_name: str, op: "DataFrameGroupByAgg", in_data: pd.DataFrame ) -> Union[pd.Series, pd.DataFrame]: + # Must be tuple way, like x=('col', 'agg_func_name') + # See `is_funcs_aggregate` func, + # if not this way, the code doesn't go here or switch to transform execution. + if op.raw_func is None: + func_name = list(op.raw_func_kw.values())[0][1] + if op.stage == OperandStage.map: return custom_agg_functions[func_name].execute_map(op, in_data) elif op.stage == OperandStage.combine: diff --git a/python/xorbits/_mars/dataframe/groupby/nunique.py b/python/xorbits/_mars/dataframe/groupby/nunique.py index 424b9797e..41273a9a8 100644 --- a/python/xorbits/_mars/dataframe/groupby/nunique.py +++ b/python/xorbits/_mars/dataframe/groupby/nunique.py @@ -136,6 +136,11 @@ def _get_execute_agg_result( # since level field in op.groupby_params is not correct. groupby_params["level"] = op.raw_groupby_params["level"] + # For the tuple usage: .agg(x=('a', 'nunique')), firstly set `as_index=True`. + # Otherwise, subsequent processing will lose the information about the grouped columns. + # TODO: This is due to `reduction` functions, but for now, let's keep it simple. + if op.raw_func is None: + groupby_params["as_index"] = True res = in_data.groupby(**groupby_params).nunique() return res diff --git a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py index 08bb49706..fba9d9516 100644 --- a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py +++ b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py @@ -11,6 +11,8 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +import itertools + import numpy as np import pandas as pd import pytest @@ -332,3 +334,74 @@ def test_groupby_agg_nunique(setup, gen_data1): ) expected = df.groupby("b", sort=sort).agg(["sum", "nunique"]) pd.testing.assert_frame_equal(r.sort_index(), expected.sort_index()) + + +@pytest.mark.parametrize( + "chunk_size, as_index, sort", + itertools.product([None, 13], [True, False], [True, False]), +) +def test_groupby_agg_nunique_with_tuple_kwargs( + setup, gen_data2, chunk_size, as_index, sort +): + df = gen_data2 + mdf = md.DataFrame(df, chunk_size=chunk_size) + + res = mdf.groupby("b", as_index=as_index, sort=sort).agg(e=("a", "nunique")) + expected = df.groupby("b", as_index=as_index, sort=sort).agg(e=("a", "nunique")) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("c", "nunique") + ) + expected = df.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("c", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "sum") + ) + expected = df.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "sum") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "sum"), g=("c", "nunique") + ) + expected = df.groupby("b", as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "sum"), g=("c", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # by multi columns + res = mdf.groupby(["b", "c"], as_index=as_index, sort=sort).agg(e=("a", "nunique")) + expected = df.groupby(["b", "c"], as_index=as_index, sort=sort).agg( + e=("a", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby(["b", "c"], as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "mean") + ) + expected = df.groupby(["b", "c"], as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("d", "mean") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # index after groupby + res = mdf.groupby(["b"], as_index=as_index, sort=sort)[["a"]].agg( + e=("a", "nunique") + ) + expected = df.groupby(["b"], as_index=as_index, sort=sort)[["a"]].agg( + e=("a", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby(["b"], as_index=as_index, sort=sort)[["a", "c"]].agg( + e=("a", "nunique"), f=("c", "nunique") + ) + expected = df.groupby(["b"], as_index=as_index, sort=sort)[["a", "c"]].agg( + e=("a", "nunique"), f=("c", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) From 882f1b97b37fba56bc0c39196d3d09200ad33b68 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Sun, 8 Oct 2023 02:18:23 -0500 Subject: [PATCH 18/27] ENH: groupby.nunique supports by series (#726) --- .../_mars/dataframe/groupby/aggregation.py | 48 ++++++++---- .../_mars/dataframe/groupby/nunique.py | 61 ++++++++++++--- .../tests/test_groupby_nunique_execution.py | 77 +++++++++++++++++++ 3 files changed, 164 insertions(+), 22 deletions(-) diff --git a/python/xorbits/_mars/dataframe/groupby/aggregation.py b/python/xorbits/_mars/dataframe/groupby/aggregation.py index a46a002ba..1d038944b 100644 --- a/python/xorbits/_mars/dataframe/groupby/aggregation.py +++ b/python/xorbits/_mars/dataframe/groupby/aggregation.py @@ -27,6 +27,7 @@ from ...core import ENTITY_TYPE, OutputType from ...core.context import get_context from ...core.custom_log import redirect_custom_log +from ...core.entity.utils import recursive_tile from ...core.operand import OperandStage from ...serialization.serializables import ( AnyField, @@ -480,6 +481,7 @@ def _gen_map_chunks( # force as_index=True for map phase map_op.output_types = op.output_types map_op.groupby_params = map_op.groupby_params.copy() + map_op.raw_groupby_params = map_op.raw_groupby_params.copy() map_op.groupby_params["as_index"] = True if isinstance(map_op.groupby_params["by"], list): by = [] @@ -491,6 +493,7 @@ def _gen_map_chunks( else: by.append(v) map_op.groupby_params["by"] = by + map_op.raw_groupby_params["by"] = by map_op.stage = OperandStage.map map_op.pre_funcs = func_infos.pre_funcs map_op.agg_funcs = func_infos.agg_funcs @@ -926,6 +929,20 @@ def tile(cls, op: "DataFrameGroupByAgg"): in_df = build_concatenated_rows_frame(in_df) out_df = op.outputs[0] + by = op.groupby_params["by"] + in_df_nsplits_settled: bool = all([not np.isnan(v) for v in in_df.nsplits[0]]) + if isinstance(by, list): + for i, _by in enumerate(by): + if ( + isinstance(_by, ENTITY_TYPE) + and all([not np.isnan(v) for v in _by.nsplits[0]]) + and in_df_nsplits_settled + ): + by[i] = yield from recursive_tile( + _by.rechunk({0: in_df.nsplits[0]}) + ) + yield by[i].chunks + func_infos = cls._compile_funcs(op, in_df) if op.method == "auto": @@ -943,6 +960,10 @@ def tile(cls, op: "DataFrameGroupByAgg"): else: # pragma: no cover raise NotImplementedError + @classmethod + def _get_new_by_data(cls, by: List, ctx: Dict): + return [ctx[v.key] if isinstance(v, ENTITY_TYPE) else v for v in by] + @classmethod def _get_grouped(cls, op: "DataFrameGroupByAgg", df, ctx, copy=False, grouper=None): if copy: @@ -956,13 +977,7 @@ def _get_grouped(cls, op: "DataFrameGroupByAgg", df, ctx, copy=False, grouper=No params["by"] = grouper params.pop("level", None) elif isinstance(params.get("by"), list): - new_by = [] - for v in params["by"]: - if isinstance(v, ENTITY_TYPE): - new_by.append(ctx[v.key]) - else: - new_by.append(v) - params["by"] = new_by + params["by"] = cls._get_new_by_data(params["by"], ctx) grouped = df.groupby(**params) @@ -984,16 +999,23 @@ def _pack_inputs(agg_funcs: List[ReductionAggStep], in_data): pos += step.output_limit return out_dict - @staticmethod + @classmethod def _do_custom_agg( - func_name: str, op: "DataFrameGroupByAgg", in_data: pd.DataFrame + cls, func_name: str, op: "DataFrameGroupByAgg", in_data: pd.DataFrame, ctx: Dict ) -> Union[pd.Series, pd.DataFrame]: # Must be tuple way, like x=('col', 'agg_func_name') # See `is_funcs_aggregate` func, # if not this way, the code doesn't go here or switch to transform execution. if op.raw_func is None: func_name = list(op.raw_func_kw.values())[0][1] - + if ( + func_name == "nunique" + and "by" in op.groupby_params + and isinstance(op.groupby_params["by"], list) + ): + op.raw_groupby_params["by"] = cls._get_new_by_data( + op.groupby_params["by"], ctx + ) if op.stage == OperandStage.map: return custom_agg_functions[func_name].execute_map(op, in_data) elif op.stage == OperandStage.combine: @@ -1111,7 +1133,7 @@ def _wrapped_func(col): ) in op.agg_funcs: input_obj = ret_map_groupbys[input_key] if map_func_name == "custom_reduction": - agg_dfs.append(cls._do_custom_agg(raw_func_name, op, in_data)) + agg_dfs.append(cls._do_custom_agg(raw_func_name, op, in_data, ctx)) else: single_func = map_func_name == op.raw_func agg_dfs.append( @@ -1159,7 +1181,7 @@ def _execute_combine(cls, ctx, op: "DataFrameGroupByAgg"): ) in zip(ctx[op.inputs[0].key], op.agg_funcs): input_obj = in_data_dict[output_key] if agg_func_name == "custom_reduction": - combines.append(cls._do_custom_agg(raw_func_name, op, raw_input)) + combines.append(cls._do_custom_agg(raw_func_name, op, raw_input, ctx)) else: combines.append( cls._do_predefined_agg(input_obj, agg_func_name, gpu=op.gpu, **kwds) @@ -1200,7 +1222,7 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): ) in op.agg_funcs: if agg_func_name == "custom_reduction": in_data_dict[output_key] = cls._do_custom_agg( - raw_func_name, op, in_data_dict[output_key] + raw_func_name, op, in_data_dict[output_key], ctx ) else: input_obj = cls._get_grouped(op, in_data_dict[output_key], ctx) diff --git a/python/xorbits/_mars/dataframe/groupby/nunique.py b/python/xorbits/_mars/dataframe/groupby/nunique.py index 41273a9a8..c07577ba9 100644 --- a/python/xorbits/_mars/dataframe/groupby/nunique.py +++ b/python/xorbits/_mars/dataframe/groupby/nunique.py @@ -15,8 +15,9 @@ import pandas as pd -from ...core import OutputType +from ...core import ENTITY_TYPE, OutputType from ...utils import implements +from ..utils import is_dataframe from .aggregation import DataFrameGroupByAgg from .custom_aggregation import ( DataFrameCustomGroupByAggMixin, @@ -79,6 +80,22 @@ def _get_selection_columns(cls, op: DataFrameGroupByAgg) -> Union[None, List]: selection = [selection] return selection + @classmethod + def _drop_duplicates_by_series(cls, in_data: pd.DataFrame, origin_cols: List): + if isinstance(in_data.index, pd.MultiIndex): + origin_index_name = in_data.index.names + else: + origin_index_name = in_data.index.name + res = in_data.reset_index() + new_cols = list(res.columns) + index_cols = [v for v in new_cols if v not in origin_cols] + res = res.drop_duplicates().set_index(index_cols) + if isinstance(res.index, pd.MultiIndex): + res.index.names = origin_index_name + else: + res.index.name = origin_index_name + return res + @classmethod def _get_execute_map_result( cls, op: DataFrameGroupByAgg, in_data: pd.DataFrame @@ -86,10 +103,23 @@ def _get_execute_map_result( selections = cls._get_selection_columns(op) by_cols = op.raw_groupby_params["by"] if by_cols is not None: - cols = ( - [*selections, *by_cols] if selections is not None else in_data.columns - ) - res = in_data[cols].drop_duplicates(subset=cols).set_index(by_cols) + # When `by` some series, the series will be used to determine the groups. + # We first need to set the index of the data to these series, + # and then `reset_index` to let these series become data columns. + # Next bring these columns for `drop_duplicates` and reset these columns to index. + if isinstance(by_cols, list) and any( + [isinstance(v, pd.Series) for v in by_cols] + ): + origin_cols = list(in_data.columns) + res = in_data.set_index(by_cols) + res = cls._drop_duplicates_by_series(res, origin_cols) + else: + cols = ( + [*selections, *by_cols] + if selections is not None + else in_data.columns + ) + res = in_data[cols].drop_duplicates(subset=cols).set_index(by_cols) else: # group by level selections = selections if selections is not None else in_data.columns level_indexes = cls._get_level_indexes(op, in_data) @@ -111,9 +141,17 @@ def _get_execute_map_result( def _get_execute_combine_result( cls, op: DataFrameGroupByAgg, in_data: pd.DataFrame ) -> Union[pd.DataFrame, pd.Series]: - # in_data.index.names means MultiIndex (groupby on multi cols) - index_col = in_data.index.name or in_data.index.names - res = in_data.reset_index().drop_duplicates().set_index(index_col) + by = op.raw_groupby_params["by"] + if isinstance(by, list) and any([isinstance(v, ENTITY_TYPE) for v in by]): + # `in_data` may be series when there is index op after groupby + origin_cols = ( + list(in_data.columns) if is_dataframe(in_data) else [in_data.name] + ) + res = cls._drop_duplicates_by_series(in_data, origin_cols) + else: + # in_data.index.names means MultiIndex (groupby on multi cols) + index_col = in_data.index.name or in_data.index.names + res = in_data.reset_index().drop_duplicates().set_index(index_col) if op.output_types[0] == OutputType.series: res = res.squeeze() return res @@ -127,7 +165,12 @@ def _get_execute_agg_result( by = op.raw_groupby_params["by"] if by is not None: - if op.output_types[0] == OutputType.dataframe: + if isinstance(by, list) and any( + [isinstance(_by, ENTITY_TYPE) for _by in by] + ): + # nothing to do here, just group by level is correct + pass + elif op.output_types[0] == OutputType.dataframe: groupby_params.pop("level", None) groupby_params["by"] = cols in_data = in_data.reset_index() diff --git a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py index fba9d9516..68e678230 100644 --- a/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py +++ b/python/xorbits/_mars/dataframe/groupby/tests/test_groupby_nunique_execution.py @@ -405,3 +405,80 @@ def test_groupby_agg_nunique_with_tuple_kwargs( e=("a", "nunique"), f=("c", "nunique") ) pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + +@pytest.mark.parametrize( + "chunk_size, as_index, sort", + itertools.product([None, 13], [True, False], [True, False]), +) +def test_groupby_nunique_by_series(setup, gen_data2, chunk_size, as_index, sort): + df = gen_data2 + mdf = md.DataFrame(df, chunk_size=chunk_size) + + by1 = pd.Series([i + 100 for i in range(100)]) + mby1 = md.Series(by1) + + by2 = pd.Series([i + 200 for i in range(100)]) + mby2 = md.Series(by2) + + res = mdf.groupby(mby1, as_index=as_index, sort=sort).nunique() + expected = df.groupby(by1, as_index=as_index, sort=sort).nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby([mby1, mby2], as_index=as_index, sort=sort).nunique() + expected = df.groupby([by1, by2], as_index=as_index, sort=sort).nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby([mby1, mby2], as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("c", "nunique") + ) + expected = df.groupby([by1, by2], as_index=as_index, sort=sort).agg( + e=("a", "nunique"), f=("c", "nunique") + ) + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # test by with duplicates + rs = np.random.RandomState(0) + by3 = pd.Series(rs.choice([i for i in range(1, 6)], size=(100,))) + mby3 = md.Series(by3) + + res = mdf.groupby(mby3, as_index=as_index, sort=sort).nunique() + expected = df.groupby(by3, as_index=as_index, sort=sort).nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # test by other chunk size + by4 = pd.Series(rs.choice([i for i in range(10)], size=(100,))) + mby4 = md.Series(by4, chunk_size=21) + + res = mdf.groupby(mby4, as_index=as_index, sort=sort).nunique() + expected = df.groupby(by4, as_index=as_index, sort=sort).nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # test index after groupby + res = mdf.groupby(mby3, as_index=as_index, sort=sort)[["a", "b"]].nunique() + expected = df.groupby(by3, as_index=as_index, sort=sort)[["a", "b"]].nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby(mby3, as_index=as_index, sort=sort)[["a"]].nunique() + expected = df.groupby(by3, as_index=as_index, sort=sort)[["a"]].nunique() + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + res = mdf.groupby(mby3, as_index=as_index, sort=sort)["a"].nunique() + expected = df.groupby(by3, as_index=as_index, sort=sort)["a"].nunique() + if as_index: + pd.testing.assert_series_equal(res.execute().fetch(), expected) + else: + pd.testing.assert_frame_equal(res.execute().fetch(), expected) + + # test different methods + for method in ["auto", "tree", "shuffle"]: + res = mdf.groupby(mby3, as_index=as_index, sort=sort).nunique(method=method) + expected = df.groupby(by3, as_index=as_index, sort=sort).nunique() + real = res.execute().fetch() + if method == "shuffle": + pd.testing.assert_frame_equal( + real.sort_values(["a", "b", "c", "d"]).reset_index(drop=True), + expected.sort_values(["a", "b", "c", "d"]).reset_index(drop=True), + ) + else: + pd.testing.assert_frame_equal(real, expected) From 771858d2987d22583d03f482cfcf390822bed1ea Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Sun, 8 Oct 2023 03:44:42 -0500 Subject: [PATCH 19/27] BUG: read parquet from s3 error: No Such Bucket (#732) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .pre-commit-config.yaml | 2 +- .../_mars/dataframe/datasource/read_parquet.py | 13 +++---------- python/xorbits/_mars/learn/linear_model/_base.py | 2 +- .../_mars/learn/linear_model/tests/test_base.py | 4 ++-- .../_mars/learn/metrics/pairwise/pairwise.py | 2 +- python/xorbits/_mars/tensor/base/tile.py | 2 +- python/xorbits/_mars/tensor/utils.py | 2 +- 7 files changed, 10 insertions(+), 17 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index a7239dfcb..c1ed9e8c5 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -32,7 +32,7 @@ repos: - id: prettier types_or: [html, javascript] - repo: https://github.com/codespell-project/codespell - rev: v2.2.5 + rev: v2.2.6 hooks: - id: codespell exclude: _mars/lib diff --git a/python/xorbits/_mars/dataframe/datasource/read_parquet.py b/python/xorbits/_mars/dataframe/datasource/read_parquet.py index a030485d9..a7d317095 100644 --- a/python/xorbits/_mars/dataframe/datasource/read_parquet.py +++ b/python/xorbits/_mars/dataframe/datasource/read_parquet.py @@ -391,8 +391,7 @@ def _tile_no_partitioned(cls, op: "DataFrameReadParquet"): paths = sorted(paths) if not isinstance(fs, fsspec.implementations.local.LocalFileSystem): parsed_path = urlparse(op.path) - path_prefix = f"{parsed_path.scheme}://{parsed_path.netloc}" - paths = [path_prefix + path for path in paths] + paths = [f"{parsed_path.scheme}://{path}" for path in paths] elif isinstance(op.path, str) and op.path.endswith(".zip"): file = fs.open(op.path, storage_options=op.storage_options) z = zipfile.ZipFile(file) @@ -406,8 +405,7 @@ def _tile_no_partitioned(cls, op: "DataFrameReadParquet"): paths = fs.glob(op.path, storage_options=op.storage_options) if not isinstance(fs, fsspec.implementations.local.LocalFileSystem): parsed_path = urlparse(op.path) - path_prefix = f"{parsed_path.scheme}://{parsed_path.netloc}" - paths = [path_prefix + path for path in paths] + paths = [f"{parsed_path.scheme}://{path}" for path in paths] first_chunk_row_num, first_chunk_raw_bytes = None, None for i, pth in enumerate(paths): if i == 0: @@ -416,7 +414,7 @@ def _tile_no_partitioned(cls, op: "DataFrameReadParquet"): first_chunk_row_num = get_engine(op.engine).get_row_num(f) first_chunk_raw_bytes = sys.getsizeof(f) else: - of = fsspec.open(pth, storage_options=op.storage_options) + of = fsspec.open(pth) with of as f: first_chunk_row_num = get_engine(op.engine).get_row_num(f) first_chunk_raw_bytes = fsspec.get_fs_token_paths( @@ -857,11 +855,6 @@ def read_parquet( else: if not isinstance(path, list): file_path = fs.glob(path, storage_options=storage_options)[0] - if not isinstance(fs, fsspec.implementations.local.LocalFileSystem): - parsed_path = urlparse(path) - path_prefix = f"{parsed_path.scheme}://{parsed_path.netloc}" - file_path = path_prefix + file_path - else: file_path = path[0] with fs.open(file_path, storage_options=storage_options) as f: diff --git a/python/xorbits/_mars/learn/linear_model/_base.py b/python/xorbits/_mars/learn/linear_model/_base.py index bf1e27b82..0b6b9510e 100644 --- a/python/xorbits/_mars/learn/linear_model/_base.py +++ b/python/xorbits/_mars/learn/linear_model/_base.py @@ -302,7 +302,7 @@ def fit(self, X, y, sample_weight=None): self.coef_.execute() except LinAlgError: # TODO: implement linalg.lstsq first - raise NotImplementedError("Does not support sigular matrix!") + raise NotImplementedError("Does not support singular matrix!") if y.ndim == 1: self.coef_ = mt.ravel(self.coef_) diff --git a/python/xorbits/_mars/learn/linear_model/tests/test_base.py b/python/xorbits/_mars/learn/linear_model/tests/test_base.py index eaf32f0ef..942a5c342 100644 --- a/python/xorbits/_mars/learn/linear_model/tests/test_base.py +++ b/python/xorbits/_mars/learn/linear_model/tests/test_base.py @@ -52,7 +52,7 @@ def test_linear_regression(setup): assert_array_almost_equal(reg.predict(X), model.predict(X)) # Regular model fitting, #samples <= 2, # features < 2 - error_msg = re.escape("Does not support sigular matrix!") + error_msg = re.escape("Does not support singular matrix!") X = [[1], [2]] Y = [1, 2] @@ -68,7 +68,7 @@ def test_linear_regression(setup): assert_array_almost_equal(reg.predict(X), model.predict(X)) # Extra case #1: singular matrix, degenerate input - error_msg = re.escape("Does not support sigular matrix!") + error_msg = re.escape("Does not support singular matrix!") X = [[1]] Y = [0] diff --git a/python/xorbits/_mars/learn/metrics/pairwise/pairwise.py b/python/xorbits/_mars/learn/metrics/pairwise/pairwise.py index 8353cff39..f1e650f70 100644 --- a/python/xorbits/_mars/learn/metrics/pairwise/pairwise.py +++ b/python/xorbits/_mars/learn/metrics/pairwise/pairwise.py @@ -72,7 +72,7 @@ "precomputed": None, # HACK: precomputed is always allowed, never called } -# These distances recquire boolean tensors, when using mars.tensor.spatial.distance +# These distances require boolean tensors, when using mars.tensor.spatial.distance PAIRWISE_BOOLEAN_FUNCTIONS = [ "dice", "jaccard", diff --git a/python/xorbits/_mars/tensor/base/tile.py b/python/xorbits/_mars/tensor/base/tile.py index ee654ee80..326434b0a 100644 --- a/python/xorbits/_mars/tensor/base/tile.py +++ b/python/xorbits/_mars/tensor/base/tile.py @@ -29,7 +29,7 @@ def tile(A, reps): behavior, promote `A` to d-dimensions manually before calling this function. - If ``A.ndim > d``, `reps` is promoted to `A`.ndim by pre-pending 1's to it. + If ``A.ndim > d``, `reps` is promoted to `A`.ndim by prepending 1's to it. Thus for an `A` of shape (2, 3, 4, 5), a `reps` of (2, 2) is treated as (1, 1, 2, 2). diff --git a/python/xorbits/_mars/tensor/utils.py b/python/xorbits/_mars/tensor/utils.py index ba36abb5b..86bcd483d 100644 --- a/python/xorbits/_mars/tensor/utils.py +++ b/python/xorbits/_mars/tensor/utils.py @@ -774,7 +774,7 @@ def fetch_corner_data(tensor, session=None): # the tensor must have been executed, # thus the size could not be NaN if tensor.size > threshold: - # two edges for each exis + # two edges for each axis indices_iter = list(itertools.product(*(range(2) for _ in range(tensor.ndim)))) corners = np.empty(shape=(2,) * tensor.ndim, dtype=object) shape = [0 for _ in range(tensor.ndim)] From 936c82b0fa3a5b4bfa8c4912b97fbb4066b3016b Mon Sep 17 00:00:00 2001 From: codingl2k1 <138426806+codingl2k1@users.noreply.github.com> Date: Mon, 9 Oct 2023 11:46:22 +0800 Subject: [PATCH 20/27] BUG: Fix read_csv with index_col (#736) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- python/xorbits/_mars/dataframe/datasource/read_csv.py | 9 ++++++--- .../datasource/tests/test_datasource_execution.py | 4 ++++ 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/python/xorbits/_mars/dataframe/datasource/read_csv.py b/python/xorbits/_mars/dataframe/datasource/read_csv.py index b0ea4666e..af3913a2b 100644 --- a/python/xorbits/_mars/dataframe/datasource/read_csv.py +++ b/python/xorbits/_mars/dataframe/datasource/read_csv.py @@ -103,6 +103,7 @@ class DataFrameReadCSV( sep = StringField("sep") header = AnyField("header") index_col = Int32Field("index_col") + index_names = ListField("index_names") skiprows = Int32Field("skiprows") compression = StringField("compression") usecols = AnyField("usecols") @@ -266,6 +267,7 @@ def _pandas_read_csv(cls, f, op): nrows=op.nrows, **csv_kwargs, ) + df.index.names = op.index_names if op.keep_usecols_order: df = df[op.usecols] return df @@ -352,7 +354,7 @@ def read_csv( path: str, names: Union[List, Tuple] = None, sep: str = ",", - index_col: int = None, + index_col: Union[int, str, List[int], List[str]] = None, compression: str = None, header: Union[str, List] = "infer", dtype: Union[str, Dict] = None, @@ -709,8 +711,8 @@ def read_csv( else: index_value = parse_index(mini_df.index) columns_value = parse_index(mini_df.columns, store_data=True) - if index_col and not isinstance(index_col, int): - index_col = list(mini_df.columns).index(index_col) + # Set names and index_col may lose multiindex names, so we have to fix it. + index_names = mini_df.index.names # convert path to abs_path abs_path = convert_to_abspath(path, storage_options) @@ -721,6 +723,7 @@ def read_csv( sep=sep, header=header, index_col=index_col, + index_names=index_names, usecols=usecols, skiprows=skiprows, compression=compression, diff --git a/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py b/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py index 8b89220b8..28375d15b 100644 --- a/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py +++ b/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py @@ -606,6 +606,10 @@ def test_read_csv_execution(setup): mdf2 = md.read_csv(file_path, index_col=0, chunk_bytes=100).execute().fetch() pd.testing.assert_frame_equal(pdf, mdf2) + mdf3 = md.read_csv(file_path, index_col=[0, 1]).execute().fetch() + pdf3 = pd.read_csv(file_path, index_col=[0, 1]) + pd.testing.assert_frame_equal(pdf3, mdf3) + # test nan with tempfile.TemporaryDirectory() as tempdir: file_path = os.path.join(tempdir, "test.csv") From 30cac74992e025bfe379923a8c15ae1f1e88429e Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Mon, 9 Oct 2023 04:59:16 -0500 Subject: [PATCH 21/27] FEAT: Support `read_csv` via `http` (#704) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../_mars/dataframe/datasource/core.py | 15 +++- .../_mars/dataframe/datasource/read_csv.py | 85 ++++++++++++++++++- .../tests/test_datasource_execution.py | 46 +++++++++- 3 files changed, 138 insertions(+), 8 deletions(-) diff --git a/python/xorbits/_mars/dataframe/datasource/core.py b/python/xorbits/_mars/dataframe/datasource/core.py index dad9f2049..2d9b0e828 100644 --- a/python/xorbits/_mars/dataframe/datasource/core.py +++ b/python/xorbits/_mars/dataframe/datasource/core.py @@ -134,7 +134,10 @@ def post_tile(cls, op: OperandType, results: List[TileableType]): if ( op.incremental_index and results is not None - and isinstance(results[0].index_value.value, IndexValue.RangeIndex) + and ( + results[0].index_value is None + or isinstance(results[0].index_value.value, IndexValue.RangeIndex) + ) ): result = results[0] chunks = [] @@ -159,7 +162,10 @@ def pre_execute(cls, ctx: Union[dict, Context], op: OperandType): out = op.outputs[0] if ( op.incremental_index - and isinstance(out.index_value.value, IndexValue.RangeIndex) + and ( + out.index_value is None + or isinstance(out.index_value.value, IndexValue.RangeIndex) + ) and getattr(op, "incremental_index_recorder_name", None) ): index = out.index[0] @@ -173,7 +179,10 @@ def post_execute(cls, ctx: Union[dict, Context], op: OperandType): result = ctx[out.key] if ( op.incremental_index - and isinstance(out.index_value.value, IndexValue.RangeIndex) + and ( + out.index_value is None + or isinstance(out.index_value.value, IndexValue.RangeIndex) + ) and getattr(op, "incremental_index_recorder_name", None) ): recorder_name = op.incremental_index_recorder_name diff --git a/python/xorbits/_mars/dataframe/datasource/read_csv.py b/python/xorbits/_mars/dataframe/datasource/read_csv.py index af3913a2b..43a29c47e 100644 --- a/python/xorbits/_mars/dataframe/datasource/read_csv.py +++ b/python/xorbits/_mars/dataframe/datasource/read_csv.py @@ -115,6 +115,7 @@ class DataFrameReadCSV( storage_options = DictField("storage_options") merge_small_files = BoolField("merge_small_files") merge_small_file_options = DictField("merge_small_file_options") + is_http_url = BoolField("is_http_url", None) def get_columns(self): return self.usecols @@ -151,8 +152,32 @@ def _tile_compressed(cls, op): nsplits=nsplits, ) + @classmethod + def _tile_http_url(cls, op: "DataFrameReadCSV"): + out_chunks = [] + out_df = op.outputs[0] + for i, url in enumerate(op.path): + chunk_op = op.copy().reset_key() + chunk_op.path = url + out_chunks.append( + chunk_op.new_chunk(None, index=(i, 0), shape=(np.nan, np.nan)) + ) + new_op = op.copy() + nsplits = ((np.nan,) * len(out_chunks), (np.nan,)) + return new_op.new_dataframes( + None, + out_df.shape, + dtypes=out_df.dtypes, + index_value=out_df.index_value, + columns_value=out_df.columns_value, + chunks=out_chunks, + nsplits=nsplits, + ) + @classmethod def _tile(cls, op: "DataFrameReadCSV"): + if op.is_http_url: + return cls._tile_http_url(op) if op.compression: return cls._tile_compressed(op) @@ -302,8 +327,34 @@ def _cudf_read_csv(cls, op): # pragma: no cover df = df[op.usecols] return df + @classmethod + def _execute_http_url(cls, ctx, op): + xdf = cudf if op.gpu else pd + out_df = op.outputs[0] + csv_kwargs = op.extra_params.copy() + if xdf is pd and op.use_arrow_dtype: + csv_kwargs.update(arrow_dtype_kwargs()) + df = xdf.read_csv( + op.path, + sep=op.sep, + names=op.names, + header=op.header, + index_col=op.index_col, + usecols=op.usecols, + nrows=op.nrows, + compression=op.compression, + **csv_kwargs, + ) + if op.keep_usecols_order: + df = df[op.usecols] + ctx[out_df.key] = df + @classmethod def execute(cls, ctx, op): + if op.is_http_url: + cls._execute_http_url(ctx, op) + return + xdf = cudf if op.gpu else pd out_df = op.outputs[0] csv_kwargs = op.extra_params.copy() @@ -332,6 +383,8 @@ def execute(cls, ctx, op): ctx[out_df.key] = df def estimate_size(cls, ctx, op): + if op.is_http_url: + return super().estimate_size(ctx, op) phy_size = op.size * (op.memory_scale or 1) ctx[op.outputs[0].key] = (phy_size, phy_size * 2) @@ -339,7 +392,10 @@ def __call__( self, index_value=None, columns_value=None, dtypes=None, chunk_bytes=None ): self._output_types = [OutputType.dataframe] - shape = (np.nan, len(dtypes)) + if dtypes is not None: + shape = (np.nan, len(dtypes)) + else: + shape = (np.nan, np.nan) return self.new_dataframe( None, shape, @@ -659,6 +715,33 @@ def read_csv( """ if use_arrow_dtype is None: use_arrow_dtype = options.dataframe.use_arrow_dtype + + single_path = path[0] if isinstance(path, (list, tuple)) else path + if isinstance(single_path, str) and ( + single_path.startswith("http://") or single_path.startswith("https://") + ): + urls = path if isinstance(path, (list, tuple)) else [path] + op = DataFrameReadCSV( + path=urls, + names=names, + sep=sep, + header=header, + index_col=index_col, + usecols=usecols, + skiprows=skiprows, + compression=compression, + gpu=gpu, + incremental_index=incremental_index, + use_arrow_dtype=use_arrow_dtype, + storage_options=storage_options, + memory_scale=memory_scale, + merge_small_files=merge_small_files, + merge_small_file_options=merge_small_file_options, + is_http_url=True, + **kwargs, + ) + return op() + # infer dtypes and columns if isinstance(path, (list, tuple)): file_path = path[0] diff --git a/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py b/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py index 28375d15b..1e7148934 100644 --- a/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py +++ b/python/xorbits/_mars/dataframe/datasource/tests/test_datasource_execution.py @@ -1556,7 +1556,9 @@ def test_read_parquet_fast_parquet(setup): # assert sum(s[0] for s in size_res) > test_df.memory_usage(deep=True).sum() -def _start_tornado(port: int, file_path0: str, file_path1: str, zip_path: str): +def _start_tornado( + port: int, file_path0: str, file_path1: str, csv_path: str, zip_path: str +): import tornado.ioloop import tornado.web @@ -1570,6 +1572,11 @@ def get(self): with open(file_path1, "rb") as f: self.write(f.read()) + class CSVHandler(tornado.web.RequestHandler): + def get(self): + with open(csv_path, "rb") as f: + self.write(f.read()) + class RangeZipFileHandler(tornado.web.RequestHandler): def get(self): file_path = zip_path @@ -1607,6 +1614,7 @@ def parse_range_header(self, range_header): (r"/read-parquet0", Parquet0Handler), (r"/read-parquet1", Parquet1Handler), (r"/test.zip", RangeZipFileHandler), + (r"/read-csv", CSVHandler), ] ) app.listen(port) @@ -1618,6 +1626,7 @@ def start_http_server(): with tempfile.TemporaryDirectory() as tempdir: file_path0 = os.path.join(tempdir, "test0.parquet") file_path1 = os.path.join(tempdir, "test1.parquet") + csv_path = os.path.join(tempdir, "test.csv") df = pd.DataFrame( { @@ -1628,6 +1637,7 @@ def start_http_server(): ) df.iloc[:50].to_parquet(file_path0) df.iloc[50:].to_parquet(file_path1) + df.to_csv(csv_path) import zipfile zip_path = os.path.join(tempdir, "test.zip") @@ -1638,7 +1648,8 @@ def start_http_server(): port = get_next_port() proc = multiprocessing.Process( - target=_start_tornado, args=(port, file_path0, file_path1, zip_path) + target=_start_tornado, + args=(port, file_path0, file_path1, csv_path, zip_path), ) proc.daemon = True proc.start() @@ -1646,13 +1657,13 @@ def start_http_server(): yield df, [ f"http://127.0.0.1:{port}/read-parquet0", f"http://127.0.0.1:{port}/read-parquet1", - ], f"http://127.0.0.1:{port}/test.zip" + ], f"http://127.0.0.1:{port}/test.zip", f"http://127.0.0.1:{port}/read-csv" # Terminate the process proc.terminate() def test_read_parquet_with_http_url(setup, start_http_server): - df, urls, zip_url = start_http_server + df, urls, zip_url, _ = start_http_server if PD_VERSION_GREATER_THAN_2_10: df = df.convert_dtypes(dtype_backend="pyarrow") mdf = md.read_parquet(urls).execute().fetch() @@ -1803,3 +1814,30 @@ def test_read_parquet_ftp(ftp_writable, setup): "ftp://{}:{}@{}:{}/test.zip".format(user, pw, host, port) ) pd.testing.assert_frame_equal(df, mdf_zip.to_pandas()) + + +def test_read_csv_http_url(setup, start_http_server): + df, _, _, csv_url = start_http_server + mdf = md.read_csv(csv_url) + pd.testing.assert_frame_equal(pd.read_csv(csv_url), mdf.execute().fetch()) + + mdf = md.read_csv(csv_url, names=["col1", "col2", "col3"]) + pd.testing.assert_frame_equal( + pd.read_csv(csv_url, names=["col1", "col2", "col3"]), mdf.execute().fetch() + ) + + mdf = md.read_csv(csv_url, header=0) + pd.testing.assert_frame_equal(pd.read_csv(csv_url, header=0), mdf.execute().fetch()) + + mdf = md.read_csv(csv_url, header=None) + pd.testing.assert_frame_equal( + pd.read_csv(csv_url, header=None), mdf.execute().fetch() + ) + + if is_pandas_2(): + df = df.convert_dtypes(dtype_backend="pyarrow") + mdf = md.read_csv(csv_url, use_arrow_dtype=True).execute().fetch() + pd.testing.assert_frame_equal( + pd.read_csv(csv_url, dtype_backend="pyarrow"), mdf + ) + assert isinstance(mdf.dtypes.iloc[1], pd.ArrowDtype) From 7ea43167cc1f5846a46a3a6763dace24534c7610 Mon Sep 17 00:00:00 2001 From: codingl2k1 <138426806+codingl2k1@users.noreply.github.com> Date: Tue, 10 Oct 2023 11:00:10 +0800 Subject: [PATCH 22/27] Fix reduction agg with UDF (#737) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- python/xorbits/_mars/dataframe/reduction/sum.py | 10 +++++++++- .../reduction/tests/test_reduction_execution.py | 10 ++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/python/xorbits/_mars/dataframe/reduction/sum.py b/python/xorbits/_mars/dataframe/reduction/sum.py index 69cd4d534..ff8c81b6c 100644 --- a/python/xorbits/_mars/dataframe/reduction/sum.py +++ b/python/xorbits/_mars/dataframe/reduction/sum.py @@ -46,7 +46,14 @@ def sum_(value): def sum_series( - df, axis=None, skipna=True, level=None, min_count=0, combine_size=None, method=None + df, + axis=None, + skipna=True, + level=None, + min_count=0, + combine_size=None, + method=None, + **kwargs, # kwargs for compatible with numpy reduction ): op = DataFrameSum( axis=axis, @@ -69,6 +76,7 @@ def sum_dataframe( numeric_only=None, combine_size=None, method=None, + **kwargs, # kwargs for compatible with numpy reduction ): op = DataFrameSum( axis=axis, diff --git a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py index de3127bf3..885dd36e6 100644 --- a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py +++ b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py @@ -771,6 +771,16 @@ def test_dataframe_aggregate(setup, check_ref_counts): ] data = pd.DataFrame(np.random.rand(20, 20)) + def realized_volatility(series): + print(series) + return np.sqrt(np.sum(series**2)) + + df = md.DataFrame(data) + result = df.agg(realized_volatility) + pd.testing.assert_series_equal( + result.execute().fetch(), data.agg(realized_volatility) + ) + df = md.DataFrame(data) result = df.agg(all_aggs) pd.testing.assert_frame_equal(result.execute().fetch(), data.agg(all_aggs)) From b1d828f9c08a7906c502104a40565bc01d75b8d4 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Mon, 16 Oct 2023 00:32:32 -0500 Subject: [PATCH 23/27] BUG: `merge` performance issue caused by `DataFrameAutoMergeMixin` (#740) --- python/xorbits/_mars/dataframe/base/cartesian_chunk.py | 3 ++- python/xorbits/_mars/dataframe/base/core.py | 1 + python/xorbits/_mars/dataframe/merge/merge.py | 3 ++- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/python/xorbits/_mars/dataframe/base/cartesian_chunk.py b/python/xorbits/_mars/dataframe/base/cartesian_chunk.py index 30db54e1d..774f6747f 100644 --- a/python/xorbits/_mars/dataframe/base/cartesian_chunk.py +++ b/python/xorbits/_mars/dataframe/base/cartesian_chunk.py @@ -139,9 +139,10 @@ def tile(cls, op: "DataFrameCartesianChunk"): auto_merge_threshold = op.auto_merge_threshold auto_merge_before, auto_merge_after = cls._get_auto_merge_options(op.auto_merge) - yield from cls._merge_before( + merge_before_res = yield from cls._merge_before( op, auto_merge_before, auto_merge_threshold, left, right, logger ) + left, right = merge_before_res[0], merge_before_res[1] if left.ndim == 2 and left.chunk_shape[1] > 1: if has_unknown_shape(left): diff --git a/python/xorbits/_mars/dataframe/base/core.py b/python/xorbits/_mars/dataframe/base/core.py index 5f444c439..57796babc 100644 --- a/python/xorbits/_mars/dataframe/base/core.py +++ b/python/xorbits/_mars/dataframe/base/core.py @@ -128,6 +128,7 @@ def _merge_before( right.shape, len(right.chunks), ) + return [left, right] @classmethod def _merge_after( diff --git a/python/xorbits/_mars/dataframe/merge/merge.py b/python/xorbits/_mars/dataframe/merge/merge.py index 8f1baf5e5..4db6993aa 100644 --- a/python/xorbits/_mars/dataframe/merge/merge.py +++ b/python/xorbits/_mars/dataframe/merge/merge.py @@ -744,9 +744,10 @@ def tile(cls, op: "DataFrameMerge"): auto_merge_threshold = op.auto_merge_threshold auto_merge_before, auto_merge_after = cls._get_auto_merge_options(op.auto_merge) - yield from cls._merge_before( + merge_before_res = yield from cls._merge_before( op, auto_merge_before, auto_merge_threshold, left, right, logger ) + left, right = merge_before_res[0], merge_before_res[1] method = cls._choose_merge_method(op, left, right) if cls._if_apply_bloom_filter(method, op, left, right): From c15148dfafe927176c59e2d0d4567d2cb35c2791 Mon Sep 17 00:00:00 2001 From: Weizheng Lu Date: Mon, 16 Oct 2023 17:05:55 +0800 Subject: [PATCH 24/27] Fix reduction agg with numpy udf (#742) Co-authored-by: Peiyuan Liu <77595899+Hank0626@users.noreply.github.com> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../xorbits/_mars/dataframe/reduction/max.py | 11 +++++++++- .../xorbits/_mars/dataframe/reduction/mean.py | 11 +++++++++- .../xorbits/_mars/dataframe/reduction/min.py | 11 +++++++++- .../tests/test_reduction_execution.py | 21 +++++++++++++++++++ 4 files changed, 51 insertions(+), 3 deletions(-) diff --git a/python/xorbits/_mars/dataframe/reduction/max.py b/python/xorbits/_mars/dataframe/reduction/max.py index 0df433075..5e8d3dd34 100644 --- a/python/xorbits/_mars/dataframe/reduction/max.py +++ b/python/xorbits/_mars/dataframe/reduction/max.py @@ -27,7 +27,15 @@ def is_atomic(self): return True -def max_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): +def max_series( + df, + axis=None, + skipna=True, + level=None, + combine_size=None, + method=None, + **kwargs, # kwargs for compatible with numpy reduction +): op = DataFrameMax( axis=axis, skipna=skipna, @@ -47,6 +55,7 @@ def max_dataframe( numeric_only=None, combine_size=None, method=None, + **kwargs, # kwargs for compatible with numpy reduction ): op = DataFrameMax( axis=axis, diff --git a/python/xorbits/_mars/dataframe/reduction/mean.py b/python/xorbits/_mars/dataframe/reduction/mean.py index c04f21095..4d22d608a 100644 --- a/python/xorbits/_mars/dataframe/reduction/mean.py +++ b/python/xorbits/_mars/dataframe/reduction/mean.py @@ -32,7 +32,15 @@ def mean(x): return mean -def mean_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): +def mean_series( + df, + axis=None, + skipna=True, + level=None, + combine_size=None, + method=None, + **kwargs, # kwargs for compatible with numpy reduction +): op = DataFrameMean( axis=axis, skipna=skipna, @@ -52,6 +60,7 @@ def mean_dataframe( numeric_only=None, combine_size=None, method=None, + **kwargs, # kwargs for compatible with numpy reduction ): op = DataFrameMean( axis=axis, diff --git a/python/xorbits/_mars/dataframe/reduction/min.py b/python/xorbits/_mars/dataframe/reduction/min.py index 90b253330..0bcb91634 100644 --- a/python/xorbits/_mars/dataframe/reduction/min.py +++ b/python/xorbits/_mars/dataframe/reduction/min.py @@ -27,7 +27,15 @@ def is_atomic(self): return True -def min_series(df, axis=None, skipna=True, level=None, combine_size=None, method=None): +def min_series( + df, + axis=None, + skipna=True, + level=None, + combine_size=None, + method=None, + **kwargs, # kwargs for compatible with numpy reduction +): op = DataFrameMin( axis=axis, skipna=skipna, @@ -47,6 +55,7 @@ def min_dataframe( numeric_only=None, combine_size=None, method=None, + **kwargs, # kwargs for compatible with numpy reduction ): op = DataFrameMin( axis=axis, diff --git a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py index 885dd36e6..53b121805 100644 --- a/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py +++ b/python/xorbits/_mars/dataframe/reduction/tests/test_reduction_execution.py @@ -781,6 +781,27 @@ def realized_volatility(series): result.execute().fetch(), data.agg(realized_volatility) ) + def trip_type(x): + return np.min(x) + + df = md.DataFrame(data) + result = df.agg(trip_type) + pd.testing.assert_series_equal(result.execute().fetch(), data.agg(trip_type)) + + def trip_type_max(x): + return np.max(x) + + df = md.DataFrame(data) + result = df.agg(trip_type_max) + pd.testing.assert_series_equal(result.execute().fetch(), data.agg(trip_type_max)) + + def trip_type_mean(x): + return np.mean(x) + + df = md.DataFrame(data) + result = df.agg(trip_type_mean) + pd.testing.assert_series_equal(result.execute().fetch(), data.agg(trip_type_mean)) + df = md.DataFrame(data) result = df.agg(all_aggs) pd.testing.assert_frame_equal(result.execute().fetch(), data.agg(all_aggs)) From 251d1caefa7dec98340ea50d0cf13c30726aaabb Mon Sep 17 00:00:00 2001 From: leojia <59967634+JiaYaobo@users.noreply.github.com> Date: Tue, 17 Oct 2023 19:16:29 +0800 Subject: [PATCH 25/27] FEAT: Add `xorbits.sklearn` module (#716) --- python/xorbits/__init__.py | 2 + python/xorbits/sklearn/__init__.py | 31 ++++ python/xorbits/sklearn/cluster/__init__.py | 49 ++++++ .../sklearn/cluster/mars_adapters/__init__.py | 14 ++ .../sklearn/cluster/mars_adapters/core.py | 35 +++++ .../xorbits/sklearn/cluster/tests/__init__.py | 13 ++ .../sklearn/cluster/tests/test_core.py | 57 +++++++ python/xorbits/sklearn/datasets/__init__.py | 48 ++++++ .../datasets/mars_adapters/__init__.py | 14 ++ .../sklearn/datasets/mars_adapters/core.py | 22 +++ .../sklearn/datasets/tests/__init__.py | 13 ++ .../sklearn/datasets/tests/test_core.py | 131 ++++++++++++++++ .../xorbits/sklearn/decomposition/__init__.py | 49 ++++++ .../decomposition/mars_adapters/__init__.py | 14 ++ .../decomposition/mars_adapters/core.py | 43 ++++++ .../sklearn/decomposition/tests/__init__.py | 13 ++ .../sklearn/decomposition/tests/test_core.py | 87 +++++++++++ python/xorbits/sklearn/ensemble/__init__.py | 49 ++++++ .../ensemble/mars_adapters/__init__.py | 14 ++ .../sklearn/ensemble/mars_adapters/core.py | 49 ++++++ .../sklearn/ensemble/tests/__init__.py | 13 ++ .../sklearn/ensemble/tests/test_core.py | 124 +++++++++++++++ .../xorbits/sklearn/linear_model/__init__.py | 49 ++++++ .../linear_model/mars_adapters/__init__.py | 14 ++ .../linear_model/mars_adapters/core.py | 42 ++++++ .../sklearn/linear_model/tests/__init__.py | 13 ++ .../sklearn/linear_model/tests/test_core.py | 73 +++++++++ python/xorbits/sklearn/metrics/__init__.py | 49 ++++++ .../sklearn/metrics/mars_adapters/__init__.py | 14 ++ .../sklearn/metrics/mars_adapters/core.py | 22 +++ .../xorbits/sklearn/metrics/tests/__init__.py | 13 ++ .../sklearn/metrics/tests/test_core.py | 142 ++++++++++++++++++ .../sklearn/model_selection/__init__.py | 49 ++++++ .../model_selection/mars_adapters/__init__.py | 14 ++ .../model_selection/mars_adapters/core.py | 51 +++++++ .../sklearn/model_selection/tests/__init__.py | 13 ++ .../model_selection/tests/test_core.py | 66 ++++++++ python/xorbits/sklearn/neighbors/__init__.py | 49 ++++++ .../neighbors/mars_adapters/__init__.py | 14 ++ .../sklearn/neighbors/mars_adapters/core.py | 39 +++++ .../sklearn/neighbors/tests/__init__.py | 13 ++ .../sklearn/neighbors/tests/test_core.py | 34 +++++ .../xorbits/sklearn/preprocessing/__init__.py | 49 ++++++ .../preprocessing/mars_adapters/__init__.py | 14 ++ .../preprocessing/mars_adapters/core.py | 51 +++++++ .../sklearn/preprocessing/tests/__init__.py | 13 ++ .../sklearn/preprocessing/tests/test_core.py | 82 ++++++++++ .../sklearn/semi_supervised/__init__.py | 49 ++++++ .../semi_supervised/mars_adapters/__init__.py | 14 ++ .../semi_supervised/mars_adapters/core.py | 35 +++++ .../sklearn/semi_supervised/tests/__init__.py | 13 ++ .../semi_supervised/tests/test_core.py | 48 ++++++ python/xorbits/sklearn/utils.py | 72 +++++++++ 53 files changed, 2097 insertions(+) create mode 100644 python/xorbits/sklearn/__init__.py create mode 100644 python/xorbits/sklearn/cluster/__init__.py create mode 100644 python/xorbits/sklearn/cluster/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/cluster/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/cluster/tests/__init__.py create mode 100644 python/xorbits/sklearn/cluster/tests/test_core.py create mode 100644 python/xorbits/sklearn/datasets/__init__.py create mode 100644 python/xorbits/sklearn/datasets/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/datasets/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/datasets/tests/__init__.py create mode 100644 python/xorbits/sklearn/datasets/tests/test_core.py create mode 100644 python/xorbits/sklearn/decomposition/__init__.py create mode 100644 python/xorbits/sklearn/decomposition/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/decomposition/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/decomposition/tests/__init__.py create mode 100644 python/xorbits/sklearn/decomposition/tests/test_core.py create mode 100644 python/xorbits/sklearn/ensemble/__init__.py create mode 100644 python/xorbits/sklearn/ensemble/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/ensemble/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/ensemble/tests/__init__.py create mode 100644 python/xorbits/sklearn/ensemble/tests/test_core.py create mode 100644 python/xorbits/sklearn/linear_model/__init__.py create mode 100644 python/xorbits/sklearn/linear_model/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/linear_model/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/linear_model/tests/__init__.py create mode 100644 python/xorbits/sklearn/linear_model/tests/test_core.py create mode 100644 python/xorbits/sklearn/metrics/__init__.py create mode 100644 python/xorbits/sklearn/metrics/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/metrics/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/metrics/tests/__init__.py create mode 100644 python/xorbits/sklearn/metrics/tests/test_core.py create mode 100644 python/xorbits/sklearn/model_selection/__init__.py create mode 100644 python/xorbits/sklearn/model_selection/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/model_selection/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/model_selection/tests/__init__.py create mode 100644 python/xorbits/sklearn/model_selection/tests/test_core.py create mode 100644 python/xorbits/sklearn/neighbors/__init__.py create mode 100644 python/xorbits/sklearn/neighbors/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/neighbors/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/neighbors/tests/__init__.py create mode 100644 python/xorbits/sklearn/neighbors/tests/test_core.py create mode 100644 python/xorbits/sklearn/preprocessing/__init__.py create mode 100644 python/xorbits/sklearn/preprocessing/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/preprocessing/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/preprocessing/tests/__init__.py create mode 100644 python/xorbits/sklearn/preprocessing/tests/test_core.py create mode 100644 python/xorbits/sklearn/semi_supervised/__init__.py create mode 100644 python/xorbits/sklearn/semi_supervised/mars_adapters/__init__.py create mode 100644 python/xorbits/sklearn/semi_supervised/mars_adapters/core.py create mode 100644 python/xorbits/sklearn/semi_supervised/tests/__init__.py create mode 100644 python/xorbits/sklearn/semi_supervised/tests/test_core.py create mode 100644 python/xorbits/sklearn/utils.py diff --git a/python/xorbits/__init__.py b/python/xorbits/__init__.py index ff1566ba1..49df2c017 100644 --- a/python/xorbits/__init__.py +++ b/python/xorbits/__init__.py @@ -24,6 +24,7 @@ def _install(): from .lightgbm import _install as _install_lightgbm from .numpy import _install as _install_numpy from .pandas import _install as _install_pandas + from .sklearn import _install as _install_sklearn from .web import _install as _install_web from .xgboost import _install as _install_xgboost @@ -34,6 +35,7 @@ def _install(): _install_xgboost() _install_datasets() _install_experimental() + _install_sklearn() _install() diff --git a/python/xorbits/sklearn/__init__.py b/python/xorbits/sklearn/__init__.py new file mode 100644 index 000000000..9c9f2b596 --- /dev/null +++ b/python/xorbits/sklearn/__init__.py @@ -0,0 +1,31 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +def _install(): + """Nothing required for installing sklearn.""" + + +__all__ = [ + "cluster", + "datasets", + "decomposition", + "ensemble", + "linear_model", + "metrics", + "model_selection", + "neighbors", + "preprocessing", + "semi_supervised", +] diff --git a/python/xorbits/sklearn/cluster/__init__.py b/python/xorbits/sklearn/cluster/__init__.py new file mode 100644 index 000000000..da13e6baa --- /dev/null +++ b/python/xorbits/sklearn/cluster/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_CLUSTER_CALLABLES + + return list(MARS_SKLEARN_CLUSTER_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.cluster as sk_cluster + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_CLUSTER_CALLABLES + + if name in MARS_SKLEARN_CLUSTER_CALLABLES: + return MARS_SKLEARN_CLUSTER_CALLABLES[name] + else: + if not hasattr(sk_cluster, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_cluster, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/cluster/mars_adapters/__init__.py b/python/xorbits/sklearn/cluster/mars_adapters/__init__.py new file mode 100644 index 000000000..e9aabb1cc --- /dev/null +++ b/python/xorbits/sklearn/cluster/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_CLUSTER_CALLABLES diff --git a/python/xorbits/sklearn/cluster/mars_adapters/core.py b/python/xorbits/sklearn/cluster/mars_adapters/core.py new file mode 100644 index 000000000..cdd3f302e --- /dev/null +++ b/python/xorbits/sklearn/cluster/mars_adapters/core.py @@ -0,0 +1,35 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.cluster as sk_cluster + +from ...._mars.learn import cluster as mars_cluster +from ...._mars.learn.cluster import KMeans as MarsKMeans +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class KMeans(SKLearnBase): + _marscls = MarsKMeans + + +SKLEARN_CLUSTER_CLS_MAP = {KMeans: MarsKMeans} + +MARS_SKLEARN_CLUSTER_CALLABLES = _collect_module_callables( + mars_cluster, sk_cluster, skip_members=["register_op"] +) +_install_cls_members( + SKLEARN_CLUSTER_CLS_MAP, MARS_SKLEARN_CLUSTER_CALLABLES, sk_cluster +) +attach_module_callable_docstring(KMeans, sk_cluster, sk_cluster.KMeans) diff --git a/python/xorbits/sklearn/cluster/tests/__init__.py b/python/xorbits/sklearn/cluster/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/cluster/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/cluster/tests/test_core.py b/python/xorbits/sklearn/cluster/tests/test_core.py new file mode 100644 index 000000000..c2a10ec61 --- /dev/null +++ b/python/xorbits/sklearn/cluster/tests/test_core.py @@ -0,0 +1,57 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest + +from .... import numpy as xnp +from .. import KMeans + +n_rows = 1000 +n_clusters = 8 +n_columns = 10 +chunk_size = 200 +rs = xnp.random.RandomState(0) +X = rs.rand(n_rows, n_columns, chunk_size=chunk_size) +X_new = rs.rand(n_rows, n_columns, chunk_size=chunk_size) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = KMeans.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.cluster." + ) + + docstring = KMeans.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.cluster._kmeans.KMeans." + ) + + +@pytest.mark.skipif(sklearn is None, reason="sci-kit-learn not installed") +def test_kmeans_cluster(): + kms = KMeans(n_clusters=n_clusters, random_state=0) + kms.fit(X) + predict = kms.predict(X_new).fetch() + + assert kms.n_clusters == n_clusters + assert np.shape(kms.labels_.fetch()) == (n_rows,) + assert np.shape(kms.cluster_centers_.fetch()) == (n_clusters, n_columns) + assert np.shape(predict) == (n_rows,) diff --git a/python/xorbits/sklearn/datasets/__init__.py b/python/xorbits/sklearn/datasets/__init__.py new file mode 100644 index 000000000..accf8fbcb --- /dev/null +++ b/python/xorbits/sklearn/datasets/__init__.py @@ -0,0 +1,48 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_DATASETS_CALLABLES + + return list(MARS_SKLEARN_DATASETS_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.datasets as sk_datasets + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_DATASETS_CALLABLES + + if name in MARS_SKLEARN_DATASETS_CALLABLES: + return MARS_SKLEARN_DATASETS_CALLABLES[name] + else: + if not hasattr(sk_datasets, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_datasets, name)): + raise NotImplementedError(f"This function is not implemented yet.") + else: + raise AttributeError diff --git a/python/xorbits/sklearn/datasets/mars_adapters/__init__.py b/python/xorbits/sklearn/datasets/mars_adapters/__init__.py new file mode 100644 index 000000000..050a5f86b --- /dev/null +++ b/python/xorbits/sklearn/datasets/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_DATASETS_CALLABLES diff --git a/python/xorbits/sklearn/datasets/mars_adapters/core.py b/python/xorbits/sklearn/datasets/mars_adapters/core.py new file mode 100644 index 000000000..a312031b5 --- /dev/null +++ b/python/xorbits/sklearn/datasets/mars_adapters/core.py @@ -0,0 +1,22 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.datasets as sk_datasets + +from ...._mars.learn import datasets as mars_datasets +from ...utils import _collect_module_callables + +MARS_SKLEARN_DATASETS_CALLABLES = _collect_module_callables( + mars_datasets, sk_datasets, skip_members=["register_op"] +) diff --git a/python/xorbits/sklearn/datasets/tests/__init__.py b/python/xorbits/sklearn/datasets/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/datasets/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/datasets/tests/test_core.py b/python/xorbits/sklearn/datasets/tests/test_core.py new file mode 100644 index 000000000..a7a06f71b --- /dev/null +++ b/python/xorbits/sklearn/datasets/tests/test_core.py @@ -0,0 +1,131 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import pytest + +import xorbits.numpy as np + +from ... import datasets +from ...datasets import ( + make_blobs, + make_classification, + make_low_rank_matrix, + make_regression, +) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = datasets.make_blobs.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.datasets." + ) + + docstring = datasets.make_classification.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.datasets." + ) + + docstring = datasets.make_low_rank_matrix.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.datasets." + ) + + docstring = datasets.make_regression.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.datasets." + ) + + +def test_make_classification(): + weights = [0.1, 0.25] + X, y = make_classification( + n_samples=100, + n_features=20, + n_informative=5, + n_redundant=1, + n_repeated=1, + n_classes=3, + n_clusters_per_class=1, + hypercube=False, + shift=None, + scale=None, + weights=weights, + random_state=0, + flip_y=-1, + ) + X, y = X.execute().fetch(), y.execute().fetch() + assert X.shape == (100, 20) + assert y.shape == (100,) + assert np.unique(y).shape == (3,) + assert (y == 0).sum() == 10 + assert (y == 1).sum() == 25 + assert (y == 2).sum() == 65 + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_make_regression(): + X, y, c = make_regression( + n_samples=100, + n_features=10, + n_informative=3, + effective_rank=5, + coef=True, + bias=0.0, + noise=1.0, + random_state=0, + ) + X, y, c = X.execute().fetch(), y.execute().fetch(), c.execute().fetch() + assert X.shape == (100, 10), "X shape mismatch" + assert y.shape == (100,), "y shape mismatch" + assert c.shape == (10,), "coef shape mismatch" + assert sum(c != 0.0) == 3, "Unexpected number of informative features" + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_make_blobs(): + cluster_stds = np.array([0.05, 0.2, 0.4]) + cluster_centers = np.array([[0.0, 0.0], [1.0, 1.0], [0.0, 1.0]]) + X, y = make_blobs( + random_state=0, + n_samples=50, + n_features=2, + centers=cluster_centers, + cluster_std=cluster_stds, + ) + X, y = X.execute().fetch(), y.execute().fetch() + assert X.shape == (50, 2) + assert y.shape == (50,) + assert np.unique(y).shape == (3,) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_make_low_rank_matrix(): + X = make_low_rank_matrix( + n_samples=50, + n_features=25, + effective_rank=5, + tail_strength=0.01, + random_state=0, + ) + X = X.execute().fetch() + assert X.shape == (50, 25) + _, s, _ = np.linalg.svd(X) + s = s.execute().fetch() + assert (s.sum() - 5) < 0.1 diff --git a/python/xorbits/sklearn/decomposition/__init__.py b/python/xorbits/sklearn/decomposition/__init__.py new file mode 100644 index 000000000..a54d6b392 --- /dev/null +++ b/python/xorbits/sklearn/decomposition/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_DECOMP_CALLABLES + + return list(MARS_SKLEARN_DECOMP_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.decomposition as sk_decomp + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_DECOMP_CALLABLES + + if name in MARS_SKLEARN_DECOMP_CALLABLES: + return MARS_SKLEARN_DECOMP_CALLABLES[name] + else: + if not hasattr(sk_decomp, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_decomp, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/decomposition/mars_adapters/__init__.py b/python/xorbits/sklearn/decomposition/mars_adapters/__init__.py new file mode 100644 index 000000000..8c9727dc3 --- /dev/null +++ b/python/xorbits/sklearn/decomposition/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_DECOMP_CALLABLES diff --git a/python/xorbits/sklearn/decomposition/mars_adapters/core.py b/python/xorbits/sklearn/decomposition/mars_adapters/core.py new file mode 100644 index 000000000..49f3242ed --- /dev/null +++ b/python/xorbits/sklearn/decomposition/mars_adapters/core.py @@ -0,0 +1,43 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.decomposition as sk_decomposition + +from ...._mars.learn import decomposition as mars_decomposition +from ...._mars.learn.decomposition import PCA as MarsPCA +from ...._mars.learn.decomposition import TruncatedSVD as MarsTruncatedSVD +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class PCA(SKLearnBase): + _marscls = MarsPCA + + +class TruncatedSVD(SKLearnBase): + _marscls = MarsTruncatedSVD + + +SKLEARN_DECOMP_CLS_MAP = {PCA: MarsPCA, TruncatedSVD: MarsTruncatedSVD} + +MARS_SKLEARN_DECOMP_CALLABLES = _collect_module_callables( + mars_decomposition, sk_decomposition, skip_members=["register_op"] +) +_install_cls_members( + SKLEARN_DECOMP_CLS_MAP, MARS_SKLEARN_DECOMP_CALLABLES, sk_decomposition +) +attach_module_callable_docstring(PCA, sk_decomposition, sk_decomposition.PCA) +attach_module_callable_docstring( + TruncatedSVD, sk_decomposition, sk_decomposition.TruncatedSVD +) diff --git a/python/xorbits/sklearn/decomposition/tests/__init__.py b/python/xorbits/sklearn/decomposition/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/decomposition/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/decomposition/tests/test_core.py b/python/xorbits/sklearn/decomposition/tests/test_core.py new file mode 100644 index 000000000..361b7cd3d --- /dev/null +++ b/python/xorbits/sklearn/decomposition/tests/test_core.py @@ -0,0 +1,87 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest +import scipy.sparse as sp +from numpy.testing import assert_array_almost_equal, assert_equal +from sklearn import datasets +from sklearn.utils import check_random_state + +from .. import PCA, TruncatedSVD + +iris = np.asarray(datasets.load_iris().data) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = PCA.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.decomposition." + ) + + docstring = PCA.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.decomposition._pca.PCA." + ) + + docstring = TruncatedSVD.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.decomposition." + ) + + docstring = TruncatedSVD.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.decomposition._truncated_svd.TruncatedSVD." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_pca(): + X = iris + + for n_comp in np.arange(X.shape[1]): + pca = PCA(n_components=n_comp, svd_solver="full") + pca.fit(X) + X_r = pca.transform(X).fetch() + assert_equal(X_r.shape[1], n_comp) + + X_r2 = pca.fit_transform(X).fetch() + assert_array_almost_equal(X_r, X_r2) + + X_r = pca.transform(X).fetch() + X_r2 = pca.fit_transform(X).fetch() + assert_array_almost_equal(X_r, X_r2) + + # Test get_covariance and get_precision + cov = pca.get_covariance() + precision = pca.get_precision() + assert_array_almost_equal(np.dot(cov, precision), np.eye(X.shape[1]), 12) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_truncated_svd(): + shape = 60, 55 + n_samples, n_features = shape + rng = check_random_state(42) + X = rng.randint(-100, 20, np.product(shape)).reshape(shape) + X = sp.csr_matrix(np.maximum(X, 0), dtype=np.float64) + for n_components in (10, 25, 41): + tsvd = TruncatedSVD(n_components).fit(X) + assert tsvd.n_components == n_components + assert tsvd.components_.shape == (n_components, n_features) diff --git a/python/xorbits/sklearn/ensemble/__init__.py b/python/xorbits/sklearn/ensemble/__init__.py new file mode 100644 index 000000000..92ad66397 --- /dev/null +++ b/python/xorbits/sklearn/ensemble/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_EN_CALLABLES + + return list(MARS_SKLEARN_EN_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.ensemble as sk_en + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_EN_CALLABLES + + if name in MARS_SKLEARN_EN_CALLABLES: + return MARS_SKLEARN_EN_CALLABLES[name] + else: + if not hasattr(sk_en, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_en, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/ensemble/mars_adapters/__init__.py b/python/xorbits/sklearn/ensemble/mars_adapters/__init__.py new file mode 100644 index 000000000..8b02cbb39 --- /dev/null +++ b/python/xorbits/sklearn/ensemble/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_EN_CALLABLES diff --git a/python/xorbits/sklearn/ensemble/mars_adapters/core.py b/python/xorbits/sklearn/ensemble/mars_adapters/core.py new file mode 100644 index 000000000..d7025e5a3 --- /dev/null +++ b/python/xorbits/sklearn/ensemble/mars_adapters/core.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.ensemble as sk_en + +from ...._mars.learn import ensemble as mars_en +from ...._mars.learn.ensemble import BaggingClassifier as MarsBaggingClassifier +from ...._mars.learn.ensemble import BaggingRegressor as MarsBaggingRegressor +from ...._mars.learn.ensemble import IsolationForest as MarsIsolationForest +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class BaggingClassifier(SKLearnBase): + _marscls = MarsBaggingClassifier + + +class BaggingRegressor(SKLearnBase): + _marscls = MarsBaggingRegressor + + +class IsolationForest(SKLearnBase): + _marscls = MarsIsolationForest + + +SKLEARN_EN_CLS_MAP = { + BaggingClassifier: MarsBaggingClassifier, + IsolationForest: MarsIsolationForest, + BaggingRegressor: MarsBaggingRegressor, +} + +MARS_SKLEARN_EN_CALLABLES = _collect_module_callables( + mars_en, sk_en, skip_members=["register_op"] +) +_install_cls_members(SKLEARN_EN_CLS_MAP, MARS_SKLEARN_EN_CALLABLES, sk_en) +attach_module_callable_docstring(BaggingClassifier, sk_en, sk_en.BaggingClassifier) +attach_module_callable_docstring(BaggingRegressor, sk_en, sk_en.BaggingRegressor) +attach_module_callable_docstring(IsolationForest, sk_en, sk_en.IsolationForest) diff --git a/python/xorbits/sklearn/ensemble/tests/__init__.py b/python/xorbits/sklearn/ensemble/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/ensemble/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/ensemble/tests/test_core.py b/python/xorbits/sklearn/ensemble/tests/test_core.py new file mode 100644 index 000000000..07cdfc537 --- /dev/null +++ b/python/xorbits/sklearn/ensemble/tests/test_core.py @@ -0,0 +1,124 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest +from sklearn.linear_model import LinearRegression +from sklearn.svm import SVC + +from ...datasets import make_classification, make_regression +from ...ensemble import BaggingClassifier, BaggingRegressor, IsolationForest + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = BaggingClassifier.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble." + ) + + docstring = BaggingRegressor.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble." + ) + + docstring = IsolationForest.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble." + ) + + docstring = BaggingClassifier.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble._bagging.BaggingClassifier." + ) + + docstring = BaggingRegressor.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble._bagging.BaggingRegressor." + ) + + docstring = IsolationForest.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.ensemble._iforest.IsolationForest." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_baggingclassifier(): + rs = np.random.RandomState(0) + + raw_x, raw_y = make_classification( + n_samples=100, + n_features=4, + n_informative=2, + n_redundant=0, + random_state=rs, + shuffle=False, + ) + + clf = BaggingClassifier( + base_estimator=SVC(), + n_estimators=10, + max_samples=10, + max_features=1, + random_state=rs, + warm_start=True, + ) + + clf.fit(raw_x, raw_y) + log_proba = clf.predict_log_proba(raw_x) + log_proba = log_proba.fetch() + exp_log_proba_array = np.exp(log_proba) + assert clf.n_estimators == 10 + assert np.all((exp_log_proba_array >= 0) & (exp_log_proba_array <= 1)) + assert np.allclose(np.sum(exp_log_proba_array, axis=1), 1.0) + + +def test_bagging_regression(): + rs = np.random.RandomState(0) + + raw_x, raw_y = make_regression( + n_samples=100, n_features=4, n_informative=2, random_state=rs, shuffle=False + ) + clf = BaggingRegressor( + base_estimator=LinearRegression(), + n_estimators=10, + max_samples=10, + max_features=0.5, + random_state=rs, + warm_start=True, + ) + clf.fit(raw_x, raw_y) + + predict_y = clf.predict(raw_x) + predict_y_array = predict_y.fetch() + assert predict_y_array.shape == raw_y.shape + + +def test_iforest(): + rs = np.random.RandomState(0) + raw_train = rs.poisson(size=(100, 10)) + raw_test = rs.poisson(size=(200, 10)) + + clf = IsolationForest(random_state=rs, n_estimators=10, max_samples=1) + pred = clf.fit(raw_train).predict(raw_test).fetch() + score = clf.score_samples(raw_test).fetch() + + assert clf.n_estimators == 10 + assert pred.shape == (200,) + assert score.shape == (200,) diff --git a/python/xorbits/sklearn/linear_model/__init__.py b/python/xorbits/sklearn/linear_model/__init__.py new file mode 100644 index 000000000..1011a91b5 --- /dev/null +++ b/python/xorbits/sklearn/linear_model/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_LM_CALLABLES + + return list(MARS_SKLEARN_LM_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.linear_model as sk_lm + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_LM_CALLABLES + + if name in MARS_SKLEARN_LM_CALLABLES: + return MARS_SKLEARN_LM_CALLABLES[name] + else: + if not hasattr(sk_lm, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_lm, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/linear_model/mars_adapters/__init__.py b/python/xorbits/sklearn/linear_model/mars_adapters/__init__.py new file mode 100644 index 000000000..dc3dfcca7 --- /dev/null +++ b/python/xorbits/sklearn/linear_model/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_LM_CALLABLES diff --git a/python/xorbits/sklearn/linear_model/mars_adapters/core.py b/python/xorbits/sklearn/linear_model/mars_adapters/core.py new file mode 100644 index 000000000..7be6ff8e3 --- /dev/null +++ b/python/xorbits/sklearn/linear_model/mars_adapters/core.py @@ -0,0 +1,42 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.linear_model as sk_lm + +from ...._mars.learn import linear_model as mars_lm +from ...._mars.learn.glm import LogisticRegression as MarsLogisticRegression +from ...._mars.learn.linear_model import LinearRegression as MarsLinearRegression +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class LinearRegression(SKLearnBase): + _marscls = MarsLinearRegression + + +class LogisticRegression(SKLearnBase): + _marscls = MarsLogisticRegression + + +SKLEARN_LM_CLS_MAP = { + LinearRegression: MarsLinearRegression, + LogisticRegression: MarsLogisticRegression, +} + +MARS_SKLEARN_LM_CALLABLES = _collect_module_callables( + mars_lm, sk_lm, skip_members=["register_op"] +) +_install_cls_members(SKLEARN_LM_CLS_MAP, MARS_SKLEARN_LM_CALLABLES, sk_lm) +attach_module_callable_docstring(LinearRegression, sk_lm, sk_lm.LinearRegression) +attach_module_callable_docstring(LogisticRegression, sk_lm, sk_lm.LogisticRegression) diff --git a/python/xorbits/sklearn/linear_model/tests/__init__.py b/python/xorbits/sklearn/linear_model/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/linear_model/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/linear_model/tests/test_core.py b/python/xorbits/sklearn/linear_model/tests/test_core.py new file mode 100644 index 000000000..4e002dc89 --- /dev/null +++ b/python/xorbits/sklearn/linear_model/tests/test_core.py @@ -0,0 +1,73 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest + +from .. import LinearRegression, LogisticRegression + +n_rows = 100 +n_columns = 5 +X = np.random.rand(n_rows, n_columns) +y = np.random.rand(n_rows) +y_cat = np.random.randint(0, 2, n_rows) +X_new = np.random.rand(n_rows, n_columns) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = LogisticRegression.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.linear_model." + ) + + docstring = LogisticRegression.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.linear_model._logistic.LogisticRegression." + ) + + docstring = LinearRegression.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.linear_model." + ) + + docstring = LinearRegression.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.linear_model._base.LinearRegression." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_linear_regression(): + lr = LinearRegression() + lr.fit(X, y) + predict = lr.predict(X_new) + + assert np.shape(lr.coef_.fetch()) == (n_columns,) + assert np.shape(lr.intercept_.fetch()) == () + assert np.shape(predict) == (n_rows,) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_logistic_regression(): + lr = LogisticRegression(max_iter=1) + lr.fit(X, y_cat) + predict = lr.predict(X_new).fetch() + + assert np.shape(predict) == (n_rows,) diff --git a/python/xorbits/sklearn/metrics/__init__.py b/python/xorbits/sklearn/metrics/__init__.py new file mode 100644 index 000000000..c0365ddd5 --- /dev/null +++ b/python/xorbits/sklearn/metrics/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_METRICS_CALLABLES + + return list(MARS_SKLEARN_METRICS_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.metrics as sk_metrics + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_METRICS_CALLABLES + + if name in MARS_SKLEARN_METRICS_CALLABLES: + return MARS_SKLEARN_METRICS_CALLABLES[name] + else: + if not hasattr(sk_metrics, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_metrics, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/metrics/mars_adapters/__init__.py b/python/xorbits/sklearn/metrics/mars_adapters/__init__.py new file mode 100644 index 000000000..d1e23cf5e --- /dev/null +++ b/python/xorbits/sklearn/metrics/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_METRICS_CALLABLES diff --git a/python/xorbits/sklearn/metrics/mars_adapters/core.py b/python/xorbits/sklearn/metrics/mars_adapters/core.py new file mode 100644 index 000000000..b0fa6a862 --- /dev/null +++ b/python/xorbits/sklearn/metrics/mars_adapters/core.py @@ -0,0 +1,22 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.metrics as sk_metrics + +from ...._mars.learn import metrics as mars_metrics +from ...utils import _collect_module_callables + +MARS_SKLEARN_METRICS_CALLABLES = _collect_module_callables( + mars_metrics, sk_metrics, skip_members=["register_op"] +) diff --git a/python/xorbits/sklearn/metrics/tests/__init__.py b/python/xorbits/sklearn/metrics/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/metrics/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/metrics/tests/test_core.py b/python/xorbits/sklearn/metrics/tests/test_core.py new file mode 100644 index 000000000..347b05aee --- /dev/null +++ b/python/xorbits/sklearn/metrics/tests/test_core.py @@ -0,0 +1,142 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import inspect + +import numpy as np +import pytest + +from ... import metrics + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + for name, f in inspect.getmembers(metrics, inspect.isfunction): + if name.startswith("_"): + continue + docstring = f.__doc__ + assert docstring is not None + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_classification(): + from sklearn.metrics import f1_score as sklearn_f1_score + from sklearn.metrics import fbeta_score as sklearn_fbeta_score + from sklearn.metrics import ( + multilabel_confusion_matrix as sklearn_multilabel_confusion_matrix, + ) + from sklearn.metrics import ( + precision_recall_fscore_support as sklearn_precision_recall_fscore_support, + ) + from sklearn.metrics import precision_score as sklearn_precision_score + from sklearn.metrics import recall_score as sklearn_recall_score + + from ...metrics import ( + f1_score, + fbeta_score, + multilabel_confusion_matrix, + precision_recall_fscore_support, + precision_score, + recall_score, + ) + + y_true = np.array([0, 1, 2, 0, 1, 2], dtype=np.int64) + y_pred = np.array([0, 2, 1, 0, 0, 1], dtype=np.int64) + + np.testing.assert_array_almost_equal( + f1_score(y_true, y_pred, average="macro").execute().fetch(), + sklearn_f1_score(y_true, y_pred, average="macro"), + ) + np.testing.assert_array_almost_equal( + fbeta_score(y_true, y_pred, beta=0.5, average="macro").execute().fetch(), + sklearn_fbeta_score(y_true, y_pred, beta=0.5, average="macro"), + ) + + np.testing.assert_array_almost_equal( + precision_score(y_true, y_pred, average="macro").execute().fetch(), + sklearn_precision_score(y_true, y_pred, average="macro"), + ) + + np.testing.assert_array_almost_equal( + recall_score(y_true, y_pred, average="macro").execute().fetch(), + sklearn_recall_score(y_true, y_pred, average="macro"), + ) + + np.testing.assert_array_almost_equal( + multilabel_confusion_matrix(y_true, y_pred).execute().fetch(), + sklearn_multilabel_confusion_matrix(y_true, y_pred), + ) + + np.testing.assert_array_almost_equal( + precision_recall_fscore_support(y_true, y_pred)[0].execute().fetch(), + sklearn_precision_recall_fscore_support(y_true, y_pred)[0], + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_scorer(): + from sklearn.metrics import r2_score + + from ...metrics import get_scorer + + assert get_scorer("r2") is not None + assert get_scorer(r2_score) is not None + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_r2_score(): + from ...metrics import r2_score + + y_true = np.array([[1, 0, 0, 1], [0, 1, 1, 1], [1, 1, 0, 1]]) + y_pred = np.array([[0, 0, 0, 1], [1, 0, 1, 1], [0, 0, 0, 1]]) + + error = r2_score(y_true, y_pred, multioutput="variance_weighted") + np.testing.assert_almost_equal(error.fetch(), 1.0 - 5.0 / 2) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_ranking(): + from sklearn.metrics import accuracy_score as sklearn_accuracy_score + from sklearn.metrics import auc as sklearn_auc + from sklearn.metrics import roc_curve as sklearn_roc_curve + from sklearn.metrics.tests.test_ranking import make_prediction + + from ...metrics import accuracy_score, auc, roc_auc_score, roc_curve + + y_true, y_score, _ = make_prediction(binary=True) + + np.testing.assert_almost_equal( + accuracy_score(y_true, y_score).fetch(), + sklearn_accuracy_score(y_true, y_score), + ) + rs = np.random.RandomState(0) + y = rs.randint(0, 10, (10,)) + pred = rs.rand(10) + fpr, tpr, thresholds = roc_curve(y, pred, pos_label=2) + m = auc(fpr, tpr) + + sk_fpr, sk_tpr, sk_threshod = sklearn_roc_curve( + y, + pred, + pos_label=2, + ) + expect_m = sklearn_auc(sk_fpr, sk_tpr) + assert pytest.approx(m.fetch()) == expect_m + y_true = np.array([0, 0, 1, 1], dtype=np.int64) + assert roc_auc_score(y_true, y_true, max_fpr=1) == 1 diff --git a/python/xorbits/sklearn/model_selection/__init__.py b/python/xorbits/sklearn/model_selection/__init__.py new file mode 100644 index 000000000..3b18a2c03 --- /dev/null +++ b/python/xorbits/sklearn/model_selection/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_ML_CALLABLES + + return list(MARS_SKLEARN_ML_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.model_selection as sk_ml + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_ML_CALLABLES + + if name in MARS_SKLEARN_ML_CALLABLES: + return MARS_SKLEARN_ML_CALLABLES[name] + else: + if not hasattr(sk_ml, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_ml, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/model_selection/mars_adapters/__init__.py b/python/xorbits/sklearn/model_selection/mars_adapters/__init__.py new file mode 100644 index 000000000..9a8b3c370 --- /dev/null +++ b/python/xorbits/sklearn/model_selection/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_ML_CALLABLES diff --git a/python/xorbits/sklearn/model_selection/mars_adapters/core.py b/python/xorbits/sklearn/model_selection/mars_adapters/core.py new file mode 100644 index 000000000..7f7869fbc --- /dev/null +++ b/python/xorbits/sklearn/model_selection/mars_adapters/core.py @@ -0,0 +1,51 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.model_selection as sk_ml + +from ...._mars.learn import model_selection as mars_ml +from ...._mars.learn.model_selection import KFold as MarsKFold +from ...._mars.learn.model_selection import ParameterGrid as MarsParameterGrid +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class KFold(SKLearnBase): + _marscls = MarsKFold + + +class ParameterGrid(SKLearnBase): + _marscls = MarsParameterGrid + + def __len__(self): + return len(self.mars_instance) + + def __iter__(self): + return iter(self.mars_instance) + + def __getitem__(self, index): + return self.mars_instance[index] + + +SKLEARN_ML_CLS_MAP = { + KFold: MarsKFold, + ParameterGrid: MarsParameterGrid, +} + +MARS_SKLEARN_ML_CALLABLES = _collect_module_callables( + mars_ml, sk_ml, skip_members=["register_op"] +) +_install_cls_members(SKLEARN_ML_CLS_MAP, MARS_SKLEARN_ML_CALLABLES, sk_ml) +attach_module_callable_docstring(KFold, sk_ml, sk_ml.KFold) +attach_module_callable_docstring(ParameterGrid, sk_ml, sk_ml.ParameterGrid) diff --git a/python/xorbits/sklearn/model_selection/tests/__init__.py b/python/xorbits/sklearn/model_selection/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/model_selection/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/model_selection/tests/test_core.py b/python/xorbits/sklearn/model_selection/tests/test_core.py new file mode 100644 index 000000000..e9cd89328 --- /dev/null +++ b/python/xorbits/sklearn/model_selection/tests/test_core.py @@ -0,0 +1,66 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +from typing import Iterable, Sized + +import numpy as np +import pytest + +from ...model_selection import KFold, ParameterGrid, train_test_split + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = KFold.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.model_selection." + ) + + docstring = ParameterGrid.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.model_selection." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_parameter_grid(): + arr1 = [1, 2, 3] + params1 = {"foo": arr1} + grid1 = ParameterGrid(params1) + assert isinstance(grid1, Iterable) + assert isinstance(grid1, Sized) + assert len(grid1) == 3 + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_kfold(): + X = np.array([[1, 2], [3, 4], [1, 2], [3, 4]]) + kf = KFold(n_splits=2) + splits = kf.get_n_splits(X) + assert splits == 2 + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_train_test_split(): + X = np.array([[1, 2], [3, 4], [1, 2], [3, 4]]) + y = np.array([1, 2, 3, 4]) + X_train, X_test, y_train, y_test = train_test_split(X, y, test_size=0.33) + assert X_train.shape == (2, 2) + assert X_test.shape == (2, 2) + assert y_train.shape == (2,) + assert y_test.shape == (2,) diff --git a/python/xorbits/sklearn/neighbors/__init__.py b/python/xorbits/sklearn/neighbors/__init__.py new file mode 100644 index 000000000..07bd6edea --- /dev/null +++ b/python/xorbits/sklearn/neighbors/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_NEIGHBORS_CALLABLES + + return list(MARS_SKLEARN_NEIGHBORS_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.neighbors as sk_neigh + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_NEIGHBORS_CALLABLES + + if name in MARS_SKLEARN_NEIGHBORS_CALLABLES: + return MARS_SKLEARN_NEIGHBORS_CALLABLES[name] + else: + if not hasattr(sk_neigh, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_neigh, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/neighbors/mars_adapters/__init__.py b/python/xorbits/sklearn/neighbors/mars_adapters/__init__.py new file mode 100644 index 000000000..4c58c1f1f --- /dev/null +++ b/python/xorbits/sklearn/neighbors/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_NEIGHBORS_CALLABLES diff --git a/python/xorbits/sklearn/neighbors/mars_adapters/core.py b/python/xorbits/sklearn/neighbors/mars_adapters/core.py new file mode 100644 index 000000000..bfa906478 --- /dev/null +++ b/python/xorbits/sklearn/neighbors/mars_adapters/core.py @@ -0,0 +1,39 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.neighbors as sk_neighbors + +from ...._mars.learn import neighbors as mars_neighbors +from ...._mars.learn.neighbors import NearestNeighbors as MarsNearestNeighbors +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class NearestNeighbors(SKLearnBase): + _marscls = MarsNearestNeighbors + + +SKLEARN_NEIGHBORS_CLS_MAP = { + NearestNeighbors: MarsNearestNeighbors, +} + +MARS_SKLEARN_NEIGHBORS_CALLABLES = _collect_module_callables( + mars_neighbors, sk_neighbors, skip_members=["register_op"] +) +_install_cls_members( + SKLEARN_NEIGHBORS_CLS_MAP, MARS_SKLEARN_NEIGHBORS_CALLABLES, sk_neighbors +) +attach_module_callable_docstring( + NearestNeighbors, sk_neighbors, sk_neighbors.NearestNeighbors +) diff --git a/python/xorbits/sklearn/neighbors/tests/__init__.py b/python/xorbits/sklearn/neighbors/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/neighbors/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/neighbors/tests/test_core.py b/python/xorbits/sklearn/neighbors/tests/test_core.py new file mode 100644 index 000000000..990931fff --- /dev/null +++ b/python/xorbits/sklearn/neighbors/tests/test_core.py @@ -0,0 +1,34 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import pytest + +from ...neighbors import NearestNeighbors + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = NearestNeighbors.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.neighbors." + ) + + docstring = NearestNeighbors.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.neighbors._unsupervised.NearestNeighbors." + ) diff --git a/python/xorbits/sklearn/preprocessing/__init__.py b/python/xorbits/sklearn/preprocessing/__init__.py new file mode 100644 index 000000000..bf05574d2 --- /dev/null +++ b/python/xorbits/sklearn/preprocessing/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_PREPROC_CALLABLES + + return list(MARS_SKLEARN_PREPROC_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.preprocessing as sk_preproc + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_PREPROC_CALLABLES + + if name in MARS_SKLEARN_PREPROC_CALLABLES: + return MARS_SKLEARN_PREPROC_CALLABLES[name] + else: + if not hasattr(sk_preproc, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_preproc, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/preprocessing/mars_adapters/__init__.py b/python/xorbits/sklearn/preprocessing/mars_adapters/__init__.py new file mode 100644 index 000000000..ecfb158b7 --- /dev/null +++ b/python/xorbits/sklearn/preprocessing/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_PREPROC_CALLABLES diff --git a/python/xorbits/sklearn/preprocessing/mars_adapters/core.py b/python/xorbits/sklearn/preprocessing/mars_adapters/core.py new file mode 100644 index 000000000..b71799039 --- /dev/null +++ b/python/xorbits/sklearn/preprocessing/mars_adapters/core.py @@ -0,0 +1,51 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.preprocessing as sk_preproc + +from ...._mars.learn import preprocessing as mars_preproc +from ...._mars.learn.preprocessing import LabelBinarizer as MarsLabelBinarizer +from ...._mars.learn.preprocessing import LabelEncoder as MarsLabelEncoder +from ...._mars.learn.preprocessing import MinMaxScaler as MarsMinMaxScaler +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class MinMaxScaler(SKLearnBase): + _marscls = MarsMinMaxScaler + + +class LabelBinarizer(SKLearnBase): + _marscls = MarsLabelBinarizer + + +class LabelEncoder(SKLearnBase): + _marscls = MarsLabelEncoder + + +SKLEARN_PREPROC_CLS_MAP = { + MinMaxScaler: MarsMinMaxScaler, + LabelEncoder: MarsLabelEncoder, + LabelBinarizer: MarsLabelBinarizer, +} + +MARS_SKLEARN_PREPROC_CALLABLES = _collect_module_callables( + mars_preproc, sk_preproc, skip_members=["register_op"] +) +_install_cls_members( + SKLEARN_PREPROC_CLS_MAP, MARS_SKLEARN_PREPROC_CALLABLES, sk_preproc +) +attach_module_callable_docstring(MinMaxScaler, sk_preproc, sk_preproc.MinMaxScaler) +attach_module_callable_docstring(LabelBinarizer, sk_preproc, sk_preproc.LabelBinarizer) +attach_module_callable_docstring(LabelEncoder, sk_preproc, sk_preproc.LabelEncoder) diff --git a/python/xorbits/sklearn/preprocessing/tests/__init__.py b/python/xorbits/sklearn/preprocessing/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/preprocessing/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/preprocessing/tests/test_core.py b/python/xorbits/sklearn/preprocessing/tests/test_core.py new file mode 100644 index 000000000..68f4cf3e3 --- /dev/null +++ b/python/xorbits/sklearn/preprocessing/tests/test_core.py @@ -0,0 +1,82 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest + +from ...preprocessing import LabelBinarizer, LabelEncoder, MinMaxScaler + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = MinMaxScaler.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing." + ) + + docstring = LabelBinarizer.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing." + ) + + docstring = LabelEncoder.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing." + ) + + docstring = MinMaxScaler.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing._data.MinMaxScaler." + ) + + docstring = LabelBinarizer.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing._label.LabelBinarizer." + ) + + docstring = LabelEncoder.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.preprocessing._label.LabelEncoder." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_min_max_scaler(): + X = np.array([[1, 2], [2, 4], [4, 8], [8, 16]], dtype=np.float64) + scaler = MinMaxScaler() + scaler.fit(X) + np.testing.assert_array_equal(scaler.data_min_, [1.0, 2.0]) + np.testing.assert_array_equal(scaler.data_max_, [8.0, 16.0]) + np.testing.assert_array_equal(scaler.data_range_, [7.0, 14.0]) + + X_transformed = scaler.transform(X).fetch() + assert X_transformed.shape == (4, 2) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_label_binarizer(): + lb = LabelBinarizer() + lb.fit([1, 2, 6, 4, 2]) + assert lb.classes_.tolist() == [1, 2, 4, 6] + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_label_encoder(): + le = LabelEncoder() + le.fit([1, 2, 2, 6]) + assert le.classes_.tolist() == [1, 2, 6] diff --git a/python/xorbits/sklearn/semi_supervised/__init__.py b/python/xorbits/sklearn/semi_supervised/__init__.py new file mode 100644 index 000000000..e4d2a1aca --- /dev/null +++ b/python/xorbits/sklearn/semi_supervised/__init__.py @@ -0,0 +1,49 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from ...core.utils.fallback import unimplemented_func + + +def _install(): + """Nothing required for installing sklearn.""" + + +def __dir__(): # pragma: no cover + try: + import sklearn + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_SS_CALLABLES + + return list(MARS_SKLEARN_SS_CALLABLES.keys()) + + +def __getattr__(name: str): # pragma: no cover + import inspect + + try: + import sklearn.semi_supervised as sk_ss + except ImportError: + raise AttributeError("sklearn is required but not installed.") + from .mars_adapters import MARS_SKLEARN_SS_CALLABLES + + if name in MARS_SKLEARN_SS_CALLABLES: + return MARS_SKLEARN_SS_CALLABLES[name] + else: + if not hasattr(sk_ss, name): + raise AttributeError(name) + else: + if inspect.ismethod(getattr(sk_ss, name)): + return unimplemented_func() + else: + raise AttributeError diff --git a/python/xorbits/sklearn/semi_supervised/mars_adapters/__init__.py b/python/xorbits/sklearn/semi_supervised/mars_adapters/__init__.py new file mode 100644 index 000000000..b53765590 --- /dev/null +++ b/python/xorbits/sklearn/semi_supervised/mars_adapters/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from .core import MARS_SKLEARN_SS_CALLABLES diff --git a/python/xorbits/sklearn/semi_supervised/mars_adapters/core.py b/python/xorbits/sklearn/semi_supervised/mars_adapters/core.py new file mode 100644 index 000000000..9f144f49b --- /dev/null +++ b/python/xorbits/sklearn/semi_supervised/mars_adapters/core.py @@ -0,0 +1,35 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sklearn.semi_supervised as sk_ss + +from ...._mars.learn import semi_supervised as mars_ss +from ...._mars.learn.semi_supervised import LabelPropagation as MarsLabelPropagation +from ....core.utils.docstring import attach_module_callable_docstring +from ...utils import SKLearnBase, _collect_module_callables, _install_cls_members + + +class LabelPropagation(SKLearnBase): + _marscls = MarsLabelPropagation + + +SKLEARN_SS_CLS_MAP = { + LabelPropagation: MarsLabelPropagation, +} + +MARS_SKLEARN_SS_CALLABLES = _collect_module_callables( + mars_ss, sk_ss, skip_members=["register_op"] +) +_install_cls_members(SKLEARN_SS_CLS_MAP, MARS_SKLEARN_SS_CALLABLES, sk_ss) +attach_module_callable_docstring(LabelPropagation, sk_ss, sk_ss.LabelPropagation) diff --git a/python/xorbits/sklearn/semi_supervised/tests/__init__.py b/python/xorbits/sklearn/semi_supervised/tests/__init__.py new file mode 100644 index 000000000..37f6558d9 --- /dev/null +++ b/python/xorbits/sklearn/semi_supervised/tests/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/python/xorbits/sklearn/semi_supervised/tests/test_core.py b/python/xorbits/sklearn/semi_supervised/tests/test_core.py new file mode 100644 index 000000000..3739664cc --- /dev/null +++ b/python/xorbits/sklearn/semi_supervised/tests/test_core.py @@ -0,0 +1,48 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +try: + import sklearn +except ImportError: # pragma: no cover + sklearn = None + +import numpy as np +import pytest + +from ...semi_supervised import LabelPropagation + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_doc(): + docstring = LabelPropagation.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.semi_supervised." + ) + + docstring = LabelPropagation.fit.__doc__ + assert docstring is not None and docstring.endswith( + "This docstring was copied from sklearn.semi_supervised._label_propagation.LabelPropagation." + ) + + +@pytest.mark.skipif(sklearn is None, reason="scikit-learn not installed") +def test_label_propagation(): + rng = np.random.RandomState(0) + X = rng.rand(10, 5) + y = np.array([0, 0, 0, 1, 1, -1, -1, -1, -1, -1]) + lp = LabelPropagation() + lp.fit(X, y) + assert lp.classes_.tolist() == [0, 1] + assert lp.transduction_.tolist() == [0, 0, 0, 1, 1, 0, 0, 0, 0, 0] + assert lp.predict(X).tolist() == [0, 0, 0, 1, 1, 0, 0, 0, 0, 0] + assert lp.score(X, y) == 0.5 diff --git a/python/xorbits/sklearn/utils.py b/python/xorbits/sklearn/utils.py new file mode 100644 index 000000000..ac2834e86 --- /dev/null +++ b/python/xorbits/sklearn/utils.py @@ -0,0 +1,72 @@ +# Copyright 2022-2023 XProbe Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import functools +import inspect +from typing import Callable, Dict, List, Optional + +from ..core.adapter import to_mars, wrap_mars_callable + + +class SKLearnBase: + def __init__(self, *args, **kwargs): + self.mars_instance = self._marscls(*to_mars(args), **to_mars(kwargs)) + + def __getattr__(self, name): + return getattr(self.mars_instance, name) + + +def wrap_cls_func(marscls: Callable, name: str, submodule): + @functools.wraps(getattr(marscls, name)) + def wrapped(self, *args, **kwargs): + return getattr(self.mars_instance, name)(*args, **kwargs) + + return wrap_mars_callable( + wrapped, + member_name=name, + attach_docstring=True, + is_cls_member=True, + docstring_src_module=submodule, + docstring_src_cls=getattr(submodule, marscls.__name__, None), + ) + + +def _collect_module_callables( + mars_module, + orig_module, + skip_members: Optional[List[str]] = None, +) -> Dict[str, Callable]: + module_callables: Dict[str, Callable] = dict() + + for name, func in inspect.getmembers(mars_module, inspect.isfunction): + if skip_members is not None and name in skip_members: + continue + module_callables[name] = wrap_mars_callable( + func, + attach_docstring=True, + is_cls_member=False, + docstring_src_module=orig_module, + docstring_src=getattr(orig_module, name, None), + ) + return module_callables + + +def _install_cls_members( + module_cls_map, module_callables: Dict[str, Callable], orig_submodule +): + for x_cls, mars_cls in module_cls_map.items(): + module_callables[x_cls.__name__] = x_cls + for name, _ in inspect.getmembers(mars_cls, inspect.isfunction): + if not name.startswith("_"): + setattr(x_cls, name, wrap_cls_func(mars_cls, name, orig_submodule)) From 07bef18aabd6363eea786246b3565f8976bbd0d3 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Thu, 19 Oct 2023 02:40:32 -0500 Subject: [PATCH 26/27] TST: Fix xorbits.sklearn windows CI (#748) --- python/xorbits/_mars/tensor/statistics/bincount.py | 6 ++++-- python/xorbits/_mars/utils.py | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/python/xorbits/_mars/tensor/statistics/bincount.py b/python/xorbits/_mars/tensor/statistics/bincount.py index bb25fe03e..fcec8603a 100644 --- a/python/xorbits/_mars/tensor/statistics/bincount.py +++ b/python/xorbits/_mars/tensor/statistics/bincount.py @@ -286,8 +286,10 @@ def bincount(x, weights=None, minlength=0, chunk_size_limit=None): x = astensor(x) weights = astensor(weights) if weights is not None else None - if not np.issubdtype(x.dtype, np.int_): - raise TypeError(f"Cannot cast array data from {x.dtype} to {np.dtype(np.int_)}") + if not np.issubdtype(x.dtype, np.int64): + raise TypeError( + f"Cannot cast array data from {x.dtype} to {np.dtype(np.int64)}" + ) if x.ndim != 1: raise ValueError("'x' must be 1 dimension") if minlength < 0: diff --git a/python/xorbits/_mars/utils.py b/python/xorbits/_mars/utils.py index 7c54ddd3d..91f00ede4 100644 --- a/python/xorbits/_mars/utils.py +++ b/python/xorbits/_mars/utils.py @@ -489,13 +489,15 @@ def calc_data_size(dt: Any, shape: Tuple[int] = None) -> int: return 0 if isinstance(dt, tuple): - return sum(calc_data_size(c) for c in dt) + # int() for windows CI, otherwise may return numpy.int32 by `sum` + return int(sum(calc_data_size(c) for c in dt)) shape = getattr(dt, "shape", None) or shape if isinstance(dt, (pd.DataFrame, pd.Series)): return estimate_pandas_size(dt) if hasattr(dt, "estimate_size"): - return dt.estimate_size() + # int() for windows CI, otherwise may return numpy.int32 + return int(dt.estimate_size()) if dt.estimate_size() is not None else None if hasattr(dt, "nbytes"): return max(sys.getsizeof(dt), dt.nbytes) if hasattr(dt, "shape") and len(dt.shape) == 0: From f67ffd4d6da7a1be76c2c5342421f56a01697a09 Mon Sep 17 00:00:00 2001 From: Chengjie Li <109656400+ChengjieLi28@users.noreply.github.com> Date: Thu, 19 Oct 2023 22:04:59 -0500 Subject: [PATCH 27/27] BUG: column pruning causes missing columns on `DataFrameIndex` op (#743) Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- .../column_pruning/column_pruning_rule.py | 10 ++++- .../tests/test_column_pruning.py | 40 +++++++++++++++++++ 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/column_pruning_rule.py b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/column_pruning_rule.py index a90544a61..9b7c77e8e 100644 --- a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/column_pruning_rule.py +++ b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/column_pruning_rule.py @@ -55,9 +55,17 @@ def _get_successor_required_columns(self, data: TileableData) -> Set[Any]: """ successors = self._get_successors(data) if successors: - return set().union( + res = set().union( *[self._context[successor][data] for successor in successors] ) + # When getting the required columns of a DataFrameIndex node, we need to consider itself. + if ( + isinstance(data, BaseDataFrameData) + and isinstance(data.op, DataFrameIndex) + and len(data.dtypes) > 0 + ): + res = res.union(set(data.dtypes.index)) + return res else: return self._get_all_columns(data) diff --git a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_column_pruning.py b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_column_pruning.py index 321a911d0..9e869f48b 100644 --- a/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_column_pruning.py +++ b/python/xorbits/_mars/optimization/logical/tileable/column_pruning/tests/test_column_pruning.py @@ -15,6 +15,7 @@ import os import tempfile +import numpy as np import pandas as pd import pytest @@ -598,3 +599,42 @@ def test_setitem(setup, gen_data1): raw1["c5"] = raw2["c1"] expected = raw1.groupby(by="c1", as_index=False).sum()["c2"] pd.testing.assert_series_equal(r.execute().fetch(), expected) + + +def test_merge_index_groupby_agg(setup, gen_data1): + file_path, file_path2 = gen_data1 + left = md.read_csv(file_path) + right = md.read_csv(file_path2) + r = left.merge(right, on="c1") + data = r[["c1", "c2_x", "c2_y", "c4_x", "c4_y"]] + + def udf(x): + return np.sum(x) + + res = data.groupby("c1").agg({"c2_x": udf}) + + graph = res.build_graph() + optimize(graph) + + agg_node = graph.result_tileables[0] + assert isinstance(agg_node.op, DataFrameGroupByAgg) + + assert len(graph.predecessors(agg_node)) == 1 + index_node = graph.predecessors(agg_node)[0] + assert type(index_node.op) is DataFrameIndex + assert set(index_node.op.col_names) == {"c1", "c2_x"} + + index_node2 = graph.predecessors(index_node)[0] + assert type(index_node2.op) is DataFrameIndex + assert set(index_node2.op.col_names) == {"c1", "c2_x", "c2_y", "c4_x", "c4_y"} + + merge_node = graph.predecessors(index_node2)[0] + assert type(merge_node.op) is DataFrameMerge + + read_csv_node_left, read_csv_node_right = graph.predecessors(merge_node) + assert type(read_csv_node_left.op) is DataFrameReadCSV + assert type(read_csv_node_right.op) is DataFrameReadCSV + assert len(read_csv_node_left.op.usecols) == 3 + assert len(read_csv_node_right.op.usecols) == 3 + assert set(read_csv_node_left.op.usecols) == {"c1", "c2", "c4"} + assert set(read_csv_node_right.op.usecols) == {"c1", "c2", "c4"}