PandasOnDaskDataframePartition#
The class is the specific implementation of PandasDataframePartition
,
providing the API to perform operations on a block partition, namely, pandas.DataFrame
, using Dask as the execution engine.
In addition to wrapping a pandas.DataFrame
, the class also holds the following metadata:
length
- length ofpandas.DataFrame
wrappedwidth
- width ofpandas.DataFrame
wrappedip
- node IP address that holdspandas.DataFrame
wrapped
An operation on a block partition can be performed in two modes:
asynchronously - via
apply()
lazily - via
add_to_apply_calls()
Public API#
- class modin.core.execution.dask.implementations.pandas_on_dask.partitioning.PandasOnDaskDataframePartition(data, length=None, width=None, ip=None, call_queue=None)#
The class implements the interface in
PandasDataframePartition
.- Parameters
data (distributed.Future) – A reference to pandas DataFrame that need to be wrapped with this class.
length (distributed.Future or int, optional) – Length or reference to it of wrapped pandas DataFrame.
width (distributed.Future or int, optional) – Width or reference to it of wrapped pandas DataFrame.
ip (distributed.Future or str, optional) – Node IP address or reference to it that holds wrapped pandas DataFrame.
call_queue (list, optional) – Call queue that needs to be executed on wrapped pandas DataFrame.
- apply(func, *args, **kwargs)#
Apply a function to the object wrapped by this partition.
- Parameters
func (callable or distributed.Future) – A function to apply.
*args (iterable) – Additional positional arguments to be passed in func.
**kwargs (dict) – Additional keyword arguments to be passed in func.
- Returns
A new
PandasOnDaskDataframePartition
object.- Return type
Notes
The keyword arguments are sent as a dictionary.
- drain_call_queue()#
Execute all operations stored in the call queue on the object wrapped by this partition.
- execution_wrapper#
alias of
DaskWrapper
- ip()#
Get the node IP address of the object wrapped by this partition.
- Returns
IP address of the node that holds the data.
- Return type
str
- length(materialize=True)#
Get the length of the object wrapped by this partition.
- Parameters
materialize (bool, default: True) – Whether to forcibly materialize the result into an integer. If
False
was specified, may return a future of the result if it hasn’t been materialized yet.- Returns
The length of the object.
- Return type
int or distributed.Future
- mask(row_labels, col_labels)#
Lazily create a mask that extracts the indices provided.
- Parameters
row_labels (list-like, slice or label) – The row labels for the rows to extract.
col_labels (list-like, slice or label) – The column labels for the columns to extract.
- Returns
A new
PandasOnDaskDataframePartition
object.- Return type
- classmethod preprocess_func(func)#
Preprocess a function before an
apply
call.- Parameters
func (callable) – The function to preprocess.
- Returns
An object that can be accepted by
apply
.- Return type
callable
- classmethod put(obj)#
Put an object into distributed memory and wrap it with partition object.
- Parameters
obj (any) – An object to be put.
- Returns
A new
PandasOnDaskDataframePartition
object.- Return type
- wait()#
Wait completing computations on the object wrapped by the partition.
- width(materialize=True)#
Get the width of the object wrapped by the partition.
- Parameters
materialize (bool, default: True) – Whether to forcibly materialize the result into an integer. If
False
was specified, may return a future of the result if it hasn’t been materialized yet.- Returns
The width of the object.
- Return type
int or distributed.Future