def overlap_internal(x, axes): """Share boundaries between neighboring blocks Parameters ---------- x: da.Array A dask array axes: dict The size of the shared boundary per axis The axes input informs how many cells to overlap between neighboring blocks {0: 2, 2: 5} means share two cells in 0 axis, 5 cells in 2 axis """ dims = list(map(len, x.chunks)) expand_key2 = partial(expand_key, dims=dims, axes=axes) # Make keys for each of the surrounding sub-arrays interior_keys = pipe(x.__dask_keys__(), flatten, map(expand_key2), map(flatten), concat, list) name = "overlap-" + tokenize(x, axes) getitem_name = "getitem-" + tokenize(x, axes) interior_slices = {} overlap_blocks = {} for k in interior_keys: frac_slice = fractional_slice((x.name, ) + k, axes) if (x.name, ) + k != frac_slice: interior_slices[(getitem_name, ) + k] = frac_slice else: interior_slices[(getitem_name, ) + k] = (x.name, ) + k overlap_blocks[(name, ) + k] = ( concatenate3, (concrete, expand_key2((None, ) + k, name=getitem_name)), ) chunks = [] for i, bds in enumerate(x.chunks): depth = axes.get(i, 0) if isinstance(depth, tuple): left_depth = depth[0] right_depth = depth[1] else: left_depth = depth right_depth = depth if len(bds) == 1: chunks.append(bds) else: left = [bds[0] + right_depth] right = [bds[-1] + left_depth] mid = [] for bd in bds[1:-1]: mid.append(bd + left_depth + right_depth) chunks.append(left + mid + right) dsk = merge(interior_slices, overlap_blocks) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[x]) return Array(graph, name, chunks, meta=x)
def _construct_graph(self, deserializing=False): """Construct graph for a simple overlap operation.""" axes = self.axes chunks = self.chunks name = self.name dask_keys = self._dask_keys() getitem_name = "getitem-" + self.token overlap_name = "overlap-" + self.token if deserializing: # Use CallableLazyImport objects to avoid importing dataframe # module on the scheduler concatenate3 = CallableLazyImport("dask.array.core.concatenate3") else: # Not running on distributed scheduler - Use explicit functions from dask.array.core import concatenate3 dims = list(map(len, chunks)) expand_key2 = partial(_expand_keys_around_center, dims=dims, axes=axes) # Make keys for each of the surrounding sub-arrays interior_keys = toolz.pipe(dask_keys, flatten, map(expand_key2), map(flatten), toolz.concat, list) interior_slices = {} overlap_blocks = {} for k in interior_keys: frac_slice = fractional_slice((name, ) + k, axes) if (name, ) + k != frac_slice: interior_slices[(getitem_name, ) + k] = frac_slice else: interior_slices[(getitem_name, ) + k] = (name, ) + k overlap_blocks[(overlap_name, ) + k] = ( concatenate3, (concrete, expand_key2((None, ) + k, name=getitem_name)), ) dsk = toolz.merge(interior_slices, overlap_blocks) return dsk