Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
198 changes: 162 additions & 36 deletions dask_image/ndmeasure/__init__.py
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
# -*- coding: utf-8 -*-

import collections
import functools
import operator
import collections
import warnings
from dask import compute, delayed
import dask.config as dask_config
Expand All @@ -11,6 +11,7 @@
import dask.bag as db
import dask.dataframe as dd
import numpy as np
from scipy.ndimage import label as ndimage_label

from . import _utils
from ._utils import _label
Expand Down Expand Up @@ -315,7 +316,12 @@ def histogram(image,
return result


def label(image, structure=None, wrap_axes=None):
def label(
image,
structure=None,
wrap_axes=None,
produce_sequential_labels=False,
):
"""
Label features in an array.

Expand All @@ -334,7 +340,6 @@ def label(image, structure=None, wrap_axes=None):
[[0,1,0],
[1,1,1],
[0,1,0]]

wrap_axes : tuple of int, optional
Whether labels should be wrapped across array boundaries, and if so
which axes.
Expand All @@ -343,6 +348,12 @@ def label(image, structure=None, wrap_axes=None):
- (0,) only wrap across the 0th axis.
- (0, 1) wrap across the 0th and 1st axis.
- (0, 1, 3) wrap across 0th, 1st and 3rd axis.
produce_sequential_labels : bool, optional
If False, the returned labels are not contiguous
(i.e. 0, 1, 6045, 54654). This is the default behavior
and has advantages for parallelism.
If True, the labels are relabeled to be sequential
and contiguous from 1 to the number of features.

Returns
-------
Expand All @@ -355,43 +366,158 @@ def label(image, structure=None, wrap_axes=None):

image = da.asarray(image)

labeled_blocks = np.empty(image.numblocks, dtype=object)
block_labeled = image.map_blocks(
lambda x: ndimage_label(x, structure=structure)[0],
dtype=_label.LABEL_DTYPE,
)

# First, label each block independently, incrementing the labels in that
# block by the total number of labels from previous blocks. This way, each
# block's labels are globally unique.
block_iter = zip(
np.ndindex(*image.numblocks),
map(functools.partial(operator.getitem, image),
da.core.slices_from_chunks(image.chunks))
)
index, input_block = next(block_iter)
labeled_blocks[index], total = _label.block_ndi_label_delayed(input_block,
structure)
for index, input_block in block_iter:
labeled_block, n = _label.block_ndi_label_delayed(input_block,
structure)
block_label_offset = da.where(labeled_block > 0,
total,
_label.LABEL_DTYPE.type(0))
labeled_block += block_label_offset
labeled_blocks[index] = labeled_block
total += n

# Put all the blocks together
block_labeled = da.block(labeled_blocks.tolist())

# Now, build a label connectivity graph that groups labels across blocks.
# We use this graph to find connected components and then relabel each
merged_labels_dict = merge_labels_across_chunk_boundaries(
block_labeled,
overlap_depth=0,
structure=structure,
wrap_axes=wrap_axes,
produce_sequential_labels=False,
)

relabeled = merged_labels_dict['labels']
mapping = merged_labels_dict['mapping']

if produce_sequential_labels:
relabeled = relabel_sequential(relabeled)
n = relabeled.max()

else:
def count_labels(label_block):
return len(np.unique(label_block[label_block > 0]))

total = block_labeled.map_blocks(
lambda x: (np.ones(
(1, ) * x.ndim) * count_labels(x)).astype(np.uint64),
dtype=np.uint64,
chunks=(1, ) * block_labeled.ndim
).sum()

n_reduced = da.from_delayed(
delayed(_label.count_n_of_collapsed_labels)(mapping),
shape=(),
dtype=np.uint64
)

n = total - n_reduced

return (relabeled, n)


def merge_labels_across_chunk_boundaries(
labels,
overlap_depth=0,
iou_threshold=0.8,
structure=None,
wrap_axes=None,
trim_overlap=True,
produce_sequential_labels=True,
):
"""
Merge labels across chunk boundaries.

Each chunk in ``labels`` has been labeled independently, and the labels
in different chunks may overlap. This function tries to merge labels across
chunk boundaries using a strategy dependent on ``overlap``:
- If ``overlap > 0``, the overlap region between chunks is used to
determine which between each pair of chunks should be merged.
- If ``overlap == 0``, labels that touch across chunk boundaries
are merged.

Parameters
----------
labels : dask array of int
The input labeled array, where each chunk is independently labeled.
overlap_depth : int, optional
The size of the overlap region between chunks, e.g. as produced by
`dask.array.overlap` or `map_overlap`. Default is 0.
iou_threshold : float, optional
If ``overlap > 0``, the intersection-over-union (IoU) between labels
in the overlap region is used to determine which labels should be
merged. If the IoU between two labels is greater than
``iou_threshold``, they are merged. Default is 0.8.
structure : array of bool, optional
Structuring element for determining connectivity. If None, a
cross-shaped structuring element is used.
wrap_axes : tuple of int, optional
Should labels be wrapped across array boundaries, and if so which axes.
- (0,) only wrap over the 0th axis.
- (0, 1) wrap over the 0th and 1st axis.
- (0, 1, 3) wrap over 0th, 1st and 3rd axis.
trim_overlap : bool, optional
If True, the overlap regions are trimmed from the output labels.
Default is True.

Returns
-------
dict with keys 'labels' and 'mapping'
'labels': dask array of int
The relabeled array, where labels have been merged across chunk
boundaries.
'mapping': dict
A mapping from old labels to new labels.
"""

# Make labels unique across blocks by encoding the block ID into
# the labels. We use half the bits to encode the block ID and half the
# bits to encode the label ID within the block.
block_labeled_unique = _label._make_labels_unique(
labels, encoding_dtype=np.uint32)

# Now, build a label mapping that groups labels across blocks.
# We use this mapping to find connected components and then relabel each
# block according to those.
label_groups = _label.label_adjacency_graph(
block_labeled, structure, total, wrap_axes=wrap_axes
all_mappings = _label.label_adjacency_mapping(
block_labeled_unique,
structure,
wrap_axes=wrap_axes,
iou_threshold=iou_threshold,
overlap_depth=overlap_depth,
)
new_labeling = _label.connected_components_delayed(label_groups)
relabeled = _label.relabel_blocks(block_labeled, new_labeling)
n = da.max(relabeled)

return (relabeled, n)
mapping = _label.connected_components_delayed(all_mappings)

relabeled = _label.relabel(
block_labeled_unique, mapping
)

relabeled = relabeled.astype(_label.LABEL_DTYPE)

if trim_overlap and overlap_depth > 0:
relabeled = da.overlap.trim_overlap(
relabeled, {i: overlap_depth for i in range(relabeled.ndim)},
boundary='none',
)

if produce_sequential_labels:
relabeled = relabel_sequential(relabeled)

return {
'labels': relabeled,
'mapping': mapping,
}


def relabel_sequential(labels):
"""
Relabel the labels in ``labels`` to be sequential starting at 1.
"""

u = da.unique(
da.concatenate(
[da.unique(b) for b in labels.blocks]
)
)
mapping = delayed(
lambda x: {k: v for k, v in zip(x, np.arange(len(x)))})(u)

relabeled = _label.relabel(labels, mapping)

return relabeled


def labeled_comprehension(image,
Expand Down
Loading
Loading