From b18b3af20e471c45089aa1b3085351a0c7d1bca2 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 28 Mar 2024 13:15:06 -0400 Subject: [PATCH 01/25] start --- src/dask_awkward/layers/layers.py | 164 +--------------------------- src/dask_awkward/lib/core.py | 16 ++- src/dask_awkward/lib/io/columnar.py | 25 ----- src/dask_awkward/lib/io/io.py | 27 +++-- src/dask_awkward/lib/operations.py | 2 + src/dask_awkward/lib/optimize.py | 139 +---------------------- src/dask_awkward/lib/testutils.py | 3 - 7 files changed, 38 insertions(+), 338 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index 92441443..03608e59 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -30,13 +30,6 @@ def from_blockwise(cls, layer: Blockwise) -> AwkwardBlockwiseLayer: ob.__dict__.update(layer.__dict__) return ob - def mock(self) -> AwkwardBlockwiseLayer: - layer = copy.copy(self) - nb = layer.numblocks - layer.numblocks = {k: tuple(1 for _ in v) for k, v in nb.items()} - layer.__dict__.pop("_dims", None) - return layer - def __getstate__(self) -> dict: # Indicator that this layer has been serialised state = self.__dict__.copy() @@ -54,10 +47,6 @@ def __call__(self, *args, **kwargs): ... T = TypeVar("T") -class ImplementsMocking(ImplementsIOFunction, Protocol): - def mock(self) -> AwkwardArray: ... - - class ImplementsMockEmpty(ImplementsIOFunction, Protocol): def mock_empty(self, backend: BackendT) -> AwkwardArray: ... @@ -67,9 +56,7 @@ class ImplementsReport(ImplementsIOFunction, Protocol): def return_report(self) -> bool: ... -class ImplementsProjection(ImplementsMocking, Protocol[T]): - def prepare_for_projection(self) -> tuple[AwkwardArray, TypeTracerReport, T]: ... - +class ImplementsProjection(Protocol[T]): def project(self, report: TypeTracerReport, state: T) -> ImplementsIOFunction: ... @@ -79,7 +66,7 @@ def necessary_columns( ) -> frozenset[str]: ... -class IOFunctionWithMocking(ImplementsMocking, ImplementsIOFunction): +class IOFunctionWithMocking(ImplementsIOFunction): def __init__(self, meta: AwkwardArray, io_func: ImplementsIOFunction): self._meta = meta self._io_func = io_func @@ -92,23 +79,11 @@ def __getstate__(self) -> dict: def __call__(self, *args, **kwargs): return self._io_func(*args, **kwargs) - def mock(self) -> AwkwardArray: - assert self._meta is not None - return self._meta - def io_func_implements_projection(func: ImplementsIOFunction) -> bool: return hasattr(func, "prepare_for_projection") -def io_func_implements_mocking(func: ImplementsIOFunction) -> bool: - return hasattr(func, "mock") - - -def io_func_implements_mock_empty(func: ImplementsIOFunction) -> bool: - return hasattr(func, "mock_empty") - - def io_func_implements_columnar(func: ImplementsIOFunction) -> bool: return hasattr(func, "necessary_columns") @@ -179,78 +154,10 @@ def is_projectable(self) -> bool: io_func_implements_projection(self.io_func) and not self.has_been_unpickled ) - @property - def is_mockable(self) -> bool: - # isinstance(self.io_func, ImplementsMocking) - return io_func_implements_mocking(self.io_func) - @property def is_columnar(self) -> bool: return io_func_implements_columnar(self.io_func) - def mock(self) -> AwkwardInputLayer: - assert self.is_mockable - return AwkwardInputLayer( - name=self.name, - inputs=[None][: int(list(self.numblocks.values())[0][0])], - io_func=lambda *_, **__: cast(ImplementsMocking, self.io_func).mock(), - label=self.label, - produces_tasks=self.produces_tasks, - creation_info=self.creation_info, - annotations=self.annotations, - ) - - def prepare_for_projection(self) -> tuple[AwkwardInputLayer, TypeTracerReport, T]: - """Mock the input layer as starting with a data-less typetracer. - This method is used to create new dask task graphs that - operate purely on typetracer Arrays (that is, array with - awkward structure but without real data buffers). This allows - us to test which parts of a real awkward array will be used in - a real computation. We do this by running a graph which starts - with mocked AwkwardInputLayers. - - We mock an AwkwardInputLayer in these steps: - 1. Ask the IO function to prepare a new meta array, and return - any transient state. - 2. Build a new AwkwardInputLayer whose IO function just returns - this meta (typetracer) array - 3. Return the new input layer and the transient state - - When this new layer is added to a dask task graph and that - graph is computed, the report object will be mutated. - Inspecting the report object after the compute tells us which - buffers from the original form would be required for a real - compute with the same graph. - Returns - ------- - AwkwardInputLayer - Copy of the input layer with data-less input. - TypeTracerReport - The report object used to track touched buffers. - Any - The black-box state object returned by the IO function. - """ - assert self.is_projectable - new_meta_array, report, state = cast( - ImplementsProjection, self.io_func - ).prepare_for_projection() - - new_return = new_meta_array - if io_func_implements_report(self.io_func): - if cast(ImplementsReport, self.io_func).return_report: - new_return = (new_meta_array, type(new_meta_array)([])) - - new_input_layer = AwkwardInputLayer( - name=self.name, - inputs=[None][: int(list(self.numblocks.values())[0][0])], - io_func=AwkwardTokenizable(new_return, self.name), - label=self.label, - produces_tasks=self.produces_tasks, - creation_info=self.creation_info, - annotations=self.annotations, - ) - return new_input_layer, report, state - def project( self, report: TypeTracerReport, @@ -290,68 +197,5 @@ def __init__( self.fn = fn super().__init__(mapping, **kwargs) - def mock(self) -> MaterializedLayer: - mapping = copy.copy(self.mapping) - if not mapping: - # no partitions at all - return self - name = next(iter(mapping))[0] - - npln = len(self.previous_layer_names) - # one previous layer name - # - # this case is used for mocking repartition or slicing where - # we maybe have multiple partitions that need to be included - # in a task. - if npln == 1: - prev_name: str = self.previous_layer_names[0] - if (name, 0) in mapping: - task = mapping[(name, 0)] - task = tuple( - ( - (prev_name, 0) - if isinstance(v, tuple) and len(v) == 2 and v[0] == prev_name - else v - ) - for v in task - ) - - # when using Array.partitions we need to mock that we - # just want the first partition. - if len(task) == 2 and isinstance(task[1], int) and task[1] > 0: - task = (task[0], 0) - return MaterializedLayer({(name, 0): task}) - return self - - # zero previous layers; this is likely a known scalar. - # - # we just use the existing mapping - elif npln == 0: - return MaterializedLayer({(name, 0): mapping[(name, 0)]}) - - # more than one previous_layer_names - # - # this case is needed for dak.concatenate on axis=0; we need - # the first partition of _each_ of the previous layer names! - else: - if self.fn is None: - raise ValueError( - "For multiple previous layers the fn argument cannot be None." - ) - name0s = tuple((name, 0) for name in self.previous_layer_names) - task = (self.fn, *name0s) - return MaterializedLayer({(name, 0): task}) - - -class AwkwardTreeReductionLayer(DataFrameTreeReduction): - def mock(self) -> AwkwardTreeReductionLayer: - return AwkwardTreeReductionLayer( - name=self.name, - name_input=self.name_input, - npartitions_input=1, - concat_func=self.concat_func, - tree_node_func=self.tree_node_func, - finalize_func=self.finalize_func, - split_every=self.split_every, - tree_node_name=self.tree_node_name, - ) + +class AwkwardTreeReductionLayer(DataFrameTreeReduction): ... diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index 40898b11..d60522b1 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -386,6 +386,10 @@ def _rebuild(self, dsk, *, rename=None): def __reduce__(self): return (Scalar, (self.dask, self.name, None, self.dtype, self.known_value)) + @property + def report(self): + return getattr(self.meta, "report", set()) + @property def dask(self) -> HighLevelGraph: return self._dask @@ -937,6 +941,10 @@ def reset_meta(self) -> None: """Assign an empty typetracer array as the collection metadata.""" self._meta = empty_typetracer() + @property + def report(self): + return getattr(self.meta, "report", set()) + def repartition( self, npartitions: int | None = None, @@ -1734,6 +1742,7 @@ def new_array_object( attrs: Mapping[str, Any] | None = None, npartitions: int | None = None, divisions: tuple[int, ...] | tuple[None, ...] | None = None, + report=set(), ) -> Array: """Instantiate a new Array collection object. @@ -1801,6 +1810,9 @@ def new_array_object( actual_meta.attrs = attrs out = Array(dsk, name, actual_meta, divs) + if report: + [r.commit(out.name) for r in report] + actual_meta._report = report if actual_meta.__doc__ != actual_meta.__class__.__doc__: out.__doc__ = actual_meta.__doc__ @@ -2195,10 +2207,6 @@ def non_trivial_reduction( if combiner is None: combiner = reducer - # is_positional == True is not implemented - # if is_positional: - # assert combiner is reducer - # For `axis=None`, we prepare each array to have the following structure: # [[[ ... [x1 x2 x3 ... xN] ... ]]] (length-1 outer lists) # This makes the subsequent reductions an `axis=-1` reduction diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index 32ffb3ae..a21b8b10 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -59,12 +59,6 @@ class ColumnProjectionMixin(ImplementsNecessaryColumns[FormStructure]): when only metadata buffers are required. """ - def mock(self: S) -> AwkwardArray: - return cast( - AwkwardArray, - typetracer_from_form(self.form, behavior=self.behavior, attrs=self.attrs), - ) - def mock_empty(self: S, backend: BackendT = "cpu") -> AwkwardArray: return cast( AwkwardArray, @@ -75,25 +69,6 @@ def mock_empty(self: S, backend: BackendT = "cpu") -> AwkwardArray: ), ) - def prepare_for_projection( - self: S, - ) -> tuple[AwkwardArray, TypeTracerReport, FormStructure]: - form = form_with_unique_keys(self.form, "@") - - # Build typetracer and associated report object - (meta, report) = typetracer_with_report( - form, - highlevel=True, - behavior=self.behavior, - buffer_key=render_buffer_key, - ) - - return ( - cast(AwkwardArray, meta), - report, - trace_form_structure(form, buffer_key=render_buffer_key), - ) - def necessary_columns( self: S, report: TypeTracerReport, diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index d8fd60b1..5dc9fc7d 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -11,7 +11,7 @@ import dask.config import numpy as np from awkward.types.numpytype import primitive_to_dtype -from awkward.typetracer import length_zero_if_typetracer +from awkward.typetracer import length_zero_if_typetracer, typetracer_with_report from dask.base import flatten, tokenize from dask.highlevelgraph import HighLevelGraph from dask.local import identity @@ -23,10 +23,9 @@ AwkwardInputLayer, AwkwardMaterializedLayer, AwkwardTreeReductionLayer, - ImplementsMocking, ImplementsReport, IOFunctionWithMocking, - io_func_implements_mocking, + io_func_implements_projection, io_func_implements_report, ) from dask_awkward.lib.core import ( @@ -37,6 +36,7 @@ typetracer_array, ) from dask_awkward.lib.io.columnar import ColumnProjectionMixin +from dask_awkward.lib.utils import render_buffer_key from dask_awkward.utils import first, second if TYPE_CHECKING: @@ -620,11 +620,18 @@ def from_map( packed=packed, ) - # Special `io_func` implementations can implement mocking and optionally - # support buffer projection. - if io_func_implements_mocking(func): + kw = {} + if io_func_implements_projection(func): + # Special `io_func` implementations can do buffer projection - choosing columns + # so here we start with a blank report io_func = func - array_meta = cast(ImplementsMocking, func).mock() + array_meta, report = typetracer_with_report( + io_func.form, + highlevel=True, + behavior=io_func.behavior, + buffer_key=render_buffer_key, + ) + kw["report"] = {report} # column tracking report, not failure report, below # If we know the meta, we can spoof mocking elif meta is not None: io_func = IOFunctionWithMocking(meta, func) @@ -638,9 +645,11 @@ def from_map( hlg = HighLevelGraph.from_collections(name, dsk) if divisions is not None: - result = new_array_object(hlg, name, meta=array_meta, divisions=divisions) + result = new_array_object(hlg, name, meta=array_meta, divisions=divisions, **kw) else: - result = new_array_object(hlg, name, meta=array_meta, npartitions=len(inputs)) + result = new_array_object( + hlg, name, meta=array_meta, npartitions=len(inputs), **kw + ) if io_func_implements_report(io_func): if cast(ImplementsReport, io_func).return_report: diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index 6f1da8e5..70811e98 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -60,6 +60,7 @@ def concatenate( name = f"{label}-{token}" metas = [c._meta for c in arrays] + report = set.union(getattr(m, "_report", set()) for m in metas) if len(metas) == 0: raise ValueError("Need at least one array to concatenate") @@ -127,6 +128,7 @@ def concatenate( name, meta=meta_no_report, npartitions=sum(a.npartitions for a in arrays), + report=report, ) if axis > 0: diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index a1ab9d6a..4dcde735 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -75,80 +75,6 @@ def optimize(dsk: HighLevelGraph, keys: Sequence[Key], **_: Any) -> Mapping: return dsk -def _prepare_buffer_projection( - dsk: HighLevelGraph, keys: Sequence[Key] -) -> tuple[dict[str, TypeTracerReport], dict[str, Any]] | None: - """Pair layer names with lists of necessary columns.""" - import awkward as ak - - if not _has_projectable_awkward_io_layer(dsk): - return None - - layer_to_projection_state: dict[str, Any] = {} - layer_to_reports: dict[str, TypeTracerReport] = {} - projection_layers = dict(dsk.layers) - - for name, lay in dsk.layers.items(): - if isinstance(lay, AwkwardInputLayer): - if lay.is_projectable: - # Insert mocked array into layers, replacing generation func - # Keep track of mocked state - ( - projection_layers[name], - layer_to_reports[name], - layer_to_projection_state[name], - ) = lay.prepare_for_projection() - elif lay.is_mockable: - projection_layers[name] = lay.mock() - elif hasattr(lay, "mock"): - projection_layers[name] = lay.mock() - - for name in _ak_output_layer_names(dsk): - projection_layers[name] = _mock_output(projection_layers[name]) - - hlg = HighLevelGraph(projection_layers, dsk.dependencies) - - minimal_keys: set[Key] = set() - for k in keys: - if isinstance(k, tuple) and len(k) == 2: - minimal_keys.add((k[0], 0)) - else: - minimal_keys.add(k) - - # now we try to compute for each possible output layer key (leaf - # node on partition 0); this will cause the typetacer reports to - # get correct fields/columns touched. If the result is a record or - # an array we of course want to touch all of the data/fields. - try: - for layer in hlg.layers.values(): - layer.__dict__.pop("_cached_dict", None) - results = get_sync(hlg, list(minimal_keys)) - for out in results: - if isinstance(out, (ak.Array, ak.Record)): - touch_data(out) - except Exception as err: - on_fail = dask.config.get("awkward.optimization.on-fail") - # this is the default, throw a warning but skip the optimization. - if on_fail == "warn": - warnings.warn( - COLUMN_OPT_FAILED_WARNING_MSG.format(exception=type(err), message=err) - ) - # option "pass" means do not throw warning but skip the optimization. - elif on_fail == "pass": - log.debug("Column projection optimization failed; optimization skipped.") - # option "raise" to raise the exception here - elif on_fail == "raise": - raise - else: - raise ValueError( - f"Invalid awkward.optimization.on-fail option: {on_fail}.\n" - "Valid options are 'warn', 'pass', or 'raise'." - ) - return None - else: - return layer_to_reports, layer_to_projection_state - - def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: """Run column projection optimization. @@ -178,70 +104,9 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph New, optimized task graph with column-projected ``AwkwardInputLayer``. """ - projection_data = _prepare_buffer_projection(dsk, keys) - if projection_data is None: - return dsk - - # Unpack result - layer_to_reports, layer_to_projection_state = projection_data - - # Project layers using projection state - layers = dict(dsk.layers) - for name, state in layer_to_projection_state.items(): - layers[name] = cast(AwkwardInputLayer, layers[name]).project( - report=layer_to_reports[name], state=state - ) - - return HighLevelGraph(layers, dsk.dependencies) - - -def _layers_with_annotation(dsk: HighLevelGraph, key: str) -> list[str]: - return [n for n, v in dsk.layers.items() if (v.annotations or {}).get(key)] - - -def _ak_output_layer_names(dsk: HighLevelGraph) -> list[str]: - """Get a list output layer names. - - Output layer names are annotated with 'ak_output'. - - Parameters - ---------- - dsk : HighLevelGraph - Graph of interest. - - Returns - ------- - list[str] - Names of the output layers. - - """ - return _layers_with_annotation(dsk, "ak_output") - - -def _has_projectable_awkward_io_layer(dsk: HighLevelGraph) -> bool: - """Check if a graph at least one AwkwardInputLayer that is project-able.""" - for _, v in dsk.layers.items(): - if isinstance(v, AwkwardInputLayer) and v.is_projectable: - return True - return False - - -def _touch_all_data(*args, **kwargs): - """Mock writing an ak.Array to disk by touching data buffers.""" - for arg in args + tuple(kwargs.values()): - touch_data(arg) - - -def _mock_output(layer): - """Update a layer to run the _touch_all_data.""" - assert len(layer.dsk) == 1 + # TBD - new_layer = copy.deepcopy(layer) - mp = new_layer.dsk.copy() - for k in iter(mp.keys()): - mp[k] = (_touch_all_data,) + mp[k][1:] - new_layer.dsk = mp - return new_layer + return dsk # HighLevelGraph(layers, dsk.dependencies) def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: diff --git a/src/dask_awkward/lib/testutils.py b/src/dask_awkward/lib/testutils.py index cc55d858..e47d5c8d 100644 --- a/src/dask_awkward/lib/testutils.py +++ b/src/dask_awkward/lib/testutils.py @@ -294,9 +294,6 @@ def __call__(self, *args, **kwargs): return self.read_fn(*args, **kwargs) - def mock(self): - return typetracer_from_form(self.form) - def mock_empty(self, backend="cpu"): return ak.to_backend( self.form.length_zero_array(highlevel=False), From 3cdc7783ead653ed0e4085197f7208d044e20c8a Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 28 Mar 2024 15:59:49 -0400 Subject: [PATCH 02/25] Remove unused --- src/dask_awkward/layers/layers.py | 6 -- src/dask_awkward/lib/io/columnar.py | 96 +---------------------------- src/dask_awkward/lib/optimize.py | 6 -- src/dask_awkward/lib/testutils.py | 1 - 4 files changed, 2 insertions(+), 107 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index 03608e59..ecc208bf 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -177,12 +177,6 @@ def project( annotations=self.annotations, ) - def necessary_columns(self, report: TypeTracerReport, state: T) -> frozenset[str]: - assert self.is_columnar - return cast(ImplementsNecessaryColumns, self.io_func).necessary_columns( - report=report, state=state - ) - class AwkwardMaterializedLayer(MaterializedLayer): def __init__( diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index a21b8b10..7268b816 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -6,23 +6,13 @@ import awkward as ak from awkward import Array as AwkwardArray from awkward.forms import Form -from awkward.typetracer import typetracer_from_form, typetracer_with_report from dask_awkward.layers.layers import ( BackendT, ImplementsIOFunction, ImplementsNecessaryColumns, ) -from dask_awkward.lib.utils import ( - METADATA_ATTRIBUTES, - FormStructure, - buffer_keys_required_to_compute_shapes, - form_with_unique_keys, - parse_buffer_key, - render_buffer_key, - trace_form_structure, - walk_graph_depth_first, -) +from dask_awkward.lib.utils import METADATA_ATTRIBUTES, FormStructure if TYPE_CHECKING: from awkward._nplikes.typetracer import TypeTracerReport @@ -60,6 +50,7 @@ class ColumnProjectionMixin(ImplementsNecessaryColumns[FormStructure]): """ def mock_empty(self: S, backend: BackendT = "cpu") -> AwkwardArray: + # used by failure report generation return cast( AwkwardArray, ak.to_backend( @@ -68,86 +59,3 @@ def mock_empty(self: S, backend: BackendT = "cpu") -> AwkwardArray: highlevel=True, ), ) - - def necessary_columns( - self: S, - report: TypeTracerReport, - state: FormStructure, - ) -> frozenset[str]: - ## Read from stash - # Form hierarchy information - form_key_to_parent_form_key = state["form_key_to_parent_form_key"] - form_key_to_child_form_keys: dict[str, list[str]] = {} - for child_key, parent_key in form_key_to_parent_form_key.items(): - form_key_to_child_form_keys.setdefault(parent_key, []).append(child_key) # type: ignore - form_key_to_form = state["form_key_to_form"] - # Buffer hierarchy information - form_key_to_buffer_keys = state["form_key_to_buffer_keys"] - # Column hierarchy information - form_key_to_path = state["form_key_to_path"] - - # Require the data of metadata buffers above shape-only requests - data_buffers = { - *report.data_touched, - *buffer_keys_required_to_compute_shapes( - parse_buffer_key, - report.shape_touched, - form_key_to_parent_form_key, - form_key_to_buffer_keys, - ), - } - - # We can't read buffers directly, but if we encounter a metadata - # buffer, then we should be able to pick any child. - paths = set() - wildcard_form_key = set() - for buffer_key in data_buffers: - form_key, attribute = parse_buffer_key(buffer_key) - if attribute in METADATA_ATTRIBUTES: - wildcard_form_key.add(form_key) - else: - paths.add(form_key_to_path[form_key]) - - # Select the most appropriate column for each wildcard - for form_key in wildcard_form_key: - # Find (DFS) any non-empty record form in any child - recursive_child_forms = ( - form_key_to_form[k] - for k in walk_graph_depth_first(form_key, form_key_to_child_form_keys) - ) - record_form_keys_with_contents = ( - f.form_key - for f in recursive_child_forms - if isinstance(f, ak.forms.RecordForm) and f.contents - ) - # Now find the deepest of such records - try: - last_record_form_key = next(record_form_keys_with_contents) - except StopIteration: - # This is a leaf! Therefore, we read this column - paths.add(form_key_to_path[form_key]) - continue - else: - # Ensure we get the "actual" last form key - for last_record_form_key in record_form_keys_with_contents: - ... - - # First see if any child is already included - for any_child_form_key in form_key_to_child_form_keys[last_record_form_key]: - any_child_path = form_key_to_path[any_child_form_key] - if any_child_path in paths: - break - # Otherwise, add the last child - else: - paths.add(any_child_path) - return frozenset({".".join(p) for p in paths if p}) - - def project( - self: S, - report: TypeTracerReport, - state: FormStructure, - ) -> ImplementsIOFunction: - if not self.use_optimization: # type: ignore[attr-defined] - return self - - return self.project_columns(self.necessary_columns(report, state)) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 4dcde735..cc792db6 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -254,9 +254,3 @@ def _recursive_replace(args, layer, parent, indices): else: args2.append(arg) return args2 - - -def _buffer_keys_for_layer( - buffer_keys: Iterable[str], known_buffer_keys: frozenset[str] -) -> set[str]: - return {k for k in buffer_keys if k in known_buffer_keys} diff --git a/src/dask_awkward/lib/testutils.py b/src/dask_awkward/lib/testutils.py index e47d5c8d..3dee6d36 100644 --- a/src/dask_awkward/lib/testutils.py +++ b/src/dask_awkward/lib/testutils.py @@ -7,7 +7,6 @@ import awkward as ak import numpy as np -from awkward.typetracer import typetracer_from_form from dask.base import is_dask_collection from packaging.version import Version From c57693cc135a10b24dba97cd624b5b328df38b20 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 4 Apr 2024 09:44:15 -0400 Subject: [PATCH 03/25] Pass reports around --- src/dask_awkward/lib/core.py | 35 ++++++++++++++++++++--------- src/dask_awkward/lib/io/columnar.py | 8 ++----- src/dask_awkward/lib/io/io.py | 8 ++++--- src/dask_awkward/lib/operations.py | 4 ++-- 4 files changed, 34 insertions(+), 21 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index d60522b1..fc675f63 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -388,7 +388,7 @@ def __reduce__(self): @property def report(self): - return getattr(self.meta, "report", set()) + return getattr(self._meta, "_report", set()) @property def dask(self) -> HighLevelGraph: @@ -402,6 +402,10 @@ def name(self) -> str: def key(self) -> Key: return (self._name, 0) + @property + def report(self): + return getattr(self._meta, "_report", set()) + def _check_meta(self, m): if isinstance(m, MaybeNone): return ak.Array(m.content) @@ -717,6 +721,8 @@ def _check_meta(self, m: Any | None) -> Any | None: def __getitem__(self, where): token = tokenize(self, where) new_name = f"{where}-{token}" + report = self.report + [_.commit(new_name) for _ in report] new_meta = self._meta[where] # first check for array type return @@ -727,6 +733,7 @@ def __getitem__(self, where): graphlayer, dependencies=[self], ) + new_meta._report = report return new_array_object(hlg, new_name, meta=new_meta, npartitions=1) # then check for scalar (or record) type @@ -737,6 +744,7 @@ def __getitem__(self, where): dependencies=[self], ) if isinstance(new_meta, ak.Record): + new_meta._report = report return new_record_object(hlg, new_name, meta=new_meta) else: return new_scalar_object(hlg, new_name, meta=new_meta) @@ -810,7 +818,7 @@ def new_record_object(dsk: HighLevelGraph, name: str, *, meta: Any) -> Record: raise TypeError( f"meta Record must have a typetracer backend, not {ak.backend(meta)}" ) - return Record(dsk, name, meta) + return out def _is_numpy_or_cupy_like(arr: Any) -> bool: @@ -943,7 +951,7 @@ def reset_meta(self) -> None: @property def report(self): - return getattr(self.meta, "report", set()) + return getattr(self._meta, "_report", set()) def repartition( self, @@ -980,6 +988,7 @@ def repartition( new_graph = HighLevelGraph.from_collections( key, new_layer, dependencies=(self,) ) + [_.commit(key) for _ in self.report] return new_array_object( new_graph, key, @@ -1181,7 +1190,7 @@ def _partitions(self, index: Any) -> Array: # otherwise nullify the known divisions else: new_divisions = (None,) * (len(new_keys) + 1) # type: ignore - + [_.commit(name) for _ in self.report] return new_array_object( graph, name, meta=self._meta, divisions=tuple(new_divisions) ) @@ -1403,6 +1412,7 @@ def _getitem_slice_on_zero(self, where): AwkwardMaterializedLayer(dask, previous_layer_names=[self.name]), dependencies=[self], ) + [_.commit(name) for _ in self.report] return new_array_object( hlg, name, @@ -1742,7 +1752,6 @@ def new_array_object( attrs: Mapping[str, Any] | None = None, npartitions: int | None = None, divisions: tuple[int, ...] | tuple[None, ...] | None = None, - report=set(), ) -> Array: """Instantiate a new Array collection object. @@ -1810,9 +1819,6 @@ def new_array_object( actual_meta.attrs = attrs out = Array(dsk, name, actual_meta, divs) - if report: - [r.commit(out.name) for r in report] - actual_meta._report = report if actual_meta.__doc__ != actual_meta.__class__.__doc__: out.__doc__ = actual_meta.__doc__ @@ -1955,6 +1961,15 @@ def _map_partitions( if meta is None: meta = map_meta(fn, *args, **kwargs) + reps = set() + for dep in to_meta(deps): + rep = getattr(dep, "_report", None) + if rep: + [_.commit(name) for _ in rep] + [reps.add(_) for _ in rep] + + meta._report = reps + hlg = HighLevelGraph.from_collections( name, lay, @@ -1976,7 +1991,6 @@ def _map_partitions( new_divisions = tuple(map(lambda x: x * output_divisions, in_divisions)) else: new_divisions = in_divisions - if output_divisions is not None: return new_array_object( hlg, @@ -2281,7 +2295,7 @@ def non_trivial_reduction( ) graph = HighLevelGraph.from_collections(name_finalize, trl, dependencies=(chunked,)) - + [_.commit(name_finalize) for _ in array.report] meta = reducer( array._meta, axis=axis, @@ -2289,6 +2303,7 @@ def non_trivial_reduction( mask_identity=mask_identity, ) if isinstance(meta, ak.highlevel.Array): + meta._report = array.report return new_array_object(graph, name_finalize, meta=meta, npartitions=1) else: return new_scalar_object(graph, name_finalize, meta=meta) diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index 7268b816..4069740c 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -7,12 +7,8 @@ from awkward import Array as AwkwardArray from awkward.forms import Form -from dask_awkward.layers.layers import ( - BackendT, - ImplementsIOFunction, - ImplementsNecessaryColumns, -) -from dask_awkward.lib.utils import METADATA_ATTRIBUTES, FormStructure +from dask_awkward.layers.layers import BackendT, ImplementsNecessaryColumns +from dask_awkward.lib.utils import FormStructure if TYPE_CHECKING: from awkward._nplikes.typetracer import TypeTracerReport diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 5dc9fc7d..46849daf 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -25,7 +25,7 @@ AwkwardTreeReductionLayer, ImplementsReport, IOFunctionWithMocking, - io_func_implements_projection, + io_func_implements_columnar, io_func_implements_report, ) from dask_awkward.lib.core import ( @@ -621,7 +621,7 @@ def from_map( ) kw = {} - if io_func_implements_projection(func): + if io_func_implements_columnar(func): # Special `io_func` implementations can do buffer projection - choosing columns # so here we start with a blank report io_func = func @@ -631,7 +631,9 @@ def from_map( behavior=io_func.behavior, buffer_key=render_buffer_key, ) - kw["report"] = {report} # column tracking report, not failure report, below + array_meta._report = { + report + } # column tracking report, not failure report, below # If we know the meta, we can spoof mocking elif meta is not None: io_func = IOFunctionWithMocking(meta, func) diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index 70811e98..2ac5720d 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -60,7 +60,7 @@ def concatenate( name = f"{label}-{token}" metas = [c._meta for c in arrays] - report = set.union(getattr(m, "_report", set()) for m in metas) + report = set.union(*(getattr(m, "_report", set()) for m in metas)) if len(metas) == 0: raise ValueError("Need at least one array to concatenate") @@ -123,12 +123,12 @@ def concatenate( aml = AwkwardMaterializedLayer(g, previous_layer_names=[arrays[0].name]) hlg = HighLevelGraph.from_collections(name, aml, dependencies=arrays) + meta_no_report._report = report return new_array_object( hlg, name, meta=meta_no_report, npartitions=sum(a.npartitions for a in arrays), - report=report, ) if axis > 0: From 0c367df5064f2da8db842e867b89efae7ad8cb01 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Fri, 5 Apr 2024 10:02:00 -0400 Subject: [PATCH 04/25] remember to commit --- src/dask_awkward/lib/io/io.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 46849daf..0298f06a 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -36,7 +36,7 @@ typetracer_array, ) from dask_awkward.lib.io.columnar import ColumnProjectionMixin -from dask_awkward.lib.utils import render_buffer_key +from dask_awkward.lib.utils import form_with_unique_keys, render_buffer_key from dask_awkward.utils import first, second if TYPE_CHECKING: @@ -626,11 +626,12 @@ def from_map( # so here we start with a blank report io_func = func array_meta, report = typetracer_with_report( - io_func.form, + form_with_unique_keys(io_func.form, "@"), highlevel=True, behavior=io_func.behavior, buffer_key=render_buffer_key, ) + report.commit(name) array_meta._report = { report } # column tracking report, not failure report, below From 9c654b1f7351fe9cbcf5ab61334fe69129f5b346 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Tue, 9 Apr 2024 16:15:31 -0400 Subject: [PATCH 05/25] first working (for parquet) --- src/dask_awkward/layers/layers.py | 14 ++++---------- src/dask_awkward/lib/io/io.py | 1 + src/dask_awkward/lib/io/parquet.py | 6 +++--- src/dask_awkward/lib/optimize.py | 26 ++++++++++++++++++++++---- 4 files changed, 30 insertions(+), 17 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index ecc208bf..8350de34 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -57,7 +57,7 @@ def return_report(self) -> bool: ... class ImplementsProjection(Protocol[T]): - def project(self, report: TypeTracerReport, state: T) -> ImplementsIOFunction: ... + def project(self, columns: list[str]) -> ImplementsIOFunction: ... class ImplementsNecessaryColumns(ImplementsProjection[T], Protocol): @@ -81,7 +81,7 @@ def __call__(self, *args, **kwargs): def io_func_implements_projection(func: ImplementsIOFunction) -> bool: - return hasattr(func, "prepare_for_projection") + return hasattr(func, "project") def io_func_implements_columnar(func: ImplementsIOFunction) -> bool: @@ -158,15 +158,9 @@ def is_projectable(self) -> bool: def is_columnar(self) -> bool: return io_func_implements_columnar(self.io_func) - def project( - self, - report: TypeTracerReport, - state: T, - ) -> AwkwardInputLayer: + def project(self, columns: list[str]) -> AwkwardInputLayer: assert self.is_projectable - io_func = cast(ImplementsProjection, self.io_func).project( - report=report, state=state - ) + io_func = self.io_func.project(columns) return AwkwardInputLayer( name=self.name, inputs=self.inputs, diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 0298f06a..1270f328 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -631,6 +631,7 @@ def from_map( behavior=io_func.behavior, buffer_key=render_buffer_key, ) + io_func._column_report = report report.commit(name) array_meta._report = { report diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index 2fc47c25..3625e1d7 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -94,7 +94,7 @@ def __init__( def __call__(self, *args, **kwargs): ... @abc.abstractmethod - def project_columns(self, columns): ... + def project(self, columns): ... @property def return_report(self) -> bool: @@ -176,7 +176,7 @@ def __call__(self, *args, **kwargs): return self.read_fn(source) - def project_columns(self, columns): + def project(self, columns): return FromParquetFileWiseFn( fs=self.fs, form=self.form.select_columns(columns), @@ -235,7 +235,7 @@ def __call__(self, pair: Any) -> ak.Array: attrs=self.attrs, ) - def project_columns(self, columns): + def project(self, columns): return FromParquetFragmentWiseFn( fs=self.fs, form=self.form.select_columns(columns), diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index cc792db6..22f87e29 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -61,8 +61,8 @@ def all_optimizations(dsk: Mapping, keys: Sequence[Key], **_: Any) -> Mapping: def optimize(dsk: HighLevelGraph, keys: Sequence[Key], **_: Any) -> Mapping: """Run optimizations specific to dask-awkward. - This is currently limited to determining the necessary columns for - input layers. + - determine the necessary columns for input layers + - fuse linear chains of blockwise operations in linear time """ if dask.config.get("awkward.optimization.enabled"): @@ -104,9 +104,27 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph New, optimized task graph with column-projected ``AwkwardInputLayer``. """ - # TBD + dsk2 = dsk.layers.copy() + for k, lay in dsk.layers.items(): + if not isinstance(lay, AwkwardInputLayer): + continue + rep = lay.io_func._column_report + cols = rep.data_touched_in(dsk.layers) + new_lay = lay.project([c.replace("@.", "") for c in cols]) + dsk2[k] = new_lay + + return HighLevelGraph(dsk2, dsk.dependencies) - return dsk # HighLevelGraph(layers, dsk.dependencies) + +def necessary_columns(dsk, keys): + out = {} + for k, lay in dsk.layers.items(): + if not isinstance(lay, AwkwardInputLayer): + continue + rep = lay.io_func._column_report + cols = rep.data_touched_in(dsk.layers) + out[k] = lay.project([c.replace("@.", "") for c in cols]) + return out def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: From f56f77a254a72bde71d15289e2c705fae2869f46 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Fri, 12 Apr 2024 10:45:29 -0400 Subject: [PATCH 06/25] stop --- src/dask_awkward/layers/layers.py | 1 + src/dask_awkward/lib/io/json.py | 10 +++++++++- src/dask_awkward/lib/optimize.py | 14 ++++++++------ 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index 8350de34..3c8a83ef 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -160,6 +160,7 @@ def is_columnar(self) -> bool: def project(self, columns: list[str]) -> AwkwardInputLayer: assert self.is_projectable + breakpoint() io_func = self.io_func.project(columns) return AwkwardInputLayer( name=self.name, diff --git a/src/dask_awkward/lib/io/json.py b/src/dask_awkward/lib/io/json.py index 9b63a493..b3ded5a2 100644 --- a/src/dask_awkward/lib/io/json.py +++ b/src/dask_awkward/lib/io/json.py @@ -74,7 +74,15 @@ def use_optimization(self) -> bool: and self.schema is None ) - def project_columns(self, columns): + def project(self, columns: list[str]): + # transform buffer names to JSON columns + columns = { + c.replace(".content", "") + .replace("-offsets", "") + .replace("-data", "") + .replace("-index", "") + for c in columns + } form = self.form.select_columns(columns) assert form is not None schema = layout_to_jsonschema(form.length_zero_array(highlevel=False)) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 22f87e29..0be5040d 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -1,8 +1,6 @@ from __future__ import annotations -import copy import logging -import warnings from collections.abc import Iterable, Mapping, Sequence from typing import TYPE_CHECKING, Any, cast @@ -11,14 +9,12 @@ from dask.blockwise import fuse_roots, optimize_blockwise from dask.core import flatten from dask.highlevelgraph import HighLevelGraph -from dask.local import get_sync from dask_awkward.layers import AwkwardBlockwiseLayer, AwkwardInputLayer from dask_awkward.lib.utils import typetracer_nochecks from dask_awkward.utils import first if TYPE_CHECKING: - from awkward._nplikes.typetracer import TypeTracerReport from dask.typing import Key log = logging.getLogger(__name__) @@ -105,8 +101,14 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph """ dsk2 = dsk.layers.copy() + + lays = {_[0] for _ in keys if isinstance(_, tuple)} + for l in lays: + 1 for k, lay in dsk.layers.items(): - if not isinstance(lay, AwkwardInputLayer): + if not isinstance(lay, AwkwardInputLayer) or not hasattr( + lay.io_func, "_column_report" + ): continue rep = lay.io_func._column_report cols = rep.data_touched_in(dsk.layers) @@ -214,7 +216,7 @@ def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelG for chain in chains: # inputs are the inputs of chain[0] # outputs are the outputs of chain[-1] - # .dsk is composed from the .dsk of each layer + # .dsk is composed of the .dsk of each layer outkey = chain[-1] layer0 = dsk.layers[chain[0]] outlayer = layers[outkey] From 75b44169986495b29cfa8020093f27a39e344b56 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Mon, 15 Apr 2024 14:32:11 -0400 Subject: [PATCH 07/25] most pass --- src/dask_awkward/__init__.py | 8 +- src/dask_awkward/layers/layers.py | 16 +-- src/dask_awkward/lib/__init__.py | 6 +- src/dask_awkward/lib/core.py | 31 +++-- src/dask_awkward/lib/inspect.py | 204 ---------------------------- src/dask_awkward/lib/io/columnar.py | 24 +--- src/dask_awkward/lib/io/io.py | 5 +- src/dask_awkward/lib/io/json.py | 1 + src/dask_awkward/lib/operations.py | 1 + src/dask_awkward/lib/optimize.py | 43 ++++-- tests/test_core.py | 6 +- tests/test_inspect.py | 62 --------- tests/test_operations.py | 22 --- 13 files changed, 66 insertions(+), 363 deletions(-) diff --git a/src/dask_awkward/__init__.py b/src/dask_awkward/__init__.py index a9726941..8e63f9b2 100644 --- a/src/dask_awkward/__init__.py +++ b/src/dask_awkward/__init__.py @@ -19,13 +19,9 @@ partition_compatibility, ) from dask_awkward.lib.describe import fields -from dask_awkward.lib.inspect import ( - report_necessary_buffers, - report_necessary_columns, - sample, -) +from dask_awkward.lib.inspect import sample -necessary_columns = report_necessary_columns # Export for backwards compatibility. +necessary_columns = None # TODO from dask_awkward.lib.io.io import ( from_awkward, diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index 3c8a83ef..d8d97acd 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -34,6 +34,7 @@ def __getstate__(self) -> dict: # Indicator that this layer has been serialised state = self.__dict__.copy() state["has_been_unpickled"] = True + state.pop("meta") # this is a typetracer return state def __repr__(self) -> str: @@ -60,12 +61,6 @@ class ImplementsProjection(Protocol[T]): def project(self, columns: list[str]) -> ImplementsIOFunction: ... -class ImplementsNecessaryColumns(ImplementsProjection[T], Protocol): - def necessary_columns( - self, report: TypeTracerReport, state: T - ) -> frozenset[str]: ... - - class IOFunctionWithMocking(ImplementsIOFunction): def __init__(self, meta: AwkwardArray, io_func: ImplementsIOFunction): self._meta = meta @@ -147,20 +142,11 @@ def __init__( def __repr__(self) -> str: return f"AwkwardInputLayer<{self.output}>" - @property - def is_projectable(self) -> bool: - # isinstance(self.io_func, ImplementsProjection) - return ( - io_func_implements_projection(self.io_func) and not self.has_been_unpickled - ) - @property def is_columnar(self) -> bool: return io_func_implements_columnar(self.io_func) def project(self, columns: list[str]) -> AwkwardInputLayer: - assert self.is_projectable - breakpoint() io_func = self.io_func.project(columns) return AwkwardInputLayer( name=self.name, diff --git a/src/dask_awkward/lib/__init__.py b/src/dask_awkward/lib/__init__.py index a3d0703f..93a1049d 100644 --- a/src/dask_awkward/lib/__init__.py +++ b/src/dask_awkward/lib/__init__.py @@ -8,11 +8,7 @@ partition_compatibility, ) from dask_awkward.lib.describe import fields -from dask_awkward.lib.inspect import ( - report_necessary_buffers, - report_necessary_columns, - sample, -) +from dask_awkward.lib.inspect import sample from dask_awkward.lib.io.io import ( from_awkward, from_dask_array, diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index fc675f63..9bee5e01 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -734,6 +734,7 @@ def __getitem__(self, where): dependencies=[self], ) new_meta._report = report + hlg.layers[new_name].meta = new_meta return new_array_object(hlg, new_name, meta=new_meta, npartitions=1) # then check for scalar (or record) type @@ -745,6 +746,7 @@ def __getitem__(self, where): ) if isinstance(new_meta, ak.Record): new_meta._report = report + hlg.layers[new_name].meta = new_meta return new_record_object(hlg, new_name, meta=new_meta) else: return new_scalar_object(hlg, new_name, meta=new_meta) @@ -1523,9 +1525,14 @@ def __getitem__(self, where): raise RuntimeError("Lists containing integers are not supported.") if isinstance(where, tuple): - return self._getitem_tuple(where) - - return self._getitem_single(where) + out = self._getitem_tuple(where) + else: + out = self._getitem_single(where) + if self.report: + [_.commit(out.name) for _ in self.report] + out._meta._report = self._meta._report + out.dask.layers[out.name].meta = out._meta + return out def _is_method_heuristic(self, resolved: Any) -> bool: return callable(resolved) @@ -1960,16 +1967,20 @@ def _map_partitions( if meta is None: meta = map_meta(fn, *args, **kwargs) + else: + # To do any touching?? + map_meta(fn, *args, **kwargs) reps = set() - for dep in to_meta(deps): - rep = getattr(dep, "_report", None) - if rep: - [_.commit(name) for _ in rep] - [reps.add(_) for _ in rep] + for dep in deps: + rep = getattr(dep, "report", None) + if isinstance(rep, set): + for _ in rep: + _.commit(name) + reps.add(_) meta._report = reps - + lay.meta = meta hlg = HighLevelGraph.from_collections( name, lay, @@ -2295,13 +2306,13 @@ def non_trivial_reduction( ) graph = HighLevelGraph.from_collections(name_finalize, trl, dependencies=(chunked,)) - [_.commit(name_finalize) for _ in array.report] meta = reducer( array._meta, axis=axis, keepdims=keepdims, mask_identity=mask_identity, ) + [_.commit(name_finalize) for _ in array.report] if isinstance(meta, ak.highlevel.Array): meta._report = array.report return new_array_object(graph, name_finalize, meta=meta, npartitions=1) diff --git a/src/dask_awkward/lib/inspect.py b/src/dask_awkward/lib/inspect.py index 3f63fc22..b0d180bf 100644 --- a/src/dask_awkward/lib/inspect.py +++ b/src/dask_awkward/lib/inspect.py @@ -13,210 +13,6 @@ from dask_awkward.lib.core import Array -class NecessaryBuffers(NamedTuple): - data_and_shape: frozenset[str] - shape_only: frozenset[str] - - -def report_necessary_buffers( - *args: Any, traverse: bool = True -) -> dict[str, NecessaryBuffers | None]: - r"""Determine the buffer keys necessary to compute a collection. - - Parameters - ---------- - *args : Dask collections or HighLevelGraphs - The collection (or collection graph) of interest. These can be - individual objects, lists, sets, or dictionaries. - traverse : bool, optional - If True (default), builtin Python collections are traversed - looking for any Dask collections they might contain. - - Returns - ------- - dict[str, NecessaryBuffers | None] - Mapping that pairs the input layers in the graph to objects - describing the data and shape buffers that have been tagged - as required by column optimisation of the given layer. - - Examples - -------- - If we have a hypothetical parquet dataset (``ds``) with the fields - - - "foo" - - "bar" - - "baz" - - And the "baz" field has fields - - - "x" - - "y" - - The calculation of ``ds.bar + ds.baz.x`` will only require the - ``bar`` and ``baz.x`` columns from the parquet file. - - >>> import dask_awkward as dak - >>> ds = dak.from_parquet("some-dataset") - >>> ds.fields - ["foo", "bar", "baz"] - >>> ds.baz.fields - ["x", "y"] - >>> x = ds.bar + ds.baz.x - >>> dak.report_necessary_buffers(x) - { - "from-parquet-abc123": NecessaryBuffers( - data_and_shape=frozenset(...), shape_only=frozenset(...) - ) - } - - """ - import dask_awkward.lib.optimize as o - - collections, _ = unpack_collections(*args, traverse=traverse) - if not collections: - return {} - - seen_names = set() - - name_to_necessary_buffers: dict[str, NecessaryBuffers | None] = {} - for obj in collections: - dsk = obj.__dask_graph__() - keys = obj.__dask_keys__() - projection_data = o._prepare_buffer_projection(dsk, keys) - - # If the projection failed, or there are no input layers - if projection_data is None: - # Ensure that we have a record of the seen layers, if they're inputs - for name, layer in dsk.items(): - if isinstance(layer, AwkwardInputLayer): - seen_names.add(name) - continue - - # Unpack projection information - layer_to_reports, _ = projection_data - for name, report in layer_to_reports.items(): - existing_buffers = name_to_necessary_buffers.setdefault( - name, NecessaryBuffers(frozenset(), frozenset()) - ) - # Compute the shape-only keys in addition to the data and shape - data_and_shape = frozenset(report.data_touched) - shape_only = frozenset(report.shape_touched) - data_and_shape - - # Update set of touched keys - assert existing_buffers is not None - name_to_necessary_buffers[name] = NecessaryBuffers( - data_and_shape=existing_buffers.data_and_shape | data_and_shape, - shape_only=existing_buffers.shape_only | shape_only, - ) - - # Populate result with names of seen layers - for k in seen_names: - name_to_necessary_buffers.setdefault(k, None) - return name_to_necessary_buffers - - -def report_necessary_columns( - *args: Any, traverse: bool = True -) -> dict[str, frozenset[str] | None]: - r"""Get columns necessary to compute a collection - - This function is specific to sources that are columnar (e.g. Parquet). - - Parameters - ---------- - *args : Dask collections or HighLevelGraphs - The collection (or collection graph) of interest. These can be - individual objects, lists, sets, or dictionaries. - traverse : bool, optional - If True (default), builtin Python collections are traversed - looking for any Dask collections they might contain. - - Returns - ------- - dict[str, frozenset[str] | None] - Mapping that pairs the input layers in the graph to the - set of necessary IO columns that have been identified by column - optimisation of the given layer. If the layer is not backed by a - columnar source, then None is returned instead of a set. - - Examples - -------- - If we have a hypothetical parquet dataset (``ds``) with the fields - - - "foo" - - "bar" - - "baz" - - And the "baz" field has fields - - - "x" - - "y" - - The calculation of ``ds.bar + ds.baz.x`` will only require the - ``bar`` and ``baz.x`` columns from the parquet file. - - >>> import dask_awkward as dak - >>> ds = dak.from_parquet("some-dataset") - >>> ds.fields - ["foo", "bar", "baz"] - >>> ds.baz.fields - ["x", "y"] - >>> x = ds.bar + ds.baz.x - >>> dak.report_necessary_columns(x) - { - "from-parquet-abc123": frozenset({"bar", "baz.x"}) - } - - """ - import dask_awkward.lib.optimize as o - - collections, _ = unpack_collections(*args, traverse=traverse) - if not collections: - return {} - - seen_names = set() - - name_to_necessary_columns: dict[str, frozenset | None] = {} - with o.typetracer_nochecks(): - for obj in collections: - dsk = obj.__dask_graph__() - keys = obj.__dask_keys__() - projection_data = o._prepare_buffer_projection(dsk, keys) - - # If the projection failed, or there are no input layers - if projection_data is None: - # Ensure that we have a record of the seen layers, if they're inputs - for name, layer in dsk.items(): - if isinstance(layer, AwkwardInputLayer): - seen_names.add(name) - continue - - # Unpack projection information - layer_to_reports, layer_to_projection_state = projection_data - for name, report in layer_to_reports.items(): - layer = dsk.layers[name] - if not (isinstance(layer, AwkwardInputLayer) and layer.is_columnar): - continue - - existing_columns = name_to_necessary_columns.setdefault( - name, frozenset() - ) - - assert existing_columns is not None - # Update set of touched keys - name_to_necessary_columns[name] = ( - existing_columns - | layer.necessary_columns( - report=report, state=layer_to_projection_state[name] - ) - ) - - # Populate result with names of seen layers - for k in seen_names: - name_to_necessary_columns.setdefault(k, None) - return name_to_necessary_columns - - def _random_boolean_like(array_like: AwkArray, probability: float) -> AwkArray: import awkward as ak diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index 4069740c..36cfc007 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -7,7 +7,7 @@ from awkward import Array as AwkwardArray from awkward.forms import Form -from dask_awkward.layers.layers import BackendT, ImplementsNecessaryColumns +from dask_awkward.layers.layers import BackendT from dask_awkward.lib.utils import FormStructure if TYPE_CHECKING: @@ -15,28 +15,8 @@ log = logging.getLogger(__name__) -T = TypeVar("T") - -class ImplementsColumnProjectionMixin(ImplementsNecessaryColumns, Protocol): - @property - def form(self) -> Form: ... - - @property - def behavior(self) -> dict | None: ... - - @property - def attrs(self) -> dict | None: ... - - def project_columns(self: T, columns: frozenset[str]) -> T: ... - - def __call__(self, *args, **kwargs): ... - - -S = TypeVar("S", bound=ImplementsColumnProjectionMixin) - - -class ColumnProjectionMixin(ImplementsNecessaryColumns[FormStructure]): +class ColumnProjectionMixin: """A mixin to add column-centric buffer projection to an IO function. Classes that inherit from this mixin are assumed to be able to read at the diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 1270f328..fbc8e862 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -425,6 +425,8 @@ def from_dask_array( concatenate=True, ) layer = AwkwardBlockwiseLayer.from_blockwise(layer) + layer.meta = meta + meta._report = set() hlg = HighLevelGraph.from_collections(name, layer, dependencies=[array]) if np.any(np.isnan(array.chunks)): return new_array_object( @@ -636,7 +638,7 @@ def from_map( array_meta._report = { report } # column tracking report, not failure report, below - # If we know the meta, we can spoof mocking + # If we know the meta, we can spoof mocking elif meta is not None: io_func = IOFunctionWithMocking(meta, func) array_meta = meta @@ -646,6 +648,7 @@ def from_map( array_meta = None dsk = AwkwardInputLayer(name=name, inputs=inputs, io_func=io_func) + dsk.meta = array_meta hlg = HighLevelGraph.from_collections(name, dsk) if divisions is not None: diff --git a/src/dask_awkward/lib/io/json.py b/src/dask_awkward/lib/io/json.py index b3ded5a2..af625ea5 100644 --- a/src/dask_awkward/lib/io/json.py +++ b/src/dask_awkward/lib/io/json.py @@ -81,6 +81,7 @@ def project(self, columns: list[str]): .replace("-offsets", "") .replace("-data", "") .replace("-index", "") + .replace("-mask", "") for c in columns } form = self.form.select_columns(columns) diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index 2ac5720d..99487693 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -124,6 +124,7 @@ def concatenate( hlg = HighLevelGraph.from_collections(name, aml, dependencies=arrays) meta_no_report._report = report + aml.meta = meta_no_report return new_array_object( hlg, name, diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 0be5040d..fa24d097 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -104,31 +104,48 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph lays = {_[0] for _ in keys if isinstance(_, tuple)} for l in lays: - 1 + if hasattr(dsk.layers[l], "meta"): + touch_data(dsk.layers[l].meta) + [_.commit("output") for _ in getattr(dsk.layers[l].meta, "_report", ())] + + # this loop is necessary_columns + all_layers = tuple(dsk.layers) + ("output",) for k, lay in dsk.layers.items(): if not isinstance(lay, AwkwardInputLayer) or not hasattr( lay.io_func, "_column_report" ): continue + all_cols = lay.meta.layout.form.columns() rep = lay.io_func._column_report - cols = rep.data_touched_in(dsk.layers) + cols = set() + for l in all_layers: + # this loop not required after next ak release + try: + cols |= set(rep.data_touched_in((l,))) + for col in rep.shape_touched_in((l,)): + if col in cols or any(_.startswith(col) for _ in cols): + # loopy loop? + continue + col2 = ( + col[2:] + .replace(".content", "") + .replace("-offsets", "") + .replace("-data", "") + .replace("-index", "") + .replace("-mask", "") + ) + ll = list(_ for _ in all_cols if _.startswith(col2)) + if ll: + cols.add("@." + ll[0]) + + except KeyError: + pass new_lay = lay.project([c.replace("@.", "") for c in cols]) dsk2[k] = new_lay return HighLevelGraph(dsk2, dsk.dependencies) -def necessary_columns(dsk, keys): - out = {} - for k, lay in dsk.layers.items(): - if not isinstance(lay, AwkwardInputLayer): - continue - rep = lay.io_func._column_report - cols = rep.data_touched_in(dsk.layers) - out[k] = lay.project([c.replace("@.", "") for c in cols]) - return out - - def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: """Smush chains of blockwise layers into a single layer. diff --git a/tests/test_core.py b/tests/test_core.py index 369cdbb4..7aced6bf 100644 --- a/tests/test_core.py +++ b/tests/test_core.py @@ -57,7 +57,7 @@ def test_dunder_str(caa: ak.Array) -> None: def test_calculate_known_divisions(ndjson_points_file: str) -> None: daa = dak.from_json([ndjson_points_file] * 3) target = (0, 5, 10, 15) - assert calculate_known_divisions(daa) == target + # assert calculate_known_divisions(daa) == target assert calculate_known_divisions(daa.points) == target assert calculate_known_divisions(daa.points.x) == target assert calculate_known_divisions(daa["points"]["y"]) == target @@ -489,8 +489,8 @@ def test_scalar_pickle(daa: Array) -> None: # TODO: workaround since dask un/pack disappeared for lay2, lay1 in zip(s2.dask.layers.values(), s1.dask.layers.values()): - if hasattr(lay1, "_meta"): - lay2._meta = lay1._meta + if hasattr(lay1, "meta"): + lay2.meta = lay1.meta assert_eq(s1.compute(), s2.compute()) assert s1.known_value is None diff --git a/tests/test_inspect.py b/tests/test_inspect.py index b4e2651b..8e58d3aa 100644 --- a/tests/test_inspect.py +++ b/tests/test_inspect.py @@ -10,68 +10,6 @@ test_uproot_path = Path(__file__).parent / "test-uproot" -def test_report_necessary_buffers( - daa: dak.Array, tmpdir_factory: pytest.TempdirFactory -) -> None: - z = daa.points.x + daa.points.y - for k, v in dak.report_necessary_buffers(z).items(): - assert v == ( - frozenset( - { - "@.points-offsets", - "@.points.content.y-data", - "@.points.content.x-data", - } - ), - frozenset(), - ) - - w = dak.to_parquet( - daa.points.x, str(Path(tmpdir_factory.mktemp("pq")) / "out"), compute=False - ) - for k, v in dak.report_necessary_buffers(w).items(): - assert v == ( - frozenset({"@.points-offsets", "@.points.content.x-data"}), - frozenset(), - ) - - q = {"z": z, "w": w} - for k, v in dak.report_necessary_buffers(q).items(): - assert v == ( - frozenset( - { - "@.points-offsets", - "@.points.content.x-data", - "@.points.content.y-data", - } - ), - frozenset(), - ) - - z = dak.zeros_like(daa.points.x) - for k, v in dak.report_necessary_buffers(z).items(): - assert v == ( - frozenset({"@.points-offsets"}), - frozenset({"@.points.content.x-data"}), - ) - - -def test_report_necessary_columns(daa: dak.Array) -> None: - result = dak.min(daa.points.x, axis=1) - rep = dak.report_necessary_columns(result) - for k, v in rep.items(): - assert v is not None - assert sorted(["points.x"]) == sorted(v) - - result = dak.zeros_like(daa.points.y) - rep = dak.report_necessary_columns(result) - for k, v in rep.items(): - assert v is not None - points, coord = list(v)[0].split(".") - assert points == "points" - assert coord in ["x", "y"] - - def test_visualize_works(daa): query = daa.points.x diff --git a/tests/test_operations.py b/tests/test_operations.py index 14a43fd9..b7cad07d 100644 --- a/tests/test_operations.py +++ b/tests/test_operations.py @@ -16,19 +16,6 @@ def test_concatenate_simple(daa, caa, axis): ) -def test_concatenate_axis_0_logical_same(daa): - result = dak.concatenate([daa, daa], axis=0) - buffers_report = dak.report_necessary_buffers(result.points.x) - assert len(buffers_report) == 1 - - buffers = next(iter(buffers_report.values())) - - assert buffers.data_and_shape == frozenset( - ["@.points.content.x-data", "@.points-offsets"] - ) - assert buffers.shape_only == frozenset() - - def test_concatenate_axis_0_logical_different(daa): import dask.config @@ -56,15 +43,6 @@ def test_concatenate_axis_0_logical_different(daa): empty_dak_array = dak.from_awkward(empty_array, npartitions=1) result = dak.concatenate([daa, empty_dak_array], axis=0) - buffers_report = dak.report_necessary_buffers(result.points.x) - assert len(buffers_report) == 2 - - buffers = next(iter(buffers_report.values())) - assert buffers.data_and_shape == frozenset( - ["@.points.content.x-data", "@.points.content.y-data", "@.points-offsets"] - ) - assert buffers.shape_only == frozenset() - @pytest.mark.parametrize("axis", [0, 1, 2]) def test_concatenate_more_axes(axis): From a5c319d4417affd76137946062b9b08305e9c7b8 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 18 Apr 2024 11:05:41 -0400 Subject: [PATCH 08/25] fix most --- src/dask_awkward/lib/core.py | 18 +++++++++++++----- src/dask_awkward/lib/io/io.py | 2 +- src/dask_awkward/lib/io/json.py | 3 +++ src/dask_awkward/lib/io/parquet.py | 3 +++ 4 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index 9bee5e01..893aeaae 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -27,6 +27,7 @@ TypeTracerArray, create_unknown_scalar, is_unknown_scalar, + touch_data, ) from dask.base import ( DaskMethodsMixin, @@ -1965,11 +1966,18 @@ def _map_partitions( **kwargs, ) - if meta is None: - meta = map_meta(fn, *args, **kwargs) - else: - # To do any touching?? - map_meta(fn, *args, **kwargs) + try: + if meta is None: + meta = map_meta(fn, *args, **kwargs) + else: + # To do any touching?? + map_meta(fn, *args, **kwargs) + except (AssertionError, TypeError, NotImplementedError): + [ + touch_data(_) + for _ in args + if isinstance(_, ak.Array) and ak.backend(_) == "typetracer" + ] reps = set() for dep in deps: diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index fbc8e862..4e99d76b 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -648,7 +648,6 @@ def from_map( array_meta = None dsk = AwkwardInputLayer(name=name, inputs=inputs, io_func=io_func) - dsk.meta = array_meta hlg = HighLevelGraph.from_collections(name, dsk) if divisions is not None: @@ -657,6 +656,7 @@ def from_map( result = new_array_object( hlg, name, meta=array_meta, npartitions=len(inputs), **kw ) + dsk.meta = result._meta if io_func_implements_report(io_func): if cast(ImplementsReport, io_func).return_report: diff --git a/src/dask_awkward/lib/io/json.py b/src/dask_awkward/lib/io/json.py index af625ea5..52849f8a 100644 --- a/src/dask_awkward/lib/io/json.py +++ b/src/dask_awkward/lib/io/json.py @@ -9,6 +9,7 @@ import awkward as ak import dask from awkward.forms.form import Form +from awkward.typetracer import touch_data from dask.base import tokenize from dask.blockwise import BlockIndex from dask.core import flatten @@ -778,6 +779,8 @@ def to_json( map_res.dask.layers[map_res.name].annotations = {"ak_output": True} name = f"to-json-{tokenize(array, path)}" dsk = {(name, 0): (lambda *_: None, map_res.__dask_keys__())} + touch_data(array._meta) + [_.commit(name) for _ in array.report] graph = HighLevelGraph.from_collections( name, AwkwardMaterializedLayer(dsk, previous_layer_names=[map_res.name]), diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index 3625e1d7..89774142 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -12,6 +12,7 @@ import awkward.operations.ak_from_parquet as ak_from_parquet import dask from awkward.forms.form import Form +from awkward.typetracer import touch_data from dask.base import tokenize from dask.blockwise import BlockIndex from dask.highlevelgraph import HighLevelGraph @@ -689,6 +690,8 @@ def to_parquet( AwkwardMaterializedLayer(dsk, previous_layer_names=[map_res.name]), dependencies=[map_res], ) + touch_data(array._meta) + [_.commit(name) for _ in array.report] out = new_scalar_object(graph, final_name, dtype="f8") if compute: out.compute() From 243fbc1d84ee22eedb977c0315e93aafc31ff1e0 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 18 Apr 2024 21:35:53 -0400 Subject: [PATCH 09/25] probably better --- pyproject.toml | 4 ++-- src/dask_awkward/layers/layers.py | 6 ++---- src/dask_awkward/lib/core.py | 21 ++++++--------------- src/dask_awkward/lib/io/io.py | 7 +++---- src/dask_awkward/lib/optimize.py | 24 ++++++++++++++---------- tests/test_io.py | 8 ++++---- 6 files changed, 31 insertions(+), 39 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 56ab6f1e..2409cd51 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -164,5 +164,5 @@ omit = [ source = ["src/"] [tool.ruff] -ignore = ["E501", "E402"] -per-file-ignores = {"__init__.py" = ["E402", "F401"]} +lint.ignore = ["E501", "E402"] +lint.per-file-ignores = {"__init__.py" = ["E402", "F401"]} diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index d8d97acd..6ede5e08 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -1,8 +1,7 @@ from __future__ import annotations -import copy from collections.abc import Callable, Mapping -from typing import TYPE_CHECKING, Any, Literal, Protocol, TypeVar, Union, cast +from typing import TYPE_CHECKING, Any, Literal, Protocol, TypeVar, Union from dask.blockwise import Blockwise, BlockwiseDepDict, blockwise_token from dask.highlevelgraph import MaterializedLayer @@ -13,7 +12,6 @@ if TYPE_CHECKING: from awkward import Array as AwkwardArray - from awkward._nplikes.typetracer import TypeTracerReport BackendT: TypeAlias = Union[Literal["cpu"], Literal["jax"], Literal["cuda"]] @@ -34,7 +32,7 @@ def __getstate__(self) -> dict: # Indicator that this layer has been serialised state = self.__dict__.copy() state["has_been_unpickled"] = True - state.pop("meta") # this is a typetracer + state.pop("meta", None) # this is a typetracer return state def __repr__(self) -> str: diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index 893aeaae..fcfea610 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -387,10 +387,6 @@ def _rebuild(self, dsk, *, rename=None): def __reduce__(self): return (Scalar, (self.dask, self.name, None, self.dtype, self.known_value)) - @property - def report(self): - return getattr(self._meta, "_report", set()) - @property def dask(self) -> HighLevelGraph: return self._dask @@ -1973,19 +1969,14 @@ def _map_partitions( # To do any touching?? map_meta(fn, *args, **kwargs) except (AssertionError, TypeError, NotImplementedError): - [ - touch_data(_) - for _ in args - if isinstance(_, ak.Array) and ak.backend(_) == "typetracer" - ] + [touch_data(_._meta) for _ in dak_arrays] reps = set() - for dep in deps: - rep = getattr(dep, "report", None) - if isinstance(rep, set): - for _ in rep: - _.commit(name) - reps.add(_) + for dep in dak_arrays: + for rep in dep.report: + if rep not in reps: + rep.commit(name) + reps.add(rep) meta._report = reps lay.meta = meta diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 4e99d76b..f9f5c870 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -24,8 +24,7 @@ AwkwardMaterializedLayer, AwkwardTreeReductionLayer, ImplementsReport, - IOFunctionWithMocking, - io_func_implements_columnar, + io_func_implements_projection, io_func_implements_report, ) from dask_awkward.lib.core import ( @@ -623,7 +622,7 @@ def from_map( ) kw = {} - if io_func_implements_columnar(func): + if io_func_implements_projection(func): # Special `io_func` implementations can do buffer projection - choosing columns # so here we start with a blank report io_func = func @@ -640,7 +639,7 @@ def from_map( } # column tracking report, not failure report, below # If we know the meta, we can spoof mocking elif meta is not None: - io_func = IOFunctionWithMocking(meta, func) + io_func = func array_meta = meta # Without `meta`, the meta will be computed by executing the graph else: diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index fa24d097..60d40d0e 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -1,11 +1,12 @@ from __future__ import annotations import logging -from collections.abc import Iterable, Mapping, Sequence -from typing import TYPE_CHECKING, Any, cast +from collections.abc import Mapping, Sequence +from typing import TYPE_CHECKING, Any import dask.config from awkward.typetracer import touch_data +from dask.base import tokenize from dask.blockwise import fuse_roots, optimize_blockwise from dask.core import flatten from dask.highlevelgraph import HighLevelGraph @@ -103,13 +104,16 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph dsk2 = dsk.layers.copy() lays = {_[0] for _ in keys if isinstance(_, tuple)} - for l in lays: - if hasattr(dsk.layers[l], "meta"): - touch_data(dsk.layers[l].meta) - [_.commit("output") for _ in getattr(dsk.layers[l].meta, "_report", ())] + all_reps = set() + for ln in lays: + if hasattr(dsk.layers[ln], "meta"): + touch_data(dsk.layers[ln].meta) + all_reps.update(getattr(dsk.layers[ln].meta, "_report", ())) + name = tokenize("output", lays) + [_.commit(name) for _ in all_reps] # this loop is necessary_columns - all_layers = tuple(dsk.layers) + ("output",) + all_layers = tuple(dsk.layers) + (name,) for k, lay in dsk.layers.items(): if not isinstance(lay, AwkwardInputLayer) or not hasattr( lay.io_func, "_column_report" @@ -118,11 +122,11 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph all_cols = lay.meta.layout.form.columns() rep = lay.io_func._column_report cols = set() - for l in all_layers: + for ln in all_layers: # this loop not required after next ak release try: - cols |= set(rep.data_touched_in((l,))) - for col in rep.shape_touched_in((l,)): + cols |= set(rep.data_touched_in((ln,))) + for col in rep.shape_touched_in((ln,)): if col in cols or any(_.startswith(col) for _ in cols): # loopy loop? continue diff --git a/tests/test_io.py b/tests/test_io.py index fa1eac06..396f409d 100644 --- a/tests/test_io.py +++ b/tests/test_io.py @@ -360,14 +360,14 @@ def test_from_map_random_fail_from_lists(): many, meta=typetracer_array(ak.Array(many[0])), divisions=divs, - label="from-lists", + label="from-lists01", ) assert len(array.compute()) < (len(single) * len(many)) computed_report = report.compute() # we expect the 'args' field in the report to be empty if the - # from_map node succeded; so we use ak.num(..., axis=1) to filter + # from_map node succeeded; so we use ak.num(..., axis=1) to filter # those out. succ = ak.num(computed_report["args"], axis=1) == 0 fail = np.invert(succ) @@ -382,7 +382,7 @@ def test_from_map_random_fail_from_lists(): many, meta=typetracer_array(ak.Array(many[0])), divisions=divs, - label="from-lists", + label="from-lists02", ) array.compute() @@ -392,7 +392,7 @@ def test_from_map_random_fail_from_lists(): many, meta=typetracer_array(ak.Array(many[0])), divisions=divs, - label="from-lists", + label="from-lists03", ) cast(dak.Array, array).compute() From 6eebf8720b6d6a505b14851bd6e286b49a5639ca Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Mon, 22 Apr 2024 15:25:05 -0400 Subject: [PATCH 10/25] Reinstate necessary_columns (needs doc) --- src/dask_awkward/__init__.py | 4 +- src/dask_awkward/lib/__init__.py | 1 + src/dask_awkward/lib/core.py | 2 +- src/dask_awkward/lib/optimize.py | 67 +++++++++++++++++++++++--------- 4 files changed, 51 insertions(+), 23 deletions(-) diff --git a/src/dask_awkward/__init__.py b/src/dask_awkward/__init__.py index 8e63f9b2..82af89a8 100644 --- a/src/dask_awkward/__init__.py +++ b/src/dask_awkward/__init__.py @@ -20,9 +20,6 @@ ) from dask_awkward.lib.describe import fields from dask_awkward.lib.inspect import sample - -necessary_columns = None # TODO - from dask_awkward.lib.io.io import ( from_awkward, from_dask_array, @@ -38,6 +35,7 @@ from dask_awkward.lib.io.parquet import from_parquet, to_parquet from dask_awkward.lib.io.text import from_text from dask_awkward.lib.operations import concatenate +from dask_awkward.lib.optimize import necessary_columns from dask_awkward.lib.reducers import ( all, any, diff --git a/src/dask_awkward/lib/__init__.py b/src/dask_awkward/lib/__init__.py index 93a1049d..d827ae87 100644 --- a/src/dask_awkward/lib/__init__.py +++ b/src/dask_awkward/lib/__init__.py @@ -24,6 +24,7 @@ from dask_awkward.lib.io.parquet import from_parquet, to_parquet from dask_awkward.lib.io.text import from_text from dask_awkward.lib.operations import concatenate +from dask_awkward.lib.optimize import necessary_columns from dask_awkward.lib.reducers import ( all, any, diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index fcfea610..d4506015 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -719,8 +719,8 @@ def __getitem__(self, where): token = tokenize(self, where) new_name = f"{where}-{token}" report = self.report - [_.commit(new_name) for _ in report] new_meta = self._meta[where] + [_.commit(new_name) for _ in report] # first check for array type return if isinstance(new_meta, ak.Array): diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 60d40d0e..8d6e0beb 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -111,10 +111,30 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph all_reps.update(getattr(dsk.layers[ln].meta, "_report", ())) name = tokenize("output", lays) [_.commit(name) for _ in all_reps] + all_layers = tuple(dsk.layers) + (name,) + + for k, lay, cols in _optimize_columns(dsk.layers, all_layers): + new_lay = lay.project(cols) + dsk2[k] = new_lay + + return HighLevelGraph(dsk2, dsk.dependencies) + + +def _buf_to_col(s): + return ( + s[2:] + .replace(".content", "") + .replace("-offsets", "") + .replace("-data", "") + .replace("-index", "") + .replace("-mask", "") + ) + + +def _optimize_columns(dsk, all_layers): # this loop is necessary_columns - all_layers = tuple(dsk.layers) + (name,) - for k, lay in dsk.layers.items(): + for k, lay in dsk.copy().items(): if not isinstance(lay, AwkwardInputLayer) or not hasattr( lay.io_func, "_column_report" ): @@ -125,29 +145,38 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph for ln in all_layers: # this loop not required after next ak release try: - cols |= set(rep.data_touched_in((ln,))) - for col in rep.shape_touched_in((ln,)): - if col in cols or any(_.startswith(col) for _ in cols): - # loopy loop? + cols |= {_buf_to_col(s) for s in rep.data_touched_in((ln,))} + for col in (_buf_to_col(s) for s in rep.shape_touched_in((ln,))): + if col in cols: + continue + if any(_.startswith(col) for _ in cols): continue - col2 = ( - col[2:] - .replace(".content", "") - .replace("-offsets", "") - .replace("-data", "") - .replace("-index", "") - .replace("-mask", "") - ) - ll = list(_ for _ in all_cols if _.startswith(col2)) + ll = list(_ for _ in all_cols if _.startswith(col)) if ll: - cols.add("@." + ll[0]) + cols.add(ll[0]) except KeyError: pass - new_lay = lay.project([c.replace("@.", "") for c in cols]) - dsk2[k] = new_lay + yield k, lay, cols - return HighLevelGraph(dsk2, dsk.dependencies) + +def necessary_columns(*args): + dsk = {} + all_reps = set() + all_layers = set() + for arg in args: + dsk.update(arg.dask.layers) + all_layers.add(arg.name) + touch_data(arg._meta) + all_reps.update(getattr(arg.dask.layers[arg.name].meta, "_report", ())) + name = tokenize("output", args) + [_.commit(name) for _ in all_reps] + all_layers = tuple(all_layers) + (name,) + + out = {} + for k, _, cols in _optimize_columns(dsk, all_layers): + out[k] = cols + return out def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: From 43b2e43112fcdb7e1c998b2b46ee22ab669c9003 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 25 Apr 2024 11:19:38 -0400 Subject: [PATCH 11/25] pass buffer names around, not columns --- src/dask_awkward/layers/layers.py | 2 +- src/dask_awkward/lib/core.py | 5 +++- src/dask_awkward/lib/io/columnar.py | 16 +++++-------- src/dask_awkward/lib/io/io.py | 33 +++++++++++++++++--------- src/dask_awkward/lib/io/json.py | 13 +++-------- src/dask_awkward/lib/io/parquet.py | 7 ++++-- src/dask_awkward/lib/optimize.py | 36 ++++++++++++----------------- src/dask_awkward/lib/structure.py | 1 + src/dask_awkward/lib/utils.py | 12 ++++++++++ tests/test_io.py | 6 ++--- 10 files changed, 72 insertions(+), 59 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index 6ede5e08..a14bd173 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -29,8 +29,8 @@ def from_blockwise(cls, layer: Blockwise) -> AwkwardBlockwiseLayer: return ob def __getstate__(self) -> dict: - # Indicator that this layer has been serialised state = self.__dict__.copy() + # Indicator that this layer has been serialised state["has_been_unpickled"] = True state.pop("meta", None) # this is a typetracer return state diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index d4506015..b678d202 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -523,6 +523,7 @@ def f(self, other): meta = op(self._meta, other._meta) else: meta = op(self._meta, other) + [_.commit(name) for _ in self.report] return new_scalar_object(graph, name, meta=meta) return f @@ -1173,11 +1174,13 @@ def _partitions(self, index: Any) -> Array: name = f"partitions-{token}" new_keys = self.keys_array[index].tolist() dsk = {(name, i): tuple(key) for i, key in enumerate(new_keys)} + layer = AwkwardMaterializedLayer(dsk, previous_layer_names=[self.name]) graph = HighLevelGraph.from_collections( name, - AwkwardMaterializedLayer(dsk, previous_layer_names=[self.name]), + layer, dependencies=(self,), ) + layer.meta = self._meta # if a single partition was requested we trivially know the new divisions. if len(raw) == 1 and isinstance(raw[0], int) and self.known_divisions: diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index 36cfc007..ff449e8c 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -1,17 +1,10 @@ from __future__ import annotations import logging -from typing import TYPE_CHECKING, Protocol, TypeVar, cast import awkward as ak -from awkward import Array as AwkwardArray -from awkward.forms import Form from dask_awkward.layers.layers import BackendT -from dask_awkward.lib.utils import FormStructure - -if TYPE_CHECKING: - from awkward._nplikes.typetracer import TypeTracerReport log = logging.getLogger(__name__) @@ -25,10 +18,13 @@ class ColumnProjectionMixin: when only metadata buffers are required. """ - def mock_empty(self: S, backend: BackendT = "cpu") -> AwkwardArray: + def project(self, *args, **kwargs): + # default implementation does nothing + return self + + def mock_empty(self, backend: BackendT = "cpu"): # used by failure report generation - return cast( - AwkwardArray, + return ( ak.to_backend( self.form.length_zero_array(highlevel=False, behavior=self.behavior), backend, diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index f9f5c870..7c0d68f2 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -132,10 +132,13 @@ def from_awkward( ) -class _FromListsFn: - def __init__(self, behavior: Mapping | None, attrs: Mapping[str, Any] | None): +class _FromListsFn(ColumnProjectionMixin): + def __init__( + self, behavior: Mapping | None, attrs: Mapping[str, Any] | None, form=None + ): self.behavior = behavior self.attrs = attrs + self.form = form def __call__(self, x: list) -> ak.Array: return ak.Array(x, behavior=self.behavior, attrs=self.attrs) @@ -177,12 +180,13 @@ def from_lists( """ lists = list(source) divs = (0, *np.cumsum(list(map(len, lists)))) + meta = typetracer_array(ak.Array(lists[0], attrs=attrs, behavior=behavior)) return cast( Array, from_map( - _FromListsFn(behavior=behavior, attrs=attrs), + _FromListsFn(behavior=behavior, attrs=attrs, form=meta.layout.form), lists, - meta=typetracer_array(ak.Array(lists[0], attrs=attrs, behavior=behavior)), + meta=meta, divisions=divs, label="from-lists", ), @@ -425,7 +429,7 @@ def from_dask_array( ) layer = AwkwardBlockwiseLayer.from_blockwise(layer) layer.meta = meta - meta._report = set() + meta._report = set() # just because we can't project, we shouldn't track? hlg = HighLevelGraph.from_collections(name, layer, dependencies=[array]) if np.any(np.isnan(array.chunks)): return new_array_object( @@ -634,14 +638,21 @@ def from_map( ) io_func._column_report = report report.commit(name) - array_meta._report = { - report - } # column tracking report, not failure report, below - # If we know the meta, we can spoof mocking + # column tracking report, not failure report, below + array_meta._report = {report} + # Without `meta`, the meta will be computed by executing the graph elif meta is not None: + # we can still track necessary columns even if we can't project io_func = func - array_meta = meta - # Without `meta`, the meta will be computed by executing the graph + array_meta, report = typetracer_with_report( + form_with_unique_keys(meta.layout.form, "@"), + highlevel=True, + behavior=None, + buffer_key=render_buffer_key, + ) + report.commit(name) + # column tracking report, not failure report, below + array_meta._report = {report} else: io_func = func array_meta = None diff --git a/src/dask_awkward/lib/io/json.py b/src/dask_awkward/lib/io/json.py index 52849f8a..f289f3de 100644 --- a/src/dask_awkward/lib/io/json.py +++ b/src/dask_awkward/lib/io/json.py @@ -32,6 +32,7 @@ _BytesReadingInstructions, from_map, ) +from dask_awkward.lib.utils import _buf_to_col if TYPE_CHECKING: from awkward.contents.content import Content @@ -76,16 +77,8 @@ def use_optimization(self) -> bool: ) def project(self, columns: list[str]): - # transform buffer names to JSON columns - columns = { - c.replace(".content", "") - .replace("-offsets", "") - .replace("-data", "") - .replace("-index", "") - .replace("-mask", "") - for c in columns - } - form = self.form.select_columns(columns) + cols = [_buf_to_col(s) for s in columns] + form = self.form.select_columns(cols) assert form is not None schema = layout_to_jsonschema(form.length_zero_array(highlevel=False)) diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index 89774142..6863a0f8 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -24,6 +24,7 @@ from dask_awkward.lib.io.columnar import ColumnProjectionMixin from dask_awkward.lib.io.io import from_map from dask_awkward.lib.unproject_layout import unproject_layout +from dask_awkward.lib.utils import _buf_to_col if TYPE_CHECKING: pass @@ -178,9 +179,10 @@ def __call__(self, *args, **kwargs): return self.read_fn(source) def project(self, columns): + cols = [_buf_to_col(s) for s in columns] return FromParquetFileWiseFn( fs=self.fs, - form=self.form.select_columns(columns), + form=self.form.select_columns(cols), listsep=self.listsep, unnamed_root=self.unnamed_root, original_form=self.form, @@ -237,9 +239,10 @@ def __call__(self, pair: Any) -> ak.Array: ) def project(self, columns): + cols = [_buf_to_col(s) for s in columns] return FromParquetFragmentWiseFn( fs=self.fs, - form=self.form.select_columns(columns), + form=self.form.select_columns(cols), unnamed_root=self.unnamed_root, original_form=self.form, report=self.report, diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 8d6e0beb..95bb7c7e 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -12,7 +12,7 @@ from dask.highlevelgraph import HighLevelGraph from dask_awkward.layers import AwkwardBlockwiseLayer, AwkwardInputLayer -from dask_awkward.lib.utils import typetracer_nochecks +from dask_awkward.lib.utils import _buf_to_col, typetracer_nochecks from dask_awkward.utils import first if TYPE_CHECKING: @@ -106,9 +106,12 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph lays = {_[0] for _ in keys if isinstance(_, tuple)} all_reps = set() for ln in lays: - if hasattr(dsk.layers[ln], "meta"): + if ln in dsk.layers and hasattr(dsk.layers[ln], "meta"): touch_data(dsk.layers[ln].meta) all_reps.update(getattr(dsk.layers[ln].meta, "_report", ())) + print() + print(ln) + print(getattr(dsk.layers[ln].meta, "_report", ()), all_reps) name = tokenize("output", lays) [_.commit(name) for _ in all_reps] all_layers = tuple(dsk.layers) + (name,) @@ -120,17 +123,6 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph return HighLevelGraph(dsk2, dsk.dependencies) -def _buf_to_col(s): - return ( - s[2:] - .replace(".content", "") - .replace("-offsets", "") - .replace("-data", "") - .replace("-index", "") - .replace("-mask", "") - ) - - def _optimize_columns(dsk, all_layers): # this loop is necessary_columns @@ -139,21 +131,21 @@ def _optimize_columns(dsk, all_layers): lay.io_func, "_column_report" ): continue - all_cols = lay.meta.layout.form.columns() rep = lay.io_func._column_report cols = set() + # this loop not required after next ak release for ln in all_layers: - # this loop not required after next ak release try: - cols |= {_buf_to_col(s) for s in rep.data_touched_in((ln,))} - for col in (_buf_to_col(s) for s in rep.shape_touched_in((ln,))): + cols.update(rep.data_touched_in((ln,))) + except KeyError: + pass + try: + for col in rep.shape_touched_in((ln,)): if col in cols: continue if any(_.startswith(col) for _ in cols): continue - ll = list(_ for _ in all_cols if _.startswith(col)) - if ll: - cols.add(ll[0]) + cols.add(col) except KeyError: pass @@ -175,7 +167,9 @@ def necessary_columns(*args): out = {} for k, _, cols in _optimize_columns(dsk, all_layers): - out[k] = cols + first = (_buf_to_col(s) for s in cols) + # remove root "offsets", which appears when computing divisions + out[k] = sorted(s for s in first if s and s != "offsets") return out diff --git a/src/dask_awkward/lib/structure.py b/src/dask_awkward/lib/structure.py index 6b9fafad..6771e1de 100644 --- a/src/dask_awkward/lib/structure.py +++ b/src/dask_awkward/lib/structure.py @@ -336,6 +336,7 @@ def copy(array: Array) -> Array: # dask-awkward's copy is metadata-only old_meta = array._meta new_meta = ak.Array(old_meta.layout, behavior=deepcopy(old_meta._behavior)) + new_meta._report = old_meta._report return Array( array._dask, diff --git a/src/dask_awkward/lib/utils.py b/src/dask_awkward/lib/utils.py index 7b067386..c8686faf 100644 --- a/src/dask_awkward/lib/utils.py +++ b/src/dask_awkward/lib/utils.py @@ -178,3 +178,15 @@ def typetracer_nochecks(): TypeTracerArray.runtime_typechecks = oldval else: del TypeTracerArray.runtime_typechecks + + +def _buf_to_col(s): + return ( + s[2:] + .replace("content.", "") + .replace("-offsets", "") + .replace("-data", "") + .replace("-index", "") + .replace("-mask", "") + .replace("-tags", "") + ) diff --git a/tests/test_io.py b/tests/test_io.py index 396f409d..9c289ff7 100644 --- a/tests/test_io.py +++ b/tests/test_io.py @@ -434,7 +434,7 @@ def point_abs(self): caa = ak.with_name(caa.points, name="Point", behavior=behavior) daa = dak.from_awkward(caa, npartitions=2, behavior=behavior) assert_eq(caa.xsq, daa.xsq) - assert set(first(dak.necessary_columns(daa.xsq).items())[1]) == {"x"} - assert set(first(dak.necessary_columns(daa).items())[1]) == {"x", "y"} - assert set(first(dak.necessary_columns(np.abs(daa)).items())[1]) == {"x", "y"} + assert set(first(dak.necessary_columns(daa.xsq).values())) == {"x"} + assert set(first(dak.necessary_columns(daa).values())) == {"x", "y"} + assert set(first(dak.necessary_columns(np.abs(daa)).values())) == {"x", "y"} assert_eq(np.abs(caa), np.abs(daa)) From e5828fb70e6b26bfaefa96df844e470194e09b8d Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Mon, 13 May 2024 14:39:48 -0400 Subject: [PATCH 12/25] Clear cache between tests --- tests/conftest.py | 5 +++++ tests/test_io_json.py | 16 ++-------------- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 142980de..97785744 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -11,6 +11,11 @@ import dask_awkward.lib.testutils as daktu +@pytest.fixture(autouse=True) +def clear_cache(): + dak.lib.core.dak_cache.clear() + + @pytest.fixture(scope="session") def single_record_file(tmp_path_factory: pytest.TempPathFactory) -> str: fname = tmp_path_factory.mktemp("data") / "single_record.json" diff --git a/tests/test_io_json.py b/tests/test_io_json.py index 71f835bb..d6f813d5 100644 --- a/tests/test_io_json.py +++ b/tests/test_io_json.py @@ -91,26 +91,14 @@ def input_layer_array_partition0(collection: Array) -> ak.Array: """ with dask.config.set({"awkward.optimization.which": ["columns"]}): - optimized_hlg = dak_optimize(collection.dask, collection.keys) # type: ignore - layers = list(optimized_hlg.layers) # type: ignore + optimized_hlg = dak_optimize(collection.dask, collection.keys) + layers = list(optimized_hlg.layers) layer_name = [name for name in layers if name.startswith("from-json")][0] sgc, arg = optimized_hlg[(layer_name, 0)] array = sgc.dsk[layer_name][0](arg) return array -def test_json_column_projection_off(json_data_dir: Path) -> None: - source = os.path.join(str(json_data_dir), "*.json") - ds = dak.from_json(source) - fields_to_keep = ["name", "goals"] - - ds2 = ds[fields_to_keep] - with dask.config.set({"awkward.optimization.columns-opt-formats": []}): - array = input_layer_array_partition0(ds2) - - assert array.fields == ["name", "team", "goals"] - - def test_json_column_projection1(json_data_dir: Path) -> None: source = os.path.join(str(json_data_dir), "*.json") ds = dak.from_json(source) From b593f878987f5c4715cea1c60c561d5fd4e7703b Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Tue, 21 May 2024 16:43:14 -0400 Subject: [PATCH 13/25] Another one squashed --- src/dask_awkward/lib/core.py | 1 + src/dask_awkward/lib/io/io.py | 3 ++- src/dask_awkward/lib/optimize.py | 17 ++++++++--------- src/dask_awkward/lib/structure.py | 5 ++++- tests/test_optimize.py | 7 ++++--- 5 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index b678d202..e0954246 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -2314,6 +2314,7 @@ def non_trivial_reduction( keepdims=keepdims, mask_identity=mask_identity, ) + trl.meta = meta [_.commit(name_finalize) for _ in array.report] if isinstance(meta, ak.highlevel.Array): meta._report = array.report diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 7c0d68f2..cc10ced8 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -704,11 +704,12 @@ def from_map( rep_graph = HighLevelGraph.from_collections( rep_trl_name, rep_trl, dependencies=[rep_part] ) + rep_trl.meta = empty_typetracer() rep = new_array_object( rep_graph, rep_trl_name, - meta=empty_typetracer(), + meta=rep_trl.meta, npartitions=len(rep_trl.output_partitions), ) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 95bb7c7e..1170613f 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -109,9 +109,6 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph if ln in dsk.layers and hasattr(dsk.layers[ln], "meta"): touch_data(dsk.layers[ln].meta) all_reps.update(getattr(dsk.layers[ln].meta, "_report", ())) - print() - print(ln) - print(getattr(dsk.layers[ln].meta, "_report", ()), all_reps) name = tokenize("output", lays) [_.commit(name) for _ in all_reps] all_layers = tuple(dsk.layers) + (name,) @@ -141,10 +138,6 @@ def _optimize_columns(dsk, all_layers): pass try: for col in rep.shape_touched_in((ln,)): - if col in cols: - continue - if any(_.startswith(col) for _ in cols): - continue cols.add(col) except KeyError: @@ -158,7 +151,7 @@ def necessary_columns(*args): all_layers = set() for arg in args: dsk.update(arg.dask.layers) - all_layers.add(arg.name) + all_layers.update(arg.dask.layers) touch_data(arg._meta) all_reps.update(getattr(arg.dask.layers[arg.name].meta, "_report", ())) name = tokenize("output", args) @@ -169,7 +162,13 @@ def necessary_columns(*args): for k, _, cols in _optimize_columns(dsk, all_layers): first = (_buf_to_col(s) for s in cols) # remove root "offsets", which appears when computing divisions - out[k] = sorted(s for s in first if s and s != "offsets") + second = {s for s in first if s and s != "offsets"} + # remove columns that have sub-columns also in the set + for c in second.copy(): + if "." in c: + parent = c.rsplit(".", 1)[0] + second.discard(parent) + out[k] = sorted(second) return out diff --git a/src/dask_awkward/lib/structure.py b/src/dask_awkward/lib/structure.py index 6771e1de..fdf44ed8 100644 --- a/src/dask_awkward/lib/structure.py +++ b/src/dask_awkward/lib/structure.py @@ -937,8 +937,11 @@ def _array_with_rebuilt_meta( behavior = array._meta.behavior new_meta = ak.Array(array._meta, behavior=behavior, attrs=attrs) + new_meta._report = array.report + out = Array(array.dask, array.name, new_meta, array.divisions) + [_.commit(out.name) for _ in array.report] - return Array(array.dask, array.name, new_meta, array.divisions) + return out @borrow_docstring(ak.unzip) diff --git a/tests/test_optimize.py b/tests/test_optimize.py index 7f67d836..52f082b8 100644 --- a/tests/test_optimize.py +++ b/tests/test_optimize.py @@ -143,11 +143,12 @@ def something(j, k): return j.x + k.x a_compute = something(a, b) + print() nc1 = dak.necessary_columns(a_compute) - assert sorted(list(nc1.items())[0][1]) == ["points.x"] + assert list(nc1.values())[0] == ["points.x"] - nc2 = dak.necessary_columns(a_compute, (a, b)) - assert sorted(list(nc2.items())[0][1]) == ["points.x", "points.y", "points.z"] + nc2 = dak.necessary_columns(a_compute, a, b) + assert list(nc2.items())[0][1] == ["points.x", "points.y", "points.z"] x, (y, z) = dask.compute(a_compute, (a, b)) assert str(x) From e410b61e36638e53452bdc5b674e1dcac4c9199e Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Wed, 22 May 2024 11:10:00 -0400 Subject: [PATCH 14/25] Squash errors that only show when uproot.dask and hist.dask are installed (except the uproot test itself) --- src/dask_awkward/lib/optimize.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 1170613f..9a75236a 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -4,6 +4,7 @@ from collections.abc import Mapping, Sequence from typing import TYPE_CHECKING, Any +import awkward as ak import dask.config from awkward.typetracer import touch_data from dask.base import tokenize @@ -107,8 +108,14 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph all_reps = set() for ln in lays: if ln in dsk.layers and hasattr(dsk.layers[ln], "meta"): - touch_data(dsk.layers[ln].meta) - all_reps.update(getattr(dsk.layers[ln].meta, "_report", ())) + m = dsk.layers[ln].meta + if not isinstance(m, ak._nplikes.typetracer.MaybeNone): + # maybenone cases should already have been all touched + # but we could extract the .content here + touch_data(m) + rep = getattr(dsk.layers[ln].meta, "_report", ()) + if rep: + all_reps.update(rep) name = tokenize("output", lays) [_.commit(name) for _ in all_reps] all_layers = tuple(dsk.layers) + (name,) @@ -153,7 +160,9 @@ def necessary_columns(*args): dsk.update(arg.dask.layers) all_layers.update(arg.dask.layers) touch_data(arg._meta) - all_reps.update(getattr(arg.dask.layers[arg.name].meta, "_report", ())) + rep = getattr(arg.dask.layers[arg.name].meta, "_report", ()) + if rep: + all_reps.update(rep) name = tokenize("output", args) [_.commit(name) for _ in all_reps] all_layers = tuple(all_layers) + (name,) From cd537e2385fd2eb5d541e009f38d3e7c37542ab2 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 23 May 2024 14:29:31 -0400 Subject: [PATCH 15/25] fix uproot --- src/dask_awkward/lib/core.py | 4 ++++ src/dask_awkward/lib/optimize.py | 15 ++++++--------- tests/test_inspect.py | 4 ++-- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index e0954246..601470c2 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -1859,10 +1859,12 @@ def partitionwise_layer( """ pairs: list[Any] = [] numblocks: dict[str, tuple[int, ...]] = {} + reps = set() for arg in args: if isinstance(arg, Array): pairs.extend([arg.name, "i"]) numblocks[arg.name] = (arg.npartitions,) + reps.update(arg.report) elif isinstance(arg, BlockwiseDep): if len(arg.numblocks) == 1: pairs.extend([arg, "i"]) @@ -1882,6 +1884,8 @@ def partitionwise_layer( ) else: pairs.extend([arg, None]) + [_.commit(name) for _ in reps] + layer = dask_blockwise( func, name, diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index 9a75236a..ea07af40 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -128,8 +128,6 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph def _optimize_columns(dsk, all_layers): - - # this loop is necessary_columns for k, lay in dsk.copy().items(): if not isinstance(lay, AwkwardInputLayer) or not hasattr( lay.io_func, "_column_report" @@ -144,9 +142,7 @@ def _optimize_columns(dsk, all_layers): except KeyError: pass try: - for col in rep.shape_touched_in((ln,)): - cols.add(col) - + cols.update(rep.shape_touched_in((ln,))) except KeyError: pass yield k, lay, cols @@ -159,10 +155,11 @@ def necessary_columns(*args): for arg in args: dsk.update(arg.dask.layers) all_layers.update(arg.dask.layers) - touch_data(arg._meta) - rep = getattr(arg.dask.layers[arg.name].meta, "_report", ()) - if rep: - all_reps.update(rep) + if hasattr(arg, "_meta"): # isinstance(, (dak.Scalar, dak.Array)? + touch_data(arg._meta) + rep = getattr(arg.dask.layers[arg.name].meta, "_report", ()) + if rep: + all_reps.update(rep) name = tokenize("output", args) [_.commit(name) for _ in all_reps] all_layers = tuple(all_layers) + (name,) diff --git a/tests/test_inspect.py b/tests/test_inspect.py index 8e58d3aa..5679656b 100644 --- a/tests/test_inspect.py +++ b/tests/test_inspect.py @@ -35,9 +35,9 @@ def test_basic_root_works(): .fill(events.MET_pt) ) - columns = list(dak.necessary_columns(q1_hist).values())[0] - assert columns == frozenset({"MET_pt"}) dask.compute(q1_hist) + columns = list(dak.necessary_columns(q1_hist).values())[0] + assert columns == ["MET_pt"] def test_sample(daa): From baf6f4654ed9ac8cc664214ea8a4246def8def49 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Mon, 27 May 2024 13:58:48 -0400 Subject: [PATCH 16/25] fix report --- src/dask_awkward/lib/core.py | 17 +++++++++-------- src/dask_awkward/lib/io/json.py | 4 ++-- src/dask_awkward/lib/io/parquet.py | 4 ++-- src/dask_awkward/lib/optimize.py | 4 ++-- src/dask_awkward/lib/structure.py | 4 ++-- src/dask_awkward/lib/utils.py | 6 ++++++ 6 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index 601470c2..bec9e5e7 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -48,6 +48,7 @@ from dask_awkward.layers import AwkwardBlockwiseLayer, AwkwardMaterializedLayer from dask_awkward.lib.optimize import all_optimizations +from dask_awkward.lib.utils import commit_to_reports from dask_awkward.utils import ( DaskAwkwardNotImplemented, IncompatiblePartitions, @@ -523,7 +524,7 @@ def f(self, other): meta = op(self._meta, other._meta) else: meta = op(self._meta, other) - [_.commit(name) for _ in self.report] + commit_to_reports(name, self.report) return new_scalar_object(graph, name, meta=meta) return f @@ -721,7 +722,7 @@ def __getitem__(self, where): new_name = f"{where}-{token}" report = self.report new_meta = self._meta[where] - [_.commit(new_name) for _ in report] + commit_to_reports(new_name, report) # first check for array type return if isinstance(new_meta, ak.Array): @@ -988,7 +989,7 @@ def repartition( new_graph = HighLevelGraph.from_collections( key, new_layer, dependencies=(self,) ) - [_.commit(key) for _ in self.report] + commit_to_reports(key, self.report) return new_array_object( new_graph, key, @@ -1192,7 +1193,7 @@ def _partitions(self, index: Any) -> Array: # otherwise nullify the known divisions else: new_divisions = (None,) * (len(new_keys) + 1) # type: ignore - [_.commit(name) for _ in self.report] + commit_to_reports(name, self.report) return new_array_object( graph, name, meta=self._meta, divisions=tuple(new_divisions) ) @@ -1414,7 +1415,7 @@ def _getitem_slice_on_zero(self, where): AwkwardMaterializedLayer(dask, previous_layer_names=[self.name]), dependencies=[self], ) - [_.commit(name) for _ in self.report] + commit_to_reports(name, self.report) return new_array_object( hlg, name, @@ -1529,7 +1530,7 @@ def __getitem__(self, where): else: out = self._getitem_single(where) if self.report: - [_.commit(out.name) for _ in self.report] + commit_to_reports(out.name, self.report) out._meta._report = self._meta._report out.dask.layers[out.name].meta = out._meta return out @@ -1884,7 +1885,7 @@ def partitionwise_layer( ) else: pairs.extend([arg, None]) - [_.commit(name) for _ in reps] + commit_to_reports(name, reps) layer = dask_blockwise( func, @@ -2319,7 +2320,7 @@ def non_trivial_reduction( mask_identity=mask_identity, ) trl.meta = meta - [_.commit(name_finalize) for _ in array.report] + commit_to_reports(name_finalize, array.report) if isinstance(meta, ak.highlevel.Array): meta._report = array.report return new_array_object(graph, name_finalize, meta=meta, npartitions=1) diff --git a/src/dask_awkward/lib/io/json.py b/src/dask_awkward/lib/io/json.py index f289f3de..61288d0d 100644 --- a/src/dask_awkward/lib/io/json.py +++ b/src/dask_awkward/lib/io/json.py @@ -32,7 +32,7 @@ _BytesReadingInstructions, from_map, ) -from dask_awkward.lib.utils import _buf_to_col +from dask_awkward.lib.utils import _buf_to_col, commit_to_reports if TYPE_CHECKING: from awkward.contents.content import Content @@ -773,7 +773,7 @@ def to_json( name = f"to-json-{tokenize(array, path)}" dsk = {(name, 0): (lambda *_: None, map_res.__dask_keys__())} touch_data(array._meta) - [_.commit(name) for _ in array.report] + commit_to_reports(name, array.report) graph = HighLevelGraph.from_collections( name, AwkwardMaterializedLayer(dsk, previous_layer_names=[map_res.name]), diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index 6863a0f8..e379d8c6 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -24,7 +24,7 @@ from dask_awkward.lib.io.columnar import ColumnProjectionMixin from dask_awkward.lib.io.io import from_map from dask_awkward.lib.unproject_layout import unproject_layout -from dask_awkward.lib.utils import _buf_to_col +from dask_awkward.lib.utils import _buf_to_col, commit_to_reports if TYPE_CHECKING: pass @@ -694,7 +694,7 @@ def to_parquet( dependencies=[map_res], ) touch_data(array._meta) - [_.commit(name) for _ in array.report] + commit_to_reports(name, array.report) out = new_scalar_object(graph, final_name, dtype="f8") if compute: out.compute() diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index ea07af40..e5174453 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -13,7 +13,7 @@ from dask.highlevelgraph import HighLevelGraph from dask_awkward.layers import AwkwardBlockwiseLayer, AwkwardInputLayer -from dask_awkward.lib.utils import _buf_to_col, typetracer_nochecks +from dask_awkward.lib.utils import _buf_to_col, commit_to_reports, typetracer_nochecks from dask_awkward.utils import first if TYPE_CHECKING: @@ -117,7 +117,7 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph if rep: all_reps.update(rep) name = tokenize("output", lays) - [_.commit(name) for _ in all_reps] + commit_to_reports(name, all_reps) all_layers = tuple(dsk.layers) + (name,) for k, lay, cols in _optimize_columns(dsk.layers, all_layers): diff --git a/src/dask_awkward/lib/structure.py b/src/dask_awkward/lib/structure.py index fdf44ed8..f86ac1fd 100644 --- a/src/dask_awkward/lib/structure.py +++ b/src/dask_awkward/lib/structure.py @@ -24,6 +24,7 @@ new_scalar_object, partition_compatibility, ) +from dask_awkward.lib.utils import commit_to_reports from dask_awkward.utils import ( DaskAwkwardNotImplemented, IncompatiblePartitions, @@ -939,8 +940,7 @@ def _array_with_rebuilt_meta( new_meta = ak.Array(array._meta, behavior=behavior, attrs=attrs) new_meta._report = array.report out = Array(array.dask, array.name, new_meta, array.divisions) - [_.commit(out.name) for _ in array.report] - + commit_to_reports(out.name, array.report) return out diff --git a/src/dask_awkward/lib/utils.py b/src/dask_awkward/lib/utils.py index c8686faf..65b8841f 100644 --- a/src/dask_awkward/lib/utils.py +++ b/src/dask_awkward/lib/utils.py @@ -10,6 +10,7 @@ if TYPE_CHECKING: from awkward.forms import Form + from awkward.typetracer import TypeTracerReport KNOWN_LENGTH_ATTRIBUTES = frozenset(("mask",)) UNKNOWN_LENGTH_ATTRIBUTES = frozenset(("offsets", "starts", "stops", "index", "tags")) @@ -180,6 +181,11 @@ def typetracer_nochecks(): del TypeTracerArray.runtime_typechecks +def commit_to_reports(name: str, reports: Iterable[TypeTracerReport]): + for report in reports: + report.commit(name) + + def _buf_to_col(s): return ( s[2:] From e29e9292ae48484998a8e4e71563ec695c6d2538 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Thu, 6 Jun 2024 12:52:33 -0400 Subject: [PATCH 17/25] if meta fails --- src/dask_awkward/lib/core.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index bec9e5e7..973bf622 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -1970,24 +1970,24 @@ def _map_partitions( **kwargs, ) + reps = set() try: if meta is None: meta = map_meta(fn, *args, **kwargs) else: # To do any touching?? map_meta(fn, *args, **kwargs) + meta._report = reps + lay.meta = meta except (AssertionError, TypeError, NotImplementedError): [touch_data(_._meta) for _ in dak_arrays] - reps = set() for dep in dak_arrays: for rep in dep.report: if rep not in reps: rep.commit(name) reps.add(rep) - meta._report = reps - lay.meta = meta hlg = HighLevelGraph.from_collections( name, lay, From f61fdd70d095cbb2d1c8e555da175dca55ccab1d Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Tue, 23 Jul 2024 16:08:29 -0400 Subject: [PATCH 18/25] rev --- tests/conftest.py | 44 ++++++++++++++++++++++---------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 97785744..a96f6200 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -16,7 +16,7 @@ def clear_cache(): dak.lib.core.dak_cache.clear() -@pytest.fixture(scope="session") +@pytest.fixture() def single_record_file(tmp_path_factory: pytest.TempPathFactory) -> str: fname = tmp_path_factory.mktemp("data") / "single_record.json" record = {"record": [1, 2, 3]} @@ -25,7 +25,7 @@ def single_record_file(tmp_path_factory: pytest.TempPathFactory) -> str: return str(fname) -@pytest.fixture(scope="session") +@pytest.fixture() def ndjson_points1(tmp_path_factory: pytest.TempPathFactory) -> str: array = daktu.awkward_xy_points() fname = tmp_path_factory.mktemp("data") / "points_ndjson1.json" @@ -35,7 +35,7 @@ def ndjson_points1(tmp_path_factory: pytest.TempPathFactory) -> str: return str(fname) -@pytest.fixture(scope="session") +@pytest.fixture() def ndjson_points1_str(tmp_path_factory: pytest.TempPathFactory) -> str: array = daktu.awkward_xy_points_str() fname = tmp_path_factory.mktemp("data") / "points_ndjson1.json" @@ -45,7 +45,7 @@ def ndjson_points1_str(tmp_path_factory: pytest.TempPathFactory) -> str: return str(fname) -@pytest.fixture(scope="session") +@pytest.fixture() def ndjson_points2(tmp_path_factory: pytest.TempPathFactory) -> str: array = daktu.awkward_xy_points() fname = tmp_path_factory.mktemp("data") / "points_ndjson2.json" @@ -55,77 +55,77 @@ def ndjson_points2(tmp_path_factory: pytest.TempPathFactory) -> str: return str(fname) -@pytest.fixture(scope="session") +@pytest.fixture() def ndjson_points_file(ndjson_points1: str) -> str: return ndjson_points1 -@pytest.fixture(scope="session") +@pytest.fixture() def ndjson_points_file_str(ndjson_points1_str: str) -> str: return ndjson_points1_str -@pytest.fixture(scope="session") +@pytest.fixture() def daa(ndjson_points1: str) -> dak.Array: return dak.from_json([ndjson_points1] * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def pq_points_dir(daa: dak.Array, tmp_path_factory: pytest.TempPathFactory) -> str: pqdir = tmp_path_factory.mktemp("pqfiles") dak.to_parquet(daa, str(pqdir)) return str(pqdir) -@pytest.fixture(scope="session") +@pytest.fixture() def daa_parquet(pq_points_dir: str) -> dak.Array: return cast(dak.Array, dak.from_parquet(pq_points_dir)) -@pytest.fixture(scope="session") +@pytest.fixture() def daa_str(ndjson_points1_str: str) -> dak.Array: return dak.from_json([ndjson_points1_str] * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def caa(ndjson_points1: str) -> ak.Array: with open(ndjson_points1, "rb") as f: a = ak.from_json(f, line_delimited=True) return ak.concatenate([a, a, a]) -@pytest.fixture(scope="session") +@pytest.fixture() def caa_str(ndjson_points1_str: str) -> ak.Array: with open(ndjson_points1_str, "rb") as f: a = ak.from_json(f, line_delimited=True) return ak.concatenate([a, a, a]) -@pytest.fixture(scope="session") +@pytest.fixture() def daa_p1(ndjson_points1: str) -> dak.Array: return dak.from_json([ndjson_points1] * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def daa_p2(ndjson_points2: str) -> dak.Array: return dak.from_json([ndjson_points2] * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def caa_p1(ndjson_points1: str) -> ak.Array: with open(ndjson_points1) as f: lines = [json.loads(line) for line in f] return ak.Array(lines * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def caa_p2(ndjson_points2: str) -> ak.Array: with open(ndjson_points2) as f: lines = [json.loads(line) for line in f] return ak.Array(lines * 3) -@pytest.fixture(scope="session") +@pytest.fixture() def L1() -> list[list[dict[str, float]]]: return [ [{"x": 1.0, "y": 1.1}, {"x": 2.0, "y": 2.2}, {"x": 3, "y": 3.3}], @@ -136,7 +136,7 @@ def L1() -> list[list[dict[str, float]]]: ] -@pytest.fixture(scope="session") +@pytest.fixture() def L2() -> list[list[dict[str, float]]]: return [ [{"x": 0.9, "y": 1.0}, {"x": 2.0, "y": 2.2}, {"x": 2.9, "y": 3.0}], @@ -147,7 +147,7 @@ def L2() -> list[list[dict[str, float]]]: ] -@pytest.fixture(scope="session") +@pytest.fixture() def L3() -> list[list[dict[str, float]]]: return [ [{"x": 1.9, "y": 9.0}, {"x": 2.0, "y": 8.2}, {"x": 9.9, "y": 9.0}], @@ -158,7 +158,7 @@ def L3() -> list[list[dict[str, float]]]: ] -@pytest.fixture(scope="session") +@pytest.fixture() def L4() -> list[list[dict[str, float]] | None]: return [ [{"x": 1.9, "y": 9.0}, {"x": 2.0, "y": 8.2}, {"x": 9.9, "y": 9.0}], @@ -169,14 +169,14 @@ def L4() -> list[list[dict[str, float]] | None]: ] -@pytest.fixture(scope="session") +@pytest.fixture() def caa_parquet(caa: ak.Array, tmp_path_factory: pytest.TempPathFactory) -> str: fname = tmp_path_factory.mktemp("parquet_data") / "caa.parquet" ak.to_parquet(caa, str(fname), extensionarray=False) return str(fname) -@pytest.fixture(scope="session") +@pytest.fixture() def unnamed_root_parquet_file(tmp_path_factory: pytest.TempPathFactory) -> str: from dask_awkward.lib.testutils import unnamed_root_ds From 2c3abd0d7d1d631e698198b1c7883324d14ac8e1 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Wed, 24 Jul 2024 09:38:29 -0400 Subject: [PATCH 19/25] concat enforce condition --- src/dask_awkward/lib/operations.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index 99487693..ca77238e 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -87,7 +87,7 @@ def concatenate( not m.layout.form.is_equal_to( intended_form, all_parameters=True, form_key=False ) - for m in metas + for m in metas_no_report[1:] ): arrays = [ map_partitions( From 04abbc8e6edebc9d43663900f1282673b3dac421 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Mon, 29 Jul 2024 11:40:49 -0400 Subject: [PATCH 20/25] temp --- src/dask_awkward/lib/core.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index 973bf622..f6534c36 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -895,6 +895,9 @@ def __init__( self._name: str = name self._divisions: tuple[int, ...] | tuple[None, ...] = divisions self._meta: ak.Array = meta + if any(_.shape_touched for _ in self.report): + # DIAGNOSTIC - this should never happen + breakpoint() def __dask_graph__(self) -> HighLevelGraph: return self.dask @@ -1960,7 +1963,7 @@ def _map_partitions( ] dak_arrays = tuple(filter(lambda x: isinstance(x, Array), deps)) - if name in dak_cache: + if False: # name in dak_cache: hlg, meta = dak_cache[name] else: lay = partitionwise_layer( @@ -2000,6 +2003,12 @@ def _map_partitions( "should be a dask_awkward.Array collection." ) dak_cache[name] = hlg, meta + if name in dak_cache: + hlg0, meta0 = dak_cache[name] + if meta0.layout.form != meta.layout.form: + breakpoint() + if hlg0 != hlg: + breakpoint() in_npartitions = dak_arrays[0].npartitions in_divisions = dak_arrays[0].divisions if output_divisions is not None: From d876f00f0883cc278e9ec0aa262f0aa1971e1527 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Tue, 30 Jul 2024 13:52:10 -0400 Subject: [PATCH 21/25] squached some --- src/dask_awkward/layers/layers.py | 5 +- src/dask_awkward/lib/core.py | 9 +--- src/dask_awkward/lib/io/io.py | 7 ++- src/dask_awkward/lib/io/parquet.py | 8 --- src/dask_awkward/lib/optimize.py | 83 ++++++++++++++++++------------ tests/test_core.py | 1 + tests/test_inspect.py | 2 +- tests/test_optimize.py | 5 +- tests/test_utils.py | 5 +- 9 files changed, 68 insertions(+), 57 deletions(-) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index a14bd173..c84af817 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -145,7 +145,10 @@ def is_columnar(self) -> bool: return io_func_implements_columnar(self.io_func) def project(self, columns: list[str]) -> AwkwardInputLayer: - io_func = self.io_func.project(columns) + if hasattr(self.io_func, "project"): + io_func = self.io_func.project(columns) + else: + return self return AwkwardInputLayer( name=self.name, inputs=self.inputs, diff --git a/src/dask_awkward/lib/core.py b/src/dask_awkward/lib/core.py index f6534c36..58e9b202 100644 --- a/src/dask_awkward/lib/core.py +++ b/src/dask_awkward/lib/core.py @@ -895,9 +895,6 @@ def __init__( self._name: str = name self._divisions: tuple[int, ...] | tuple[None, ...] = divisions self._meta: ak.Array = meta - if any(_.shape_touched for _ in self.report): - # DIAGNOSTIC - this should never happen - breakpoint() def __dask_graph__(self) -> HighLevelGraph: return self.dask @@ -1963,7 +1960,7 @@ def _map_partitions( ] dak_arrays = tuple(filter(lambda x: isinstance(x, Array), deps)) - if False: # name in dak_cache: + if name in dak_cache: hlg, meta = dak_cache[name] else: lay = partitionwise_layer( @@ -2005,10 +2002,6 @@ def _map_partitions( dak_cache[name] = hlg, meta if name in dak_cache: hlg0, meta0 = dak_cache[name] - if meta0.layout.form != meta.layout.form: - breakpoint() - if hlg0 != hlg: - breakpoint() in_npartitions = dak_arrays[0].npartitions in_divisions = dak_arrays[0].divisions if output_divisions is not None: diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index cc10ced8..e5311a77 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -636,7 +636,6 @@ def from_map( behavior=io_func.behavior, buffer_key=render_buffer_key, ) - io_func._column_report = report report.commit(name) # column tracking report, not failure report, below array_meta._report = {report} @@ -670,9 +669,11 @@ def from_map( if io_func_implements_report(io_func): if cast(ImplementsReport, io_func).return_report: + # first element of each output tuple is the actual data res = result.map_partitions( - first, meta=array_meta, label=label, output_divisions=1 + first, meta=empty_typetracer(), label=label, output_divisions=1 ) + res._meta = array_meta concat_fn = partial( ak.concatenate, @@ -686,6 +687,8 @@ def from_map( rep_trl_name = f"{rep_trl_label}-{rep_trl_token}" rep_trl_tree_node_name = f"{rep_trl_label}-tree-node-{rep_trl_token}" + # second element of each output tuple is the result, which does not + # depend on any of the actual data rep_part = result.map_partitions( second, meta=empty_typetracer(), label=f"{label}-partitioned-report" ) diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index e379d8c6..fafb05dd 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -10,7 +10,6 @@ import awkward as ak import awkward.operations.ak_from_parquet as ak_from_parquet -import dask from awkward.forms.form import Form from awkward.typetracer import touch_data from dask.base import tokenize @@ -102,13 +101,6 @@ def project(self, columns): ... def return_report(self) -> bool: return self.report - @property - def use_optimization(self) -> bool: - return "parquet" in dask.config.get( - "awkward.optimization.columns-opt-formats", - default=[], - ) - def __repr__(self) -> str: s = ( "\nFromParquetFn(\n" diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index e5174453..d2c16cee 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -73,7 +73,9 @@ def optimize(dsk: HighLevelGraph, keys: Sequence[Key], **_: Any) -> Mapping: return dsk -def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph: +def optimize_columns( + dsk: HighLevelGraph, keys: Sequence[Key], dryrun=False +) -> HighLevelGraph: """Run column projection optimization. This optimization determines which columns from an @@ -120,20 +122,28 @@ def optimize_columns(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelGraph commit_to_reports(name, all_reps) all_layers = tuple(dsk.layers) + (name,) + if dryrun: + out = {} for k, lay, cols in _optimize_columns(dsk.layers, all_layers): - new_lay = lay.project(cols) - dsk2[k] = new_lay + if dryrun: + out[k] = cols + else: + new_lay = lay.project(cols) + dsk2[k] = new_lay + if dryrun: + return out return HighLevelGraph(dsk2, dsk.dependencies) def _optimize_columns(dsk, all_layers): for k, lay in dsk.copy().items(): - if not isinstance(lay, AwkwardInputLayer) or not hasattr( - lay.io_func, "_column_report" - ): + if not isinstance(lay, AwkwardInputLayer) or not hasattr(lay, "meta"): continue - rep = lay.io_func._column_report + rep = getattr(lay.meta, "_report", None) + if not rep: + continue + rep = first(rep) # each meta of an IL layer should have just one report cols = set() # this loop not required after next ak release for ln in all_layers: @@ -145,36 +155,45 @@ def _optimize_columns(dsk, all_layers): cols.update(rep.shape_touched_in((ln,))) except KeyError: pass - yield k, lay, cols + if cols: + yield k, lay, cols + +def necessary_columns(*args, normalize: bool = True, trim: bool = True): + """Find the columns in each input layer that are needed by given collections + + Parameters + ---------- + args: dask-awkward colections or other dask objects baseed on them + normalize: if True, will transform the internal buffer-oriented representation + to column names similar to the convention used for instance by parquet. The + raw representation is the one actually passed to the IO backends during + optimization, and includes information about which component of a field + is needed (data, offsets, index, etc.) + trim: if normalize is True, setting this True will remove parent columns -def necessary_columns(*args): + Returns + ------- + dict: the keys are the dask names of IO layers contained in the combined graph, + and for each there is a set of required columns + """ dsk = {} - all_reps = set() - all_layers = set() + keys = [] for arg in args: dsk.update(arg.dask.layers) - all_layers.update(arg.dask.layers) - if hasattr(arg, "_meta"): # isinstance(, (dak.Scalar, dak.Array)? - touch_data(arg._meta) - rep = getattr(arg.dask.layers[arg.name].meta, "_report", ()) - if rep: - all_reps.update(rep) - name = tokenize("output", args) - [_.commit(name) for _ in all_reps] - all_layers = tuple(all_layers) + (name,) - - out = {} - for k, _, cols in _optimize_columns(dsk, all_layers): - first = (_buf_to_col(s) for s in cols) - # remove root "offsets", which appears when computing divisions - second = {s for s in first if s and s != "offsets"} - # remove columns that have sub-columns also in the set - for c in second.copy(): - if "." in c: - parent = c.rsplit(".", 1)[0] - second.discard(parent) - out[k] = sorted(second) + keys.append((arg.name, 0)) + hlg = HighLevelGraph(dsk, {}) + out = optimize_columns(hlg, keys, dryrun=True) + if normalize: + for k in list(out): + # `startswith` to clobber attributes of unnamed root field + col1 = {_buf_to_col(_) for _ in out[k] if _.startswith("@.")} + if trim: + parents = {_.rsplit(".", 1)[0] for _ in col1 if "." in _} + out[k] = {_ for _ in col1 if _ not in parents} + else: + out[k] = col1 + # TODO: remove columns included in children? return out diff --git a/tests/test_core.py b/tests/test_core.py index 7aced6bf..05006ecc 100644 --- a/tests/test_core.py +++ b/tests/test_core.py @@ -856,6 +856,7 @@ def test_map_partitions_args_and_kwargs_have_collection(): def test_dask_array_in_map_partitions(daa, caa): + daa.eager_compute_divisions() x1 = dak.zeros_like(daa.points.x) y1 = da.ones(len(x1), chunks=x1.divisions[1]) z1 = x1 + y1 diff --git a/tests/test_inspect.py b/tests/test_inspect.py index 5679656b..baea4438 100644 --- a/tests/test_inspect.py +++ b/tests/test_inspect.py @@ -37,7 +37,7 @@ def test_basic_root_works(): dask.compute(q1_hist) columns = list(dak.necessary_columns(q1_hist).values())[0] - assert columns == ["MET_pt"] + assert columns == {"MET_pt"} def test_sample(daa): diff --git a/tests/test_optimize.py b/tests/test_optimize.py index 52f082b8..1992c09e 100644 --- a/tests/test_optimize.py +++ b/tests/test_optimize.py @@ -143,12 +143,11 @@ def something(j, k): return j.x + k.x a_compute = something(a, b) - print() nc1 = dak.necessary_columns(a_compute) - assert list(nc1.values())[0] == ["points.x"] + assert list(nc1.values())[0] == {"points.x"} nc2 = dak.necessary_columns(a_compute, a, b) - assert list(nc2.items())[0][1] == ["points.x", "points.y", "points.z"] + assert list(nc2.items())[0][1] == {"points.x", "points.y", "points.z"} x, (y, z) = dask.compute(a_compute, (a, b)) assert str(x) diff --git a/tests/test_utils.py b/tests/test_utils.py index 3d1b38c7..9a65e912 100644 --- a/tests/test_utils.py +++ b/tests/test_utils.py @@ -84,7 +84,8 @@ def test_field_access_to_front(pairs): def test_nocheck_context(): from awkward._nplikes.typetracer import TypeTracerArray - assert getattr(TypeTracerArray, "runtime_typechecks", True) + old = getattr(TypeTracerArray, "runtime_typechecks", True) with typetracer_nochecks(): assert not TypeTracerArray.runtime_typechecks - assert getattr(TypeTracerArray, "runtime_typechecks", True) + new = getattr(TypeTracerArray, "runtime_typechecks", True) + assert old == new From 8e1d5079c03a3c2d894a8a85297b5b16ac6da2ba Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Tue, 30 Jul 2024 16:29:42 -0400 Subject: [PATCH 22/25] add note --- src/dask_awkward/lib/optimize.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index d2c16cee..f5df4096 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -250,7 +250,8 @@ def rewrite_layer_chains(dsk: HighLevelGraph, keys: Sequence[Key]) -> HighLevelG len(children) == 1 and dsk.dependencies[first(children)] == {current_layer_key} and isinstance(dsk.layers[first(children)], AwkwardBlockwiseLayer) - and len(dsk.layers[current_layer_key]) == len(dsk.layers[first(children)]) + and len(dsk.layers[current_layer_key]) + == len(dsk.layers[first(children)]) # SLOW?! and current_layer_key not in required_layers ): # walk forwards From fc9589b92066236ff22a02e5c4914436bb5e1729 Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Wed, 31 Jul 2024 09:57:23 -0400 Subject: [PATCH 23/25] Fix concat form comparison --- src/dask_awkward/lib/operations.py | 23 ++++++----------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index ca77238e..7fdd8147 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -7,7 +7,6 @@ from awkward.operations.ak_concatenate import ( enforce_concatenated_form as enforce_layout_to_concatenated_form, ) -from awkward.typetracer import typetracer_from_form from dask.base import tokenize from dask.highlevelgraph import HighLevelGraph @@ -65,29 +64,18 @@ def concatenate( if len(metas) == 0: raise ValueError("Need at least one array to concatenate") - # Are we performing a _logical_ concatenation? if axis == 0: # There are two possible cases here: # 1. all arrays have identical metas — just grow the Dask collection # 2. some arrays have different metas — coerce arrays to same form - - # Drop reports from metas to avoid later touching any buffers - metas_no_report = [ - typetracer_from_form(x.layout.form, behavior=x.behavior, attrs=x._attrs) - for x in metas - ] - # Concatenate metas to determine result form - meta_no_report = ak.concatenate( - metas_no_report, axis=0, behavior=behavior, attrs=attrs - ) - intended_form = meta_no_report.layout.form + intended_form = metas[0].layout.form # If any forms aren't equal to this form, we must enforce each form to the same type if any( not m.layout.form.is_equal_to( intended_form, all_parameters=True, form_key=False ) - for m in metas_no_report[1:] + for m in metas[1:] ): arrays = [ map_partitions( @@ -122,13 +110,14 @@ def concatenate( aml = AwkwardMaterializedLayer(g, previous_layer_names=[arrays[0].name]) + new_meta = ak.copy(metas[0]) + new_meta._report = report hlg = HighLevelGraph.from_collections(name, aml, dependencies=arrays) - meta_no_report._report = report - aml.meta = meta_no_report + aml.meta = new_meta return new_array_object( hlg, name, - meta=meta_no_report, + meta=new_meta, npartitions=sum(a.npartitions for a in arrays), ) From 961dd0cfcd079e480dd9490d07c3782bbed0642a Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Wed, 31 Jul 2024 16:28:19 -0400 Subject: [PATCH 24/25] one more squashed --- src/dask_awkward/layers/__init__.py | 4 ---- src/dask_awkward/layers/layers.py | 31 +---------------------------- src/dask_awkward/lib/io/columnar.py | 4 +--- src/dask_awkward/lib/io/io.py | 4 +++- src/dask_awkward/lib/io/parquet.py | 8 +------- src/dask_awkward/lib/operations.py | 19 +++++++++--------- tests/test_operations.py | 5 +++++ 7 files changed, 20 insertions(+), 55 deletions(-) diff --git a/src/dask_awkward/layers/__init__.py b/src/dask_awkward/layers/__init__.py index d4ba4c5e..6ea8c1c3 100644 --- a/src/dask_awkward/layers/__init__.py +++ b/src/dask_awkward/layers/__init__.py @@ -4,8 +4,6 @@ AwkwardMaterializedLayer, AwkwardTreeReductionLayer, ImplementsIOFunction, - ImplementsProjection, - IOFunctionWithMocking, io_func_implements_projection, ) @@ -14,8 +12,6 @@ "AwkwardBlockwiseLayer", "AwkwardMaterializedLayer", "AwkwardTreeReductionLayer", - "ImplementsProjection", "ImplementsIOFunction", - "IOFunctionWithMocking", "io_func_implements_projection", ) diff --git a/src/dask_awkward/layers/layers.py b/src/dask_awkward/layers/layers.py index c84af817..f9320da9 100644 --- a/src/dask_awkward/layers/layers.py +++ b/src/dask_awkward/layers/layers.py @@ -1,21 +1,14 @@ from __future__ import annotations from collections.abc import Callable, Mapping -from typing import TYPE_CHECKING, Any, Literal, Protocol, TypeVar, Union +from typing import Any, Protocol, TypeVar from dask.blockwise import Blockwise, BlockwiseDepDict, blockwise_token from dask.highlevelgraph import MaterializedLayer from dask.layers import DataFrameTreeReduction -from typing_extensions import TypeAlias from dask_awkward.utils import LazyInputsDict -if TYPE_CHECKING: - from awkward import Array as AwkwardArray - - -BackendT: TypeAlias = Union[Literal["cpu"], Literal["jax"], Literal["cuda"]] - class AwkwardBlockwiseLayer(Blockwise): """Just like upstream Blockwise, except we override pickling""" @@ -46,33 +39,11 @@ def __call__(self, *args, **kwargs): ... T = TypeVar("T") -class ImplementsMockEmpty(ImplementsIOFunction, Protocol): - def mock_empty(self, backend: BackendT) -> AwkwardArray: ... - - class ImplementsReport(ImplementsIOFunction, Protocol): @property def return_report(self) -> bool: ... -class ImplementsProjection(Protocol[T]): - def project(self, columns: list[str]) -> ImplementsIOFunction: ... - - -class IOFunctionWithMocking(ImplementsIOFunction): - def __init__(self, meta: AwkwardArray, io_func: ImplementsIOFunction): - self._meta = meta - self._io_func = io_func - - def __getstate__(self) -> dict: - state = self.__dict__.copy() - state["_meta"] = None - return state - - def __call__(self, *args, **kwargs): - return self._io_func(*args, **kwargs) - - def io_func_implements_projection(func: ImplementsIOFunction) -> bool: return hasattr(func, "project") diff --git a/src/dask_awkward/lib/io/columnar.py b/src/dask_awkward/lib/io/columnar.py index ff449e8c..752e2369 100644 --- a/src/dask_awkward/lib/io/columnar.py +++ b/src/dask_awkward/lib/io/columnar.py @@ -4,8 +4,6 @@ import awkward as ak -from dask_awkward.layers.layers import BackendT - log = logging.getLogger(__name__) @@ -22,7 +20,7 @@ def project(self, *args, **kwargs): # default implementation does nothing return self - def mock_empty(self, backend: BackendT = "cpu"): + def mock_empty(self, backend: str = "cpu"): # used by failure report generation return ( ak.to_backend( diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index e5311a77..4cbd4bb9 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -635,6 +635,7 @@ def from_map( highlevel=True, behavior=io_func.behavior, buffer_key=render_buffer_key, + attrs=meta._attrs if meta is not None else None, ) report.commit(name) # column tracking report, not failure report, below @@ -646,7 +647,8 @@ def from_map( array_meta, report = typetracer_with_report( form_with_unique_keys(meta.layout.form, "@"), highlevel=True, - behavior=None, + behavior=meta._behavior, + attrs=meta._attrs, buffer_key=render_buffer_key, ) report.commit(name) diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index fafb05dd..349b3ec6 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -6,7 +6,7 @@ import math import operator from collections.abc import Mapping -from typing import TYPE_CHECKING, Any, Literal, TypeVar, cast +from typing import Any, Literal, cast import awkward as ak import awkward.operations.ak_from_parquet as ak_from_parquet @@ -25,15 +25,9 @@ from dask_awkward.lib.unproject_layout import unproject_layout from dask_awkward.lib.utils import _buf_to_col, commit_to_reports -if TYPE_CHECKING: - pass - log = logging.getLogger(__name__) -T = TypeVar("T") - - def report_failure(exception, *args, **kwargs): return ak.Array( [ diff --git a/src/dask_awkward/lib/operations.py b/src/dask_awkward/lib/operations.py index 7fdd8147..0b4c7ad5 100644 --- a/src/dask_awkward/lib/operations.py +++ b/src/dask_awkward/lib/operations.py @@ -1,5 +1,6 @@ from __future__ import annotations +import copy from collections.abc import Mapping from typing import TYPE_CHECKING, Any @@ -35,17 +36,10 @@ def __call__(self, *args): def _enforce_concatenated_form(array: AwkwardArray, form: Form) -> AwkwardArray: layout = ak.to_layout(array) - # TODO: should this check whether the form agrees first, or assume that the - # operation is harmless if not required? result = enforce_layout_to_concatenated_form(layout, form) return ak.Array(result, behavior=array._behavior, attrs=array._attrs) -def _concatenate_axis_0_meta(*arrays: AwkwardArray) -> AwkwardArray: - # At this stage, the metas have all been enforced to the same type - return arrays[0] - - def concatenate( arrays: list[Array], axis: int = 0, @@ -98,7 +92,6 @@ def concatenate( aml = AwkwardMaterializedLayer( g, previous_layer_names=[a.name for a in arrays], - fn=_concatenate_axis_0_meta, ) else: g = { @@ -108,9 +101,15 @@ def concatenate( ) } - aml = AwkwardMaterializedLayer(g, previous_layer_names=[arrays[0].name]) + aml = AwkwardMaterializedLayer( + g, previous_layer_names=[a.name for a in arrays] + ) + from awkward.typetracer import touch_data - new_meta = ak.copy(metas[0]) + # TODO: touching all metas we don't pass on should not be necessary + [touch_data(m) for m in metas[1:]] + [r.commit(name) for r in report] + new_meta = copy.copy(metas[0]) new_meta._report = report hlg = HighLevelGraph.from_collections(name, aml, dependencies=arrays) aml.meta = new_meta diff --git a/tests/test_operations.py b/tests/test_operations.py index b7cad07d..352063e6 100644 --- a/tests/test_operations.py +++ b/tests/test_operations.py @@ -10,6 +10,7 @@ @pytest.mark.parametrize("axis", [0, 1]) def test_concatenate_simple(daa, caa, axis): + # daa.x and daa.y have the same types assert_eq( ak.concatenate([caa.points.x, caa.points.y], axis=axis), dak.concatenate([daa.points.x, daa.points.y], axis=axis), @@ -42,6 +43,10 @@ def test_concatenate_axis_0_logical_different(daa): empty_array = ak.Array(empty_form.length_zero_array(highlevel=False)) empty_dak_array = dak.from_awkward(empty_array, npartitions=1) result = dak.concatenate([daa, empty_dak_array], axis=0) + expected = ak.concatenate( + [daa.compute(), empty_dak_array.compute()], axis=0 + ).tolist() + assert expected == result.compute().tolist() @pytest.mark.parametrize("axis", [0, 1, 2]) From c8b254bda5ee4aaaa537a91441148e0a94f22edf Mon Sep 17 00:00:00 2001 From: Martin Durant Date: Fri, 2 Aug 2024 14:49:52 -0400 Subject: [PATCH 25/25] fix IO report @lgrey - moved from IO reporting layers returning a tuple to returning a record (or dict) for my own sanity. --- src/dask_awkward/lib/io/io.py | 66 ++++-------------------------- src/dask_awkward/lib/io/parquet.py | 7 +++- src/dask_awkward/lib/optimize.py | 2 +- src/dask_awkward/lib/testutils.py | 10 ++++- 4 files changed, 23 insertions(+), 62 deletions(-) diff --git a/src/dask_awkward/lib/io/io.py b/src/dask_awkward/lib/io/io.py index 4cbd4bb9..117cdb06 100644 --- a/src/dask_awkward/lib/io/io.py +++ b/src/dask_awkward/lib/io/io.py @@ -4,17 +4,14 @@ import math from collections.abc import Callable, Iterable, Mapping from dataclasses import dataclass -from functools import partial from typing import TYPE_CHECKING, Any, cast import awkward as ak -import dask.config import numpy as np from awkward.types.numpytype import primitive_to_dtype from awkward.typetracer import length_zero_if_typetracer, typetracer_with_report from dask.base import flatten, tokenize from dask.highlevelgraph import HighLevelGraph -from dask.local import identity from dask.utils import funcname, is_integer, parse_bytes from fsspec.utils import infer_compression @@ -22,10 +19,7 @@ AwkwardBlockwiseLayer, AwkwardInputLayer, AwkwardMaterializedLayer, - AwkwardTreeReductionLayer, - ImplementsReport, io_func_implements_projection, - io_func_implements_report, ) from dask_awkward.lib.core import ( Array, @@ -36,7 +30,6 @@ ) from dask_awkward.lib.io.columnar import ColumnProjectionMixin from dask_awkward.lib.utils import form_with_unique_keys, render_buffer_key -from dask_awkward.utils import first, second if TYPE_CHECKING: from dask.array.core import Array as DaskArray @@ -661,6 +654,13 @@ def from_map( dsk = AwkwardInputLayer(name=name, inputs=inputs, io_func=io_func) hlg = HighLevelGraph.from_collections(name, dsk) + making_report = getattr(io_func, "return_report", False) + if making_report: + array_meta = ak.Array( + {"ioreport": ak.Array([0]).layout.to_typetracer(True), "data": array_meta} + ) + array_meta._report = {report} + if divisions is not None: result = new_array_object(hlg, name, meta=array_meta, divisions=divisions, **kw) else: @@ -669,56 +669,8 @@ def from_map( ) dsk.meta = result._meta - if io_func_implements_report(io_func): - if cast(ImplementsReport, io_func).return_report: - # first element of each output tuple is the actual data - res = result.map_partitions( - first, meta=empty_typetracer(), label=label, output_divisions=1 - ) - res._meta = array_meta - - concat_fn = partial( - ak.concatenate, - axis=0, - ) - - split_every = dask.config.get("awkward.aggregation.split-every", 8) - - rep_trl_label = f"{label}-report" - rep_trl_token = tokenize(result, second, concat_fn, split_every) - rep_trl_name = f"{rep_trl_label}-{rep_trl_token}" - rep_trl_tree_node_name = f"{rep_trl_label}-tree-node-{rep_trl_token}" - - # second element of each output tuple is the result, which does not - # depend on any of the actual data - rep_part = result.map_partitions( - second, meta=empty_typetracer(), label=f"{label}-partitioned-report" - ) - - rep_trl = AwkwardTreeReductionLayer( - name=rep_trl_name, - name_input=rep_part.name, - npartitions_input=rep_part.npartitions, - concat_func=concat_fn, - tree_node_func=identity, - finalize_func=identity, - split_every=split_every, - tree_node_name=rep_trl_tree_node_name, - ) - - rep_graph = HighLevelGraph.from_collections( - rep_trl_name, rep_trl, dependencies=[rep_part] - ) - rep_trl.meta = empty_typetracer() - - rep = new_array_object( - rep_graph, - rep_trl_name, - meta=rep_trl.meta, - npartitions=len(rep_trl.output_partitions), - ) - - return res, rep + if making_report: + return result.data, result.ioreport return result diff --git a/src/dask_awkward/lib/io/parquet.py b/src/dask_awkward/lib/io/parquet.py index 349b3ec6..9a03c8e1 100644 --- a/src/dask_awkward/lib/io/parquet.py +++ b/src/dask_awkward/lib/io/parquet.py @@ -158,9 +158,12 @@ def __call__(self, *args, **kwargs): if self.return_report: try: result = self.read_fn(source) - return result, report_success(self.columns, source) + return { + "data": result, + "ioreport": report_success(self.columns, source), + } except self.allowed_exceptions as err: - return self.mock_empty(), report_failure(err, source) + return {"data": ak.Array([]), "ioreport": report_failure(err, source)} return self.read_fn(source) diff --git a/src/dask_awkward/lib/optimize.py b/src/dask_awkward/lib/optimize.py index f5df4096..ab854477 100644 --- a/src/dask_awkward/lib/optimize.py +++ b/src/dask_awkward/lib/optimize.py @@ -143,7 +143,7 @@ def _optimize_columns(dsk, all_layers): rep = getattr(lay.meta, "_report", None) if not rep: continue - rep = first(rep) # each meta of an IL layer should have just one report + rep = first(rep) # each meta of an IO layer should have just one report cols = set() # this loop not required after next ak release for ln in all_layers: diff --git a/src/dask_awkward/lib/testutils.py b/src/dask_awkward/lib/testutils.py index 3dee6d36..ed82e11a 100644 --- a/src/dask_awkward/lib/testutils.py +++ b/src/dask_awkward/lib/testutils.py @@ -287,9 +287,15 @@ def __call__(self, *args, **kwargs): if self.return_report: try: result, time = time_it(self.read_fn)(*args, **kwargs) - return result, self.make_success_report(time, *args, **kwargs) + return { + "data": result, + "ioreport": self.make_success_report(time, *args, **kwargs), + } except self.allowed_exceptions as err: - return self.mock_empty(), self.make_fail_report(err, *args, **kwargs) + return { + "data": self.mock_empty(), + "ioreport": self.make_fail_report(err, *args, **kwargs), + } return self.read_fn(*args, **kwargs)