* Creates an appropriate temporary image and calls - * {@link #DoG(double[], double[], RandomAccessible, RandomAccessible, RandomAccessibleInterval, ExecutorService)} + * {@link #DoG(double[], double[], RandomAccessible, RandomAccessible, RandomAccessibleInterval)} * . *
* @@ -80,21 +90,35 @@ public class DifferenceOfGaussian * convolution). * @param dog * the Difference-of-Gaussian result image. - * @param service - * service providing threads for multi-threading */ public static < I extends NumericType< I >, T extends NumericType< T > & NativeType< T > > void DoG( final double[] sigmaSmaller, final double[] sigmaLarger, final RandomAccessible< I > input, - final RandomAccessibleInterval< T > dog, - final ExecutorService service ) + final RandomAccessibleInterval< T > dog ) { final T type = Util.getTypeFromInterval( dog ); final Img< T > g1 = Util.getArrayOrCellImgFactory( dog, type ).create( dog ); final long[] translation = new long[ dog.numDimensions() ]; dog.min( translation ); - DoG( sigmaSmaller, sigmaLarger, input, Views.translate( g1, translation ), dog, service ); + DoG( sigmaSmaller, sigmaLarger, input, Views.translate( g1, translation ), dog ); + } + + /** + * @deprecated Please use instead + * {@code Parallelization.withExecutor( service ). run( () -> DoG( sigmaSmaller, sigmaLarger, input, tmp, dog ) ); } + */ + public static < I extends NumericType< I >, T extends NumericType< T > & NativeType< T > > void DoG( + final double[] sigmaSmaller, + final double[] sigmaLarger, + final RandomAccessible< I > input, + final RandomAccessible< T > tmp, + final RandomAccessibleInterval< T > dog, + final ExecutorService service ) + { + Parallelization.runWithExecutor( service, + () -> DoG( sigmaSmaller, sigmaLarger, input, tmp, dog ) + ); } /** @@ -115,88 +139,18 @@ public static < I extends NumericType< I >, T extends NumericType< T > & NativeT * dog result image. * @param dog * the Difference-of-Gaussian result image. - * @param service - * how many threads to use for the computation. */ public static < I extends NumericType< I >, T extends NumericType< T > & NativeType< T > > void DoG( final double[] sigmaSmaller, final double[] sigmaLarger, final RandomAccessible< I > input, final RandomAccessible< T > tmp, - final RandomAccessibleInterval< T > dog, - final ExecutorService service ) + final RandomAccessibleInterval< T > dog ) { final IntervalView< T > tmpInterval = Views.interval( tmp, dog ); - try - { - Gauss3.gauss( sigmaSmaller, input, tmpInterval, service ); - Gauss3.gauss( sigmaLarger, input, dog, service ); - } - catch ( final IncompatibleTypeException e ) - { - e.printStackTrace(); - } - final IterableInterval< T > dogIterable = Views.iterable( dog ); - final IterableInterval< T > tmpIterable = Views.iterable( tmpInterval ); - final long size = dogIterable.size(); - // FIXME find better heuristic? - final int numThreads = Runtime.getRuntime().availableProcessors(); - final int numTasks = numThreads <= 1 ? 1 : numThreads * 20; - final long taskSize = size / numTasks; - final ArrayList< Future< Void > > futures = new ArrayList<>(); - for ( int taskNum = 0; taskNum < numTasks; ++taskNum ) - { - final long fromIndex = taskNum * taskSize; - final long thisTaskSize = ( taskNum == numTasks - 1 ) ? size - fromIndex : taskSize; - if ( dogIterable.iterationOrder().equals( tmpIterable.iterationOrder() ) ) - futures.add( service.submit( new Callable< Void >() - { - @Override - public Void call() - { - final Cursor< T > dogCursor = dogIterable.cursor(); - final Cursor< T > tmpCursor = tmpIterable.cursor(); - dogCursor.jumpFwd( fromIndex ); - tmpCursor.jumpFwd( fromIndex ); - for ( int i = 0; i < thisTaskSize; ++i ) - dogCursor.next().sub( tmpCursor.next() ); - return null; - } - } ) ); - else - futures.add( service.submit( new Callable< Void >() - { - @Override - public Void call() - { - final Cursor< T > dogCursor = dogIterable.localizingCursor(); - final RandomAccess< T > tmpAccess = tmpInterval.randomAccess(); - dogCursor.jumpFwd( fromIndex ); - for ( int i = 0; i < thisTaskSize; ++i ) - { - final T o = dogCursor.next(); - tmpAccess.setPosition( dogCursor ); - o.sub( tmpAccess.get() ); - } - return null; - } - } ) ); - } - for ( final Future< Void > f : futures ) - { - try - { - f.get(); - } - catch ( final InterruptedException e ) - { - e.printStackTrace(); - } - catch ( final ExecutionException e ) - { - e.printStackTrace(); - } - } + Gauss3.gauss( sigmaSmaller, input, tmpInterval ); + Gauss3.gauss( sigmaLarger, input, dog ); + LoopBuilder.setImages( dog, tmpInterval ).multiThreaded().forEachPixel( ( d, t ) -> d.sub( t ) ); } /** diff --git a/src/main/java/net/imglib2/algorithm/gauss3/Gauss3.java b/src/main/java/net/imglib2/algorithm/gauss3/Gauss3.java index 831790798..cb9a52049 100644 --- a/src/main/java/net/imglib2/algorithm/gauss3/Gauss3.java +++ b/src/main/java/net/imglib2/algorithm/gauss3/Gauss3.java @@ -43,6 +43,7 @@ import net.imglib2.algorithm.convolution.kernel.Kernel1D; import net.imglib2.algorithm.convolution.kernel.SeparableKernelConvolution; import net.imglib2.exception.IncompatibleTypeException; +import net.imglib2.parallel.Parallelization; import net.imglib2.type.numeric.NumericType; import net.imglib2.type.numeric.RealType; import net.imglib2.type.numeric.real.DoubleType; @@ -126,10 +127,9 @@ public static < S extends NumericType< S >, T extends NumericType< T > > void ga */ public static < S extends NumericType< S >, T extends NumericType< T > > void gauss( final double[] sigma, final RandomAccessible< S > source, final RandomAccessibleInterval< T > target ) throws IncompatibleTypeException { - final int numthreads = Runtime.getRuntime().availableProcessors(); - final ExecutorService service = Executors.newFixedThreadPool( numthreads ); - gauss( sigma, source, target, service ); - service.shutdown(); + final double[][] halfkernels = halfkernels( sigma ); + final Convolution< NumericType< ? > > convolution = SeparableKernelConvolution.convolution( Kernel1D.symmetric( halfkernels ) ); + convolution.process( source, target ); } /** @@ -201,10 +201,9 @@ public static < S extends NumericType< S >, T extends NumericType< T > > void ga */ public static < S extends NumericType< S >, T extends NumericType< T > > void gauss( final double[] sigma, final RandomAccessible< S > source, final RandomAccessibleInterval< T > target, final ExecutorService service ) throws IncompatibleTypeException { - final double[][] halfkernels = halfkernels( sigma ); - final Convolution< NumericType< ? > > convolution = SeparableKernelConvolution.convolution( Kernel1D.symmetric( halfkernels ) ); - convolution.setExecutor( service ); - convolution.process( source, target ); + Parallelization.runWithExecutor( service, + () -> gauss( sigma, source, target ) + ); } public static double[][] halfkernels( final double[] sigma ) diff --git a/src/main/java/net/imglib2/algorithm/gradient/PartialDerivative.java b/src/main/java/net/imglib2/algorithm/gradient/PartialDerivative.java index 1eba0287f..b25b6aeda 100644 --- a/src/main/java/net/imglib2/algorithm/gradient/PartialDerivative.java +++ b/src/main/java/net/imglib2/algorithm/gradient/PartialDerivative.java @@ -34,21 +34,18 @@ package net.imglib2.algorithm.gradient; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import net.imglib2.Cursor; -import net.imglib2.FinalInterval; import net.imglib2.RandomAccessible; import net.imglib2.RandomAccessibleInterval; import net.imglib2.loops.LoopBuilder; +import net.imglib2.parallel.TaskExecutor; +import net.imglib2.parallel.Parallelization; +import net.imglib2.parallel.TaskExecutors; import net.imglib2.type.numeric.NumericType; import net.imglib2.util.Intervals; -import net.imglib2.view.IntervalView; import net.imglib2.view.Views; /** @@ -98,7 +95,7 @@ public static < T extends NumericType< T > > void gradientCentralDifference2( fi * @param source * source image, has to provide valid data in the interval of the * gradient image plus a one pixel border in dimension. - * @param gradient + * @param result * output image * @param dimension * along which dimension the partial derivatives are computed @@ -110,57 +107,15 @@ public static < T extends NumericType< T > > void gradientCentralDifference2( fi */ public static < T extends NumericType< T > > void gradientCentralDifferenceParallel( final RandomAccessible< T > source, - final RandomAccessibleInterval< T > gradient, + final RandomAccessibleInterval< T > result, final int dimension, final int nTasks, final ExecutorService es ) throws InterruptedException, ExecutionException { - final int nDim = source.numDimensions(); - if ( nDim < 2 ) - { - gradientCentralDifference( source, gradient, dimension ); - return; - } - - long dimensionMax = Long.MIN_VALUE; - int dimensionArgMax = -1; - - for ( int d = 0; d < nDim; ++d ) - { - final long size = gradient.dimension( d ); - if ( d != dimension && size > dimensionMax ) - { - dimensionMax = size; - dimensionArgMax = d; - } - } - - final long stepSize = Math.max( dimensionMax / nTasks, 1 ); - final long stepSizeMinusOne = stepSize - 1; - final long min = gradient.min( dimensionArgMax ); - final long max = gradient.max( dimensionArgMax ); - - final ArrayList< Callable< Void > > tasks = new ArrayList<>(); - for ( long currentMin = min, minZeroBase = 0; minZeroBase < dimensionMax; currentMin += stepSize, minZeroBase += stepSize ) - { - final long currentMax = Math.min( currentMin + stepSizeMinusOne, max ); - final long[] mins = new long[ nDim ]; - final long[] maxs = new long[ nDim ]; - gradient.min( mins ); - gradient.max( maxs ); - mins[ dimensionArgMax ] = currentMin; - maxs[ dimensionArgMax ] = currentMax; - final IntervalView< T > currentInterval = Views.interval( gradient, new FinalInterval( mins, maxs ) ); - tasks.add( () -> { - gradientCentralDifference( source, currentInterval, dimension ); - return null; - } ); - } - - final List< Future< Void > > futures = es.invokeAll( tasks ); - - for ( final Future< Void > f : futures ) - f.get(); + TaskExecutor taskExecutor = TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ); + Parallelization.runWithExecutor( taskExecutor, () -> { + gradientCentralDifference( source, result, dimension ); + } ); } // fast version @@ -184,7 +139,7 @@ public static < T extends NumericType< T > > void gradientCentralDifference( fin final RandomAccessibleInterval< T > back = Views.interval( source, Intervals.translate( result, -1, dimension ) ); final RandomAccessibleInterval< T > front = Views.interval( source, Intervals.translate( result, 1, dimension ) ); - LoopBuilder.setImages( result, back, front ).forEachPixel( ( r, b, f ) -> { + LoopBuilder.setImages( result, back, front ).multiThreaded().forEachPixel( ( r, b, f ) -> { r.set( f ); r.sub( b ); r.mul( 0.5 ); @@ -207,7 +162,7 @@ public static < T extends NumericType< T > > void gradientBackwardDifference( fi final RandomAccessibleInterval< T > back = Views.interval( source, Intervals.translate( result, -1, dimension ) ); final RandomAccessibleInterval< T > front = Views.interval( source, result ); - LoopBuilder.setImages( result, back, front ).forEachPixel( ( r, b, f ) -> { + LoopBuilder.setImages( result, back, front ).multiThreaded().forEachPixel( ( r, b, f ) -> { r.set( f ); r.sub( b ); } ); @@ -229,7 +184,7 @@ public static < T extends NumericType< T > > void gradientForwardDifference( fin final RandomAccessibleInterval< T > back = Views.interval( source, result ); final RandomAccessibleInterval< T > front = Views.interval( source, Intervals.translate( result, 1, dimension ) ); - LoopBuilder.setImages( result, back, front ).forEachPixel( ( r, b, f ) -> { + LoopBuilder.setImages( result, back, front ).multiThreaded().forEachPixel( ( r, b, f ) -> { r.set( f ); r.sub( b ); } ); diff --git a/src/main/java/net/imglib2/algorithm/labeling/ConnectedComponents.java b/src/main/java/net/imglib2/algorithm/labeling/ConnectedComponents.java index 38af4dbd2..282dac7c6 100644 --- a/src/main/java/net/imglib2/algorithm/labeling/ConnectedComponents.java +++ b/src/main/java/net/imglib2/algorithm/labeling/ConnectedComponents.java @@ -39,12 +39,8 @@ import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import net.imglib2.Cursor; import net.imglib2.FinalInterval; @@ -52,6 +48,7 @@ import net.imglib2.RandomAccessible; import net.imglib2.RandomAccessibleInterval; import net.imglib2.iterator.IntervalIterator; +import net.imglib2.parallel.Parallelization; import net.imglib2.roi.labeling.ImgLabeling; import net.imglib2.roi.labeling.LabelingMapping; import net.imglib2.type.numeric.IntegerType; @@ -100,17 +97,19 @@ public CollectNeighborLabelsFactory getFactory() * @param se * structuring element to use. 8-connected or 4-connected * (respectively n-dimensional analog) + * @param service + * service providing threads for multi-threading */ public static < T extends IntegerType< T >, L, I extends IntegerType< I > > void labelAllConnectedComponents( final RandomAccessible< T > input, final ImgLabeling< L, I > labeling, final Iterator< L > labelGenerator, - final StructuringElement se ) + final StructuringElement se, + final ExecutorService service ) { - final int numThreads = Runtime.getRuntime().availableProcessors(); - final ExecutorService service = Executors.newFixedThreadPool( numThreads ); - labelAllConnectedComponents( input, labeling, labelGenerator, se, service ); - service.shutdown(); + Parallelization.runWithExecutor( service, + () -> labelAllConnectedComponents( input, labeling, labelGenerator, se ) + ); } /** @@ -131,21 +130,18 @@ public static < T extends IntegerType< T >, L, I extends IntegerType< I > > void * @param se * structuring element to use. 8-connected or 4-connected * (respectively n-dimensional analog) - * @param service - * service providing threads for multi-threading */ public static < T extends IntegerType< T >, L, I extends IntegerType< I > > void labelAllConnectedComponents( final RandomAccessible< T > input, final ImgLabeling< L, I > labeling, final Iterator< L > labelGenerator, - final StructuringElement se, - final ExecutorService service ) + final StructuringElement se ) { final RandomAccessibleInterval< I > output = labeling.getIndexImg(); for ( final I i : Views.iterable( output ) ) i.setZero(); - final int numLabels = labelAllConnectedComponents( input, output, se, service ) + 1; + final int numLabels = labelAllConnectedComponents( input, output, se ) + 1; final ArrayList< Set< L > > labelSets = new ArrayList< Set< L > >(); labelSets.add( new HashSet< L >() ); @@ -170,7 +166,6 @@ public static < T extends IntegerType< T >, L, I extends IntegerType< I > > void * ** Note, that the {@code output} image must be cleared to 0! - *
* * @param input * input image with pixels > 0 belonging to foreground. @@ -179,19 +174,20 @@ public static < T extends IntegerType< T >, L, I extends IntegerType< I > > void * @param se * structuring element to use. 8-connected or 4-connected * (respectively n-dimensional analog) + * @param service + * service providing threads for multi-threading * @return the number of connected components (that is, the highest value * occurring in the output image. */ public static < T extends IntegerType< T >, L extends IntegerType< L > > int labelAllConnectedComponents( final RandomAccessible< T > input, final RandomAccessibleInterval< L > output, - final StructuringElement se ) + final StructuringElement se, + final ExecutorService service ) { - final int numThreads = Runtime.getRuntime().availableProcessors(); - final ExecutorService service = Executors.newFixedThreadPool( numThreads ); - final int result = labelAllConnectedComponents( input, output, se, service ); - service.shutdown(); - return result; + return Parallelization.runWithExecutor( service, + () -> labelAllConnectedComponents( input, output, se ) + ); } /** @@ -201,6 +197,7 @@ public static < T extends IntegerType< T >, L extends IntegerType< L > > int lab * ** Note, that the {@code output} image must be cleared to 0! + *
* * @param input * input image with pixels > 0 belonging to foreground. @@ -209,16 +206,13 @@ public static < T extends IntegerType< T >, L extends IntegerType< L > > int lab * @param se * structuring element to use. 8-connected or 4-connected * (respectively n-dimensional analog) - * @param service - * service providing threads for multi-threading * @return the number of connected components (that is, the highest value * occurring in the output image. */ public static < T extends IntegerType< T >, L extends IntegerType< L > > int labelAllConnectedComponents( final RandomAccessible< T > input, final RandomAccessibleInterval< L > output, - final StructuringElement se, - final ExecutorService service ) + final StructuringElement se ) { final int n = output.numDimensions(); final int splitDim = n - 1; @@ -243,37 +237,14 @@ public static < T extends IntegerType< T >, L extends IntegerType< L > > int lab min[ splitDim ] += taskSize; } - final ArrayList< Future< ? > > futures = new ArrayList< Future< ? > >(); - for ( final Fragment< T, L > fragment : fragments ) - { - futures.add( service.submit( new Runnable() - { - @Override - public void run() - { - fragment.mark(); - } - } ) ); - } - getAllFutures( futures ); + Parallelization.getTaskExecutor().forEach( Arrays.asList( fragments ), Fragment::mark ); final TIntArrayList merged = mergeCanonicalLists( fragments ); for ( int i = 1; i < numTasks; ++i ) fragments[ i ].linkToPreviousFragment( fragments[ i - 1 ], merged ); final int numComponents = splitCanonicalLists( fragments, merged ); - for ( final Fragment< T, L > fragment : fragments ) - { - futures.add( service.submit( new Runnable() - { - @Override - public void run() - { - fragment.relabel(); - } - } ) ); - } - getAllFutures( futures ); + Parallelization.getTaskExecutor().forEach( Arrays.asList( fragments ), Fragment::relabel ); return numComponents; } @@ -490,26 +461,6 @@ private static < T extends IntegerType< T >, L extends IntegerType< L > > int sp return nextLabel - 1; } - private static void getAllFutures( final List< Future< ? > > futures ) - { - for ( final Future< ? > future : futures ) - { - try - { - future.get(); - } - catch ( final InterruptedException e ) - { - e.printStackTrace(); - } - catch ( final ExecutionException e ) - { - e.printStackTrace(); - } - } - futures.clear(); - } - private static interface CollectNeighborLabels< L extends IntegerType< L > > { public void collect( RandomAccess< L > la, final TIntArrayList neighborLabels, final long[] labelsMin, final long[] labelsMax ); diff --git a/src/main/java/net/imglib2/algorithm/linalg/eigen/TensorEigenValues.java b/src/main/java/net/imglib2/algorithm/linalg/eigen/TensorEigenValues.java index b654fce53..d9022e018 100644 --- a/src/main/java/net/imglib2/algorithm/linalg/eigen/TensorEigenValues.java +++ b/src/main/java/net/imglib2/algorithm/linalg/eigen/TensorEigenValues.java @@ -34,24 +34,19 @@ package net.imglib2.algorithm.linalg.eigen; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import net.imglib2.Cursor; -import net.imglib2.FinalInterval; import net.imglib2.RandomAccessibleInterval; import net.imglib2.img.Img; import net.imglib2.img.ImgFactory; +import net.imglib2.loops.LoopBuilder; +import net.imglib2.parallel.Parallelization; +import net.imglib2.parallel.TaskExecutors; import net.imglib2.type.numeric.ComplexType; import net.imglib2.type.numeric.RealType; -import net.imglib2.view.IntervalView; import net.imglib2.view.Views; -import net.imglib2.view.composite.NumericComposite; -import net.imglib2.view.composite.RealComposite; +import net.imglib2.view.composite.CompositeIntervalView; +import net.imglib2.view.composite.GenericComposite; /** * @@ -286,71 +281,10 @@ public static < T extends RealType< T >, U extends ComplexType< U > > RandomAcce final ExecutorService es ) { - assert nTasks > 0: "Passed nTasks < 1"; - - final int tensorDims = tensor.numDimensions(); - - long dimensionMax = Long.MIN_VALUE; - int dimensionArgMax = -1; - - for ( int d = 0; d < tensorDims - 1; ++d ) - { - final long size = tensor.dimension( d ); - if ( size > dimensionMax ) - { - dimensionMax = size; - dimensionArgMax = d; - } - } - - final long stepSize = Math.max( dimensionMax / nTasks, 1 ); - final long stepSizeMinusOne = stepSize - 1; - final long max = dimensionMax - 1; - - final ArrayList< Callable< RandomAccessibleInterval< U > > > tasks = new ArrayList<>(); - for ( long currentMin = 0; currentMin < dimensionMax; currentMin += stepSize ) - { - final long currentMax = Math.min( currentMin + stepSizeMinusOne, max ); - final long[] minT = new long[ tensorDims ]; - final long[] maxT = new long[ tensorDims ]; - final long[] minE = new long[ tensorDims ]; - final long[] maxE = new long[ tensorDims ]; - tensor.min( minT ); - tensor.max( maxT ); - eigenvalues.min( minE ); - eigenvalues.max( maxE ); - minE[ dimensionArgMax ] = minT[ dimensionArgMax ] = currentMin; - maxE[ dimensionArgMax ] = maxT[ dimensionArgMax ] = currentMax; - final IntervalView< T > currentTensor = Views.interval( tensor, new FinalInterval( minT, maxT ) ); - final IntervalView< U > currentEigenvalues = Views.interval( eigenvalues, new FinalInterval( minE, maxE ) ); - tasks.add( () -> calculateEigenValuesImpl( currentTensor, currentEigenvalues, ev.copy() ) ); - } - - - try - { - final List< Future< RandomAccessibleInterval< U > > > futures = es.invokeAll( tasks ); - for ( final Future< RandomAccessibleInterval< U > > f : futures ) - try - { - f.get(); - } - catch ( final ExecutionException e ) - { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - catch ( final InterruptedException e ) - { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - return eigenvalues; - - + assert nTasks > 0 : "Passed nTasks < 1"; + return Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ), + () -> calculateEigenValues( tensor, eigenvalues, ev ) ); } private static < T extends RealType< T >, U extends ComplexType< U > > RandomAccessibleInterval< U > calculateEigenValuesImpl( @@ -358,10 +292,15 @@ private static < T extends RealType< T >, U extends ComplexType< U > > RandomAcc final RandomAccessibleInterval< U > eigenvalues, final EigenValues< T, U > ev ) { - final Cursor< RealComposite< T > > m = Views.iterable( Views.collapseReal( tensor ) ).cursor(); - final Cursor< NumericComposite< U > > e = Views.iterable( Views.collapseNumeric( eigenvalues ) ).cursor(); - while ( m.hasNext() ) - ev.compute( m.next(), e.next() ); + RandomAccessibleInterval< ? extends GenericComposite< T > > tensorVectors = Views.collapse( tensor ); + CompositeIntervalView< U, ? extends GenericComposite< U > > eigenvaluesVectors = Views.collapse( eigenvalues ); + LoopBuilder.setImages( tensorVectors, eigenvaluesVectors ) + .multiThreaded() + .forEachChunk( chunk -> { + EigenValues< T, U > copy = ev.copy(); + chunk.forEachPixel( copy::compute ); + return null; + } ); return eigenvalues; } diff --git a/src/main/java/net/imglib2/algorithm/localextrema/LocalExtrema.java b/src/main/java/net/imglib2/algorithm/localextrema/LocalExtrema.java index ef9b69185..23cf4cbd5 100644 --- a/src/main/java/net/imglib2/algorithm/localextrema/LocalExtrema.java +++ b/src/main/java/net/imglib2/algorithm/localextrema/LocalExtrema.java @@ -33,33 +33,35 @@ */ package net.imglib2.algorithm.localextrema; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.stream.IntStream; -import java.util.stream.LongStream; - -import net.imglib2.Cursor; -import net.imglib2.FinalInterval; import net.imglib2.Interval; import net.imglib2.Localizable; import net.imglib2.Point; +import net.imglib2.RandomAccess; import net.imglib2.RandomAccessible; import net.imglib2.RandomAccessibleInterval; import net.imglib2.Sampler; import net.imglib2.algorithm.neighborhood.Neighborhood; import net.imglib2.algorithm.neighborhood.RectangleShape; import net.imglib2.algorithm.neighborhood.Shape; +import net.imglib2.converter.readwrite.WriteConvertedRandomAccessible; +import net.imglib2.loops.LoopBuilder; +import net.imglib2.parallel.Parallelization; +import net.imglib2.parallel.TaskExecutor; +import net.imglib2.parallel.TaskExecutors; import net.imglib2.util.ConstantUtils; -import net.imglib2.util.Intervals; import net.imglib2.util.ValuePair; import net.imglib2.view.IntervalView; import net.imglib2.view.Views; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + /** * Provides {@link #findLocalExtrema} to find pixels that are extrema in their * local neighborhood. @@ -320,38 +322,8 @@ public static < P, T > List< P > findLocalExtrema( final int numTasks, final int splitDim ) throws InterruptedException, ExecutionException { - - final long[] min = Intervals.minAsLongArray( interval ); - final long[] max = Intervals.maxAsLongArray( interval ); - - final long splitDimSize = interval.dimension( splitDim ); - final long splitDimMax = max[ splitDim ]; - final long splitDimMin = min[ splitDim ]; - final long taskSize = Math.max( splitDimSize / numTasks, 1 ); - - final ArrayList< Callable< List< P > > > tasks = new ArrayList<>(); - - for ( long start = splitDimMin, stop = splitDimMin + taskSize - 1; start <= splitDimMax; start += taskSize, stop += taskSize ) - { - final long s = start; - // need max here instead of dimension for constructor of - // FinalInterval - final long S = Math.min( stop, splitDimMax ); - tasks.add( () -> { - final long[] localMin = min.clone(); - final long[] localMax = max.clone(); - localMin[ splitDim ] = s; - localMax[ splitDim ] = S; - return findLocalExtrema( source, new FinalInterval( localMin, localMax ), localNeighborhoodCheck, shape ); - } ); - } - - final ArrayList< P > extrema = new ArrayList<>(); - final List< Future< List< P > > > futures = service.invokeAll( tasks ); - for ( final Future< List< P > > f : futures ) - extrema.addAll( f.get() ); - return extrema; - + TaskExecutor taskExecutor = TaskExecutors.forExecutorServiceAndNumTasks( service, numTasks ); + return Parallelization.runWithExecutor( taskExecutor, () -> findLocalExtrema( source, interval, localNeighborhoodCheck, shape ) ); } /** @@ -470,22 +442,28 @@ public static < P, T > List< P > findLocalExtrema( final LocalNeighborhoodCheck< P, T > localNeighborhoodCheck, final Shape shape ) { + WriteConvertedRandomAccessible< T, RandomAccess< T > > randomAccessible = new WriteConvertedRandomAccessible<>( source, sampler -> (RandomAccess< T >) sampler ); + RandomAccessibleInterval< RandomAccess< T > > centers = Views.interval( randomAccessible, interval); + RandomAccessibleInterval< Neighborhood< T > > neighborhoods = Views.interval( shape.neighborhoodsRandomAccessible( source ), interval ); + List< List< P > > extremas = LoopBuilder.setImages( centers, neighborhoods ).multiThreaded().forEachChunk( chunk -> { + List< P > extrema = new ArrayList<>(); + chunk.forEachPixel( ( center, neighborhood ) -> { + P p = localNeighborhoodCheck.check( center, neighborhood ); + if ( p != null ) + extrema.add( p ); + } ); + return extrema; + } ); + return concatenate( extremas ); + } - final IntervalView< T > sourceInterval = Views.interval( source, interval ); - - final ArrayList< P > extrema = new ArrayList<>(); - - final Cursor< T > center = Views.flatIterable( sourceInterval ).cursor(); - for ( final Neighborhood< T > neighborhood : shape.neighborhoods( sourceInterval ) ) - { - center.fwd(); - final P p = localNeighborhoodCheck.check( center, neighborhood ); - if ( p != null ) - extrema.add( p ); - } - - return extrema; - + private static < P > List concatenate( Collection
*
* A List {@link RefinedPeak} for the given list of {@link Localizable} is
* computed by, for each peak, fitting a quadratic function to the image and
@@ -68,7 +68,7 @@
* repeated at the corresponding integer coordinates. This is repeated to
* convergence, for a maximum number of iterations, or until the integer
* coordinates move out of the valid image.
- *
+ *
* @author Stephan Preibisch
* @author Tobias Pietzsch
*/
@@ -93,8 +93,6 @@ public SubpixelLocalization( final int numDimensions )
// principally one can move in any dimension
allowedToMoveInDim = new boolean[ numDimensions ];
Arrays.fill( allowedToMoveInDim, true );
-
- numThreads = Runtime.getRuntime().availableProcessors();
}
public void setAllowMaximaTolerance( final boolean allowMaximaTolerance )
@@ -164,14 +162,14 @@ public boolean getReturnInvalidPeaks()
public int getNumThreads()
{
- return numThreads;
+ return numThreads == 0 ? Parallelization.getTaskExecutor().getParallelism() : numThreads;
}
/**
* Refine a set of peaks to subpixel coordinates. Calls
* {@link #refinePeaks(List, RandomAccessible, Interval, boolean, int, boolean, float, boolean[], int)}
* with the parameters set to this object.
- *
+ *
* @param peaks
* List of integer peaks.
* @param img
@@ -184,7 +182,13 @@ public int getNumThreads()
*/
public ArrayList< RefinedPeak< P > > process( final List< P > peaks, final RandomAccessible< T > img, final Interval validInterval )
{
- return refinePeaks( peaks, img, validInterval, returnInvalidPeaks, maxNumMoves, allowMaximaTolerance, maximaTolerance, allowedToMoveInDim, numThreads );
+ if ( numThreads != 0 )
+ return Parallelization.runWithNumThreads( numThreads,
+ () -> refinePeaks( peaks, img, validInterval, returnInvalidPeaks, maxNumMoves,
+ allowMaximaTolerance, maximaTolerance, allowedToMoveInDim ) );
+ else
+ return refinePeaks( peaks, img, validInterval, returnInvalidPeaks, maxNumMoves,
+ allowMaximaTolerance, maximaTolerance, allowedToMoveInDim );
}
/**
@@ -197,7 +201,7 @@ public ArrayList< RefinedPeak< P > > process( final List< P > peaks, final Rando
* fit is repeated at the corresponding integer coordinates. This is
* repeated to convergence, for a maximum number of iterations, or until the
* integer coordinates move out of the valid image.
- *
+ *
* @param peaks
* List of integer peaks.
* @param img
@@ -231,45 +235,9 @@ public static < T extends RealType< T >, P extends Localizable > ArrayList< Refi
final int maxNumMoves, final boolean allowMaximaTolerance, final float maximaTolerance, final boolean[] allowedToMoveInDim,
final int numThreads )
{
- final int numPeaks = peaks.size();
- final ArrayList< RefinedPeak< P > > allRefinedPeaks = new ArrayList< RefinedPeak< P > >( numPeaks );
-
- if ( numPeaks == 0 )
- return allRefinedPeaks;
-
- final int numTasks = numThreads <= 1 ? 1 : ( int ) Math.min( numPeaks, numThreads * 20 );
- final int taskSize = numPeaks / numTasks;
-
- final ExecutorService ex = Executors.newFixedThreadPool( numThreads );
- final List< RefinedPeak< P > > synchronizedAllRefinedPeaks = Collections.synchronizedList( allRefinedPeaks );
- for ( int taskNum = 0; taskNum < numTasks; ++taskNum )
- {
- final int fromIndex = taskNum * taskSize;
- final int toIndex = ( taskNum == numTasks - 1 ) ? numPeaks : fromIndex + taskSize;
- final Runnable r = new Runnable()
- {
- @Override
- public void run()
- {
- final ArrayList< RefinedPeak< P > > refinedPeaks = refinePeaks(
- peaks.subList( fromIndex, toIndex ),
- img, validInterval, returnInvalidPeaks, maxNumMoves, allowMaximaTolerance, maximaTolerance, allowedToMoveInDim );
- synchronizedAllRefinedPeaks.addAll( refinedPeaks );
- }
- };
- ex.execute( r );
- }
- ex.shutdown();
- try
- {
- ex.awaitTermination( 1000, TimeUnit.DAYS );
- }
- catch ( final InterruptedException e )
- {
- e.printStackTrace();
- }
-
- return allRefinedPeaks;
+ return Parallelization.runWithNumThreads( numThreads,
+ () -> refinePeaks( peaks, img, validInterval, returnInvalidPeaks, maxNumMoves, allowMaximaTolerance, maximaTolerance,
+ allowedToMoveInDim ) );
}
/**
@@ -282,7 +250,7 @@ public void run()
* fit is repeated at the corresponding integer coordinates. This is
* repeated to convergence, for a maximum number of iterations, or until the
* integer coordinates move out of the valid image.
- *
+ *
* @param peaks
* List of integer peaks.
* @param img
@@ -313,7 +281,36 @@ public static < T extends RealType< T >, P extends Localizable > ArrayList< Refi
final List< P > peaks, final RandomAccessible< T > img, final Interval validInterval, final boolean returnInvalidPeaks,
final int maxNumMoves, final boolean allowMaximaTolerance, final float maximaTolerance, final boolean[] allowedToMoveInDim )
{
- final ArrayList< RefinedPeak< P >> refinedPeaks = new ArrayList< RefinedPeak< P > >();
+ final int numPeaks = peaks.size();
+
+ if ( numPeaks == 0 )
+ return new ArrayList<>();
+
+ TaskExecutor taskExecutor = Parallelization.getTaskExecutor();
+
+ List< Interval > chunks = IntervalChunks.chunkInterval( new FinalInterval( numPeaks ), taskExecutor.suggestNumberOfTasks() );
+
+ List< ArrayList< RefinedPeak< P > > > result = taskExecutor.forEachApply( chunks, chunk ->
+ refinePeaksChunk(
+ peaks.subList( ( int ) chunk.min( 0 ), ( int ) chunk.max( 0 ) + 1 ),
+ img, validInterval, returnInvalidPeaks, maxNumMoves, allowMaximaTolerance,
+ maximaTolerance, allowedToMoveInDim )
+ );
+
+ return concatenate( result );
+ }
+
+ private static < P > ArrayList< P > concatenate( List< ? extends List< ? extends P > > lists )
+ {
+ int size = lists.stream().mapToInt( List::size ).sum();
+ ArrayList< P > result = new ArrayList<>( size );
+ lists.forEach( result::addAll );
+ return result;
+ }
+
+ private static < T extends RealType< T >, P extends Localizable > ArrayList< RefinedPeak< P > > refinePeaksChunk( List< P > peaks, RandomAccessible< T > img, Interval validInterval, boolean returnInvalidPeaks, int maxNumMoves, boolean allowMaximaTolerance, float maximaTolerance, boolean[] allowedToMoveInDim )
+ {
+ final ArrayList< RefinedPeak< P > > refinedPeaks = new ArrayList< RefinedPeak< P > >();
final int n = img.numDimensions();
@@ -420,7 +417,7 @@ else if ( returnInvalidPeaks )
/**
* Estimate subpixel {@code offset} of extremum of quadratic function
* fitted at {@code p}.
- *
+ *
* @param p
* integer position at which to fit quadratic.
* @param access
diff --git a/src/main/java/net/imglib2/algorithm/loop/FastCursorRandomAccessLoops.java b/src/main/java/net/imglib2/algorithm/loop/FastCursorRandomAccessLoops.java
new file mode 100644
index 000000000..93109946d
--- /dev/null
+++ b/src/main/java/net/imglib2/algorithm/loop/FastCursorRandomAccessLoops.java
@@ -0,0 +1,160 @@
+/*
+ * #%L
+ * ImgLib2: a general-purpose, multidimensional image processing library.
+ * %%
+ * Copyright (C) 2009 - 2018 Tobias Pietzsch, Stephan Preibisch, Stephan Saalfeld,
+ * John Bogovic, Albert Cardona, Barry DeZonia, Christian Dietz, Jan Funke,
+ * Aivar Grislis, Jonathan Hale, Grant Harris, Stefan Helfrich, Mark Hiner,
+ * Martin Horn, Steffen Jaensch, Lee Kamentsky, Larry Lindsey, Melissa Linkert,
+ * Mark Longair, Brian Northan, Nick Perry, Curtis Rueden, Johannes Schindelin,
+ * Jean-Yves Tinevez and Michael Zinsmaier.
+ * %%
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright notice,
+ * this list of conditions and the following disclaimer in the documentation
+ * and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ * #L%
+ */
+
+package net.imglib2.algorithm.loop;
+
+import net.imglib2.Cursor;
+import net.imglib2.RandomAccess;
+import net.imglib2.loops.ClassCopyProvider;
+import net.imglib2.loops.LoopBuilder;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+final class FastCursorRandomAccessLoops
+{
+ private FastCursorRandomAccessLoops()
+ {
+ // prevent from instantiation
+ }
+
+ static void loop( final Object action, long n, final Cursor< ? > cursor, final List< ? extends RandomAccess< ? > > randomAccesses )
+ {
+ createLoop( action, cursor, randomAccesses ).accept( n );
+ }
+
+ private static final List< ClassCopyProvider< LongConsumer > > factories = Arrays.asList(
+ new ClassCopyProvider<>( OneCursorLoop.class, LongConsumer.class ),
+ new ClassCopyProvider<>( TwoCursorLoop.class, LongConsumer.class ),
+ new ClassCopyProvider<>( ThreeCursorLoop.class, LongConsumer.class ) );
+
+ private static LongConsumer createLoop( final Object action, final Cursor< ? > cursor, final List< ? extends RandomAccess< ? > > randomAccesses )
+ {
+ final Object[] arguments = Stream.concat( Stream.of( action, cursor ), randomAccesses.stream() ).toArray();
+ for ( final ClassCopyProvider< LongConsumer > factory : factories )
+ if ( factory.matches( arguments ) )
+ {
+ final List< Class< ? > > key = Stream.of( arguments ).map( Object::getClass ).collect( Collectors.toList() );
+ return factory.newInstanceForKey( key, arguments );
+ }
+ throw new IllegalArgumentException();
+ }
+
+ public static class OneCursorLoop< A > implements LongConsumer
+ {
+
+ private final Consumer< A > action;
+
+ private final Cursor< A > cursorA;
+
+ public OneCursorLoop( final Consumer< A > action, final Cursor< A > cursorA )
+ {
+ this.action = action;
+ this.cursorA = cursorA;
+ }
+
+ @Override
+ public void accept( long n )
+ {
+ while ( --n >= 0 )
+ action.accept( cursorA.next() );
+ }
+ }
+
+ public static class TwoCursorLoop< A, B > implements LongConsumer
+ {
+
+ private final BiConsumer< A, B > action;
+
+ private final Cursor< A > cursorA;
+
+ private final RandomAccess< B > randomAccessB;
+
+ public TwoCursorLoop( final BiConsumer< A, B > action, final Cursor< A > cursorA, final RandomAccess< B > randomAccessB )
+ {
+ this.action = action;
+ this.cursorA = cursorA;
+ this.randomAccessB = randomAccessB;
+ }
+
+ @Override
+ public void accept( long n )
+ {
+ while ( --n >= 0 )
+ {
+ A a = cursorA.next();
+ randomAccessB.setPosition( cursorA );
+ action.accept( a, randomAccessB.get() );
+ }
+ }
+ }
+
+ public static class ThreeCursorLoop< A, B, C > implements LongConsumer
+ {
+
+ private final LoopBuilder.TriConsumer< A, B, C > action;
+
+ private final Cursor< A > cursorA;
+
+ private final RandomAccess< B > randomAccessB;
+
+ private final RandomAccess< C > randomAccessC;
+
+ public ThreeCursorLoop( final LoopBuilder.TriConsumer< A, B, C > action, final Cursor< A > cursorA, final RandomAccess< B > randomAccessB, final RandomAccess< C > randomAccessC )
+ {
+ this.action = action;
+ this.cursorA = cursorA;
+ this.randomAccessB = randomAccessB;
+ this.randomAccessC = randomAccessC;
+ }
+
+ @Override
+ public void accept( long n )
+ {
+ while ( --n >= 0 )
+ {
+ A a = cursorA.next();
+ randomAccessB.setPosition( cursorA );
+ randomAccessC.setPosition( cursorA );
+ action.accept( a, randomAccessB.get(), randomAccessC.get() );
+ }
+ }
+ }
+
+}
diff --git a/src/main/java/net/imglib2/algorithm/loop/IterableLoopBuilder.java b/src/main/java/net/imglib2/algorithm/loop/IterableLoopBuilder.java
new file mode 100644
index 000000000..6f85dd7d0
--- /dev/null
+++ b/src/main/java/net/imglib2/algorithm/loop/IterableLoopBuilder.java
@@ -0,0 +1,143 @@
+/*
+ * #%L
+ * ImgLib2: a general-purpose, multidimensional image processing library.
+ * %%
+ * Copyright (C) 2009 - 2018 Tobias Pietzsch, Stephan Preibisch, Stephan Saalfeld,
+ * John Bogovic, Albert Cardona, Barry DeZonia, Christian Dietz, Jan Funke,
+ * Aivar Grislis, Jonathan Hale, Grant Harris, Stefan Helfrich, Mark Hiner,
+ * Martin Horn, Steffen Jaensch, Lee Kamentsky, Larry Lindsey, Melissa Linkert,
+ * Mark Longair, Brian Northan, Nick Perry, Curtis Rueden, Johannes Schindelin,
+ * Jean-Yves Tinevez and Michael Zinsmaier.
+ * %%
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright notice,
+ * this list of conditions and the following disclaimer in the documentation
+ * and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ * #L%
+ */
+
+package net.imglib2.algorithm.loop;
+
+import net.imglib2.Cursor;
+import net.imglib2.FinalInterval;
+import net.imglib2.Interval;
+import net.imglib2.IterableInterval;
+import net.imglib2.RandomAccess;
+import net.imglib2.RandomAccessible;
+import net.imglib2.loops.IntervalChunks;
+import net.imglib2.loops.LoopBuilder;
+import net.imglib2.parallel.Parallelization;
+import net.imglib2.parallel.TaskExecutor;
+import net.imglib2.parallel.TaskExecutors;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Similar to {@link LoopBuilder}, but accepts {@link IterableInterval} as first
+ * argument.
+ */
+public class IterableLoopBuilder< T >
+{
+
+ private TaskExecutor taskExecutor = TaskExecutors.singleThreaded();
+
+ private final IterableInterval< ? > firstImage;
+
+ private final List< RandomAccessible< ? > > otherImages;
+
+ private IterableLoopBuilder( IterableInterval< ? > firstImage, RandomAccessible< ? > ... otherImages )
+ {
+ this.firstImage = firstImage;
+ this.otherImages = Arrays.asList( otherImages );
+ }
+
+ public static < A > IterableLoopBuilder< Consumer< A > > setImages( IterableInterval< A > a )
+ {
+ return new IterableLoopBuilder<>( a );
+ }
+
+ public static < A, B > IterableLoopBuilder< BiConsumer< A, B > > setImages( IterableInterval< A > a, RandomAccessible< B > b )
+ {
+ return new IterableLoopBuilder<>( a, b );
+ }
+
+ public static < A, B, C > IterableLoopBuilder< LoopBuilder.TriConsumer< A, B, C > > setImages( IterableInterval< A > a, RandomAccessible< B > b, RandomAccessible< C > c )
+ {
+ return new IterableLoopBuilder<>( a, b, c );
+ }
+
+
+ public void forEachPixel( T action )
+ {
+ forEachChunk( chunk -> {
+ chunk.forEachPixel( action );
+ return null;
+ } );
+ }
+
+ public < R > List< R > forEachChunk( Function< LoopBuilder.Chunk< T >, R > chunkAction )
+ {
+ List< Interval > intervals = IntervalChunks.chunkInterval( new FinalInterval( firstImage.size() ), taskExecutor.suggestNumberOfTasks() );
+ List< Chunk< T > > chunks = intervals.stream().map( interval -> new Chunk< T >( firstImage, otherImages, interval ) ).collect( Collectors.toList() );
+ return taskExecutor.forEachApply( chunks, chunkAction );
+ }
+
+ public IterableLoopBuilder< T > multithreaded()
+ {
+ return multithreaded( Parallelization.getTaskExecutor() );
+ }
+
+ public IterableLoopBuilder< T > multithreaded( TaskExecutor taskExecutor )
+ {
+ this.taskExecutor = taskExecutor;
+ return this;
+ }
+
+ private static class Chunk< T > implements LoopBuilder.Chunk< T > {
+
+ private final IterableInterval< ? > firstImage;
+
+ private final List< RandomAccessible< ? > > otherImages;
+
+ private final Interval interval;
+
+ private Chunk( IterableInterval< ? > firstImage, List< RandomAccessible< ? > > otherImages, Interval interval )
+ {
+ this.firstImage = firstImage;
+ this.otherImages = otherImages;
+ this.interval = interval;
+ }
+
+ @Override
+ public void forEachPixel( T action )
+ {
+ Cursor< ? > cursor = firstImage.localizingCursor();
+ List< RandomAccess< ? > > randomAccesses = otherImages.stream().map( RandomAccessible::randomAccess ).collect( Collectors.toList());
+ cursor.jumpFwd( interval.min( 0 ) );
+ long size = interval.dimension( 0 );
+ FastCursorRandomAccessLoops.loop( action, size, cursor, randomAccesses );
+ }
+ }
+
+}
diff --git a/src/main/java/net/imglib2/algorithm/morphology/Dilation.java b/src/main/java/net/imglib2/algorithm/morphology/Dilation.java
index c840b5fa2..de6b80eb4 100644
--- a/src/main/java/net/imglib2/algorithm/morphology/Dilation.java
+++ b/src/main/java/net/imglib2/algorithm/morphology/Dilation.java
@@ -34,24 +34,23 @@
package net.imglib2.algorithm.morphology;
import java.util.List;
-import java.util.Vector;
+import java.util.function.BiConsumer;
-import net.imglib2.Cursor;
import net.imglib2.FinalDimensions;
import net.imglib2.Interval;
import net.imglib2.IterableInterval;
-import net.imglib2.RandomAccess;
import net.imglib2.RandomAccessible;
import net.imglib2.RandomAccessibleInterval;
+import net.imglib2.algorithm.loop.IterableLoopBuilder;
import net.imglib2.algorithm.neighborhood.Neighborhood;
import net.imglib2.algorithm.neighborhood.Shape;
import net.imglib2.img.Img;
import net.imglib2.img.ImgFactory;
-import net.imglib2.multithreading.Chunk;
-import net.imglib2.multithreading.SimpleMultiThreading;
+import net.imglib2.parallel.Parallelization;
import net.imglib2.type.Type;
import net.imglib2.type.logic.BitType;
import net.imglib2.type.numeric.RealType;
+import net.imglib2.util.Cast;
import net.imglib2.util.Util;
import net.imglib2.view.ExtendedRandomAccessibleInterval;
import net.imglib2.view.IntervalView;
@@ -59,6 +58,7 @@
public class Dilation
{
+
/**
* Performs the dilation morphological operation, on a {@link RealType}
* {@link Img} using a list of {@link Shape}s as a flat structuring element.
@@ -449,122 +449,37 @@ public static < T extends RealType< T >> void dilate( final RandomAccessible< T
*/
public static < T extends Type< T > & Comparable< T > > void dilate( final RandomAccessible< T > source, final IterableInterval< T > target, final Shape strel, final T minVal, int numThreads )
{
- numThreads = Math.max( 1, numThreads );
-
- /*
- * Prepare iteration.
- */
-
- final RandomAccessible< Neighborhood< T >> accessible = strel.neighborhoodsRandomAccessible( source );
-
- /*
- * Multithread
- */
-
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( target.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
-
- final Object tmp = minVal;
- if ( tmp instanceof BitType )
- {
- /*
- * Optimization for BitType
- */
+ final RandomAccessible< Neighborhood< T > > neighborhoods = strel.neighborhoodsRandomAccessible( source );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( target, neighborhoods ).multithreaded().forEachChunk( chunk -> {
+ chunk.forEachPixel( getDilateAction( minVal ) );
+ return null;
+ } );
+ } );
+ }
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology dilate thread " + i )
- {
- @Override
- public void run()
+ private static < T extends Type< T > & Comparable< T > > BiConsumer< T, Neighborhood< T > > getDilateAction( T minVal )
+ {
+ if ( minVal instanceof BitType )
+ return Cast.unchecked( (BiConsumer< BitType, Neighborhood< BitType > > ) ( t, neighborhood ) -> {
+ for ( BitType val1 : neighborhood )
+ if ( val1.get() )
{
- final RandomAccess< Neighborhood< T >> randomAccess = accessible.randomAccess( target );
- final Object tmp2 = target.cursor();
- @SuppressWarnings( "unchecked" )
- final Cursor< BitType > cursorDilated = ( Cursor< BitType > ) tmp2;
- cursorDilated.jumpFwd( chunk.getStartPosition() );
-
- for ( long steps = 0; steps < chunk.getLoopSize(); steps++ )
- {
- cursorDilated.fwd();
- randomAccess.setPosition( cursorDilated );
- final Neighborhood< T > neighborhood = randomAccess.get();
- final Object tmp3 = neighborhood.cursor();
- @SuppressWarnings( "unchecked" )
- final Cursor< BitType > nc = ( Cursor< BitType > ) tmp3;
-
- while ( nc.hasNext() )
- {
- nc.fwd();
- final BitType val = nc.get();
- if ( val.get() )
- {
- cursorDilated.get().set( true );
- break;
- }
- }
- }
-
+ t.set( true );
+ break;
}
- };
- }
- }
+ } );
else
{
- /*
- * All other comparable type.
- */
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology dilate thread " + i )
- {
- @Override
- public void run()
- {
- final RandomAccess< Neighborhood< T >> randomAccess = accessible.randomAccess( target );
- final Cursor< T > cursorDilated = target.cursor();
- cursorDilated.jumpFwd( chunk.getStartPosition() );
-
- final T max = MorphologyUtils.createVariable( source, target );
- for ( long steps = 0; steps < chunk.getLoopSize(); steps++ )
- {
- cursorDilated.fwd();
- randomAccess.setPosition( cursorDilated );
- final Neighborhood< T > neighborhood = randomAccess.get();
- final Cursor< T > nc = neighborhood.cursor();
-
- /*
- * Look for max in the neighborhood.
- */
-
- max.set( minVal );
- while ( nc.hasNext() )
- {
- nc.fwd();
- final T val = nc.get();
- // We need only Comparable to do this:
- if ( val.compareTo( max ) > 0 )
- {
- max.set( val );
- }
- }
- cursorDilated.get().set( max );
- }
-
- }
- };
- }
+ T max = minVal.copy();
+ return ( t, neighborhood ) -> {
+ max.set( minVal );
+ for ( T val : neighborhood )
+ if ( val.compareTo( max ) > 0 )
+ max.set( val );
+ t.set( max );
+ };
}
-
- /*
- * Launch calculation
- */
-
- SimpleMultiThreading.startAndJoin( threads );
}
/**
diff --git a/src/main/java/net/imglib2/algorithm/morphology/Erosion.java b/src/main/java/net/imglib2/algorithm/morphology/Erosion.java
index 6d28a1589..68e1e7bd4 100644
--- a/src/main/java/net/imglib2/algorithm/morphology/Erosion.java
+++ b/src/main/java/net/imglib2/algorithm/morphology/Erosion.java
@@ -34,24 +34,23 @@
package net.imglib2.algorithm.morphology;
import java.util.List;
-import java.util.Vector;
+import java.util.function.BiConsumer;
-import net.imglib2.Cursor;
import net.imglib2.FinalDimensions;
import net.imglib2.Interval;
import net.imglib2.IterableInterval;
-import net.imglib2.RandomAccess;
import net.imglib2.RandomAccessible;
import net.imglib2.RandomAccessibleInterval;
+import net.imglib2.algorithm.loop.IterableLoopBuilder;
import net.imglib2.algorithm.neighborhood.Neighborhood;
import net.imglib2.algorithm.neighborhood.Shape;
import net.imglib2.img.Img;
import net.imglib2.img.ImgFactory;
-import net.imglib2.multithreading.Chunk;
-import net.imglib2.multithreading.SimpleMultiThreading;
+import net.imglib2.parallel.Parallelization;
import net.imglib2.type.Type;
import net.imglib2.type.logic.BitType;
import net.imglib2.type.numeric.RealType;
+import net.imglib2.util.Cast;
import net.imglib2.util.Util;
import net.imglib2.view.ExtendedRandomAccessibleInterval;
import net.imglib2.view.IntervalView;
@@ -449,122 +448,37 @@ public static < T extends RealType< T >> void erode( final RandomAccessible< T >
*/
public static < T extends Type< T > & Comparable< T > > void erode( final RandomAccessible< T > source, final IterableInterval< T > target, final Shape strel, final T maxVal, int numThreads )
{
- numThreads = Math.max( 1, numThreads );
-
- /*
- * Prepare iteration.
- */
-
- final RandomAccessible< Neighborhood< T >> accessible = strel.neighborhoodsRandomAccessible( source );
-
- /*
- * Multithread
- */
-
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( target.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- final Object tmp = maxVal;
- if ( tmp instanceof BitType )
- {
- /*
- * Optimization for BitType
- */
+ final RandomAccessible< Neighborhood< T >> neighborhoods = strel.neighborhoodsRandomAccessible( source );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( target, neighborhoods ).multithreaded().forEachChunk( chunk -> {
+ chunk.forEachPixel( getDilateAction( maxVal ) );
+ return null;
+ } );
+ } );
+ }
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology erode thread " + i )
- {
- @Override
- public void run()
+ private static < T extends Type< T > & Comparable< T > > BiConsumer< T, Neighborhood< T > > getDilateAction( T maxVal )
+ {
+ if ( maxVal instanceof BitType )
+ return Cast.unchecked( ( BiConsumer< BitType, Neighborhood< BitType > > ) ( t, neighborhood ) -> {
+ for ( BitType val1 : neighborhood )
+ if ( val1.get() )
{
- final RandomAccess< Neighborhood< T >> randomAccess = accessible.randomAccess( target );
- final Object tmp2 = target.cursor();
- @SuppressWarnings( "unchecked" )
- final Cursor< BitType > cursorTarget = ( Cursor< BitType > ) tmp2;
- cursorTarget.jumpFwd( chunk.getStartPosition() );
-
- for ( long steps = 0; steps < chunk.getLoopSize(); steps++ )
- {
- cursorTarget.fwd();
- randomAccess.setPosition( cursorTarget );
- final Object tmp3 = randomAccess.get();
- @SuppressWarnings( "unchecked" )
- final Neighborhood< BitType > neighborhood = (net.imglib2.algorithm.neighborhood.Neighborhood< BitType > ) tmp3;
- final Cursor< BitType > nc = neighborhood.cursor();
-
- cursorTarget.get().set( true );
- while ( nc.hasNext() )
- {
- nc.fwd();
- final BitType val = nc.get();
- if ( !val.get() )
- {
- cursorTarget.get().set( false );
- break;
- }
- }
- }
-
+ t.set( true );
+ break;
}
- };
- }
- }
+ } );
else
{
- /*
- * All other comparable type.
- */
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology erode thread " + i )
- {
- @Override
- public void run()
- {
- final RandomAccess< Neighborhood< T >> randomAccess = accessible.randomAccess( target );
- final Cursor< T > cursorTarget = target.cursor();
- cursorTarget.jumpFwd( chunk.getStartPosition() );
-
- final T max = MorphologyUtils.createVariable( source, target );
- for ( long steps = 0; steps < chunk.getLoopSize(); steps++ )
- {
- cursorTarget.fwd();
- randomAccess.setPosition( cursorTarget );
- final Neighborhood< T > neighborhood = randomAccess.get();
- final Cursor< T > nc = neighborhood.cursor();
-
- /*
- * Look for max in the neighborhood.
- */
-
- max.set( maxVal );
- while ( nc.hasNext() )
- {
- nc.fwd();
- final T val = nc.get();
- // We need only Comparable to do this:
- if ( val.compareTo( max ) < 0 )
- {
- max.set( val );
- }
- }
- cursorTarget.get().set( max );
- }
-
- }
- };
- }
+ T min = maxVal.copy();
+ return ( t, neighborhood ) -> {
+ min.set( maxVal );
+ for ( T val : neighborhood )
+ if ( val.compareTo( min ) < 0 )
+ min.set( val );
+ t.set( min );
+ };
}
-
- /*
- * Launch calculation
- */
-
- SimpleMultiThreading.startAndJoin( threads );
}
/**
diff --git a/src/main/java/net/imglib2/algorithm/morphology/MorphologyUtils.java b/src/main/java/net/imglib2/algorithm/morphology/MorphologyUtils.java
index 1a4bd9202..59c2ca664 100644
--- a/src/main/java/net/imglib2/algorithm/morphology/MorphologyUtils.java
+++ b/src/main/java/net/imglib2/algorithm/morphology/MorphologyUtils.java
@@ -33,15 +33,13 @@
*/
package net.imglib2.algorithm.morphology;
-import java.util.Vector;
-
-import net.imglib2.Cursor;
import net.imglib2.EuclideanSpace;
import net.imglib2.Interval;
import net.imglib2.IterableInterval;
import net.imglib2.RandomAccess;
import net.imglib2.RandomAccessible;
import net.imglib2.RandomAccessibleInterval;
+import net.imglib2.algorithm.loop.IterableLoopBuilder;
import net.imglib2.algorithm.neighborhood.Neighborhood;
import net.imglib2.algorithm.neighborhood.Shape;
import net.imglib2.img.Img;
@@ -49,14 +47,11 @@
import net.imglib2.img.array.ArrayImgs;
import net.imglib2.img.array.ArrayRandomAccess;
import net.imglib2.img.basictypeaccess.array.LongArray;
-import net.imglib2.multithreading.Chunk;
-import net.imglib2.multithreading.SimpleMultiThreading;
+import net.imglib2.parallel.Parallelization;
import net.imglib2.type.Type;
import net.imglib2.type.logic.BitType;
import net.imglib2.type.operators.Sub;
-import net.imglib2.util.Intervals;
import net.imglib2.util.Util;
-import net.imglib2.view.IntervalView;
import net.imglib2.view.Views;
public class MorphologyUtils
@@ -66,17 +61,15 @@ public class MorphologyUtils
* Static util to compute the final image dimensions and required offset
* when performing a full dilation with the specified strel.
*
- * @param source
- * the source image.
- * @param strel
- * the strel to use for dilation.
+ * @param source the source image.
+ * @param strel the strel to use for dilation.
* @return a 2-elements {@code long[][]}:
- * > lists )
+ {
+ int size = lists.stream().mapToInt( List::size ).sum();
+ List< P > result = new ArrayList<>( size );
+ for ( List< P > list : lists )
+ result.addAll( list );
+ return result;
}
/**
diff --git a/src/main/java/net/imglib2/algorithm/localextrema/SubpixelLocalization.java b/src/main/java/net/imglib2/algorithm/localextrema/SubpixelLocalization.java
index 95abd98bf..23e88d079 100644
--- a/src/main/java/net/imglib2/algorithm/localextrema/SubpixelLocalization.java
+++ b/src/main/java/net/imglib2/algorithm/localextrema/SubpixelLocalization.java
@@ -34,14 +34,9 @@
package net.imglib2.algorithm.localextrema;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
+import Jama.LUDecomposition;
+import Jama.Matrix;
+import net.imglib2.FinalInterval;
import net.imglib2.Interval;
import net.imglib2.Localizable;
import net.imglib2.Point;
@@ -49,17 +44,22 @@
import net.imglib2.RandomAccessible;
import net.imglib2.RealPoint;
import net.imglib2.RealPositionable;
+import net.imglib2.loops.IntervalChunks;
+import net.imglib2.parallel.Parallelization;
+import net.imglib2.parallel.TaskExecutor;
import net.imglib2.type.numeric.RealType;
import net.imglib2.util.Intervals;
-import Jama.LUDecomposition;
-import Jama.Matrix;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
/**
* Refine a set of peaks to subpixel coordinates. This class provides the static
* {@link #refinePeaks(List, RandomAccessible, Interval, boolean, int, boolean, float, boolean[], int)}
* method to do this, but this has a lot of parameters. Therefore, this class
* can also be instantiated to encapsulate the parameter settings.
- *
+ *
- *
+ *
+ *
*/
public static final < T > long[][] computeTargetImageDimensionsAndOffset( final Interval source, final Shape strel )
{
@@ -291,61 +284,16 @@ private static final void appendSingleSlice( final RandomAccess< BitType > ra, f
static < T extends Type< T > > void copy( final IterableInterval< T > source, final RandomAccessible< T > target, final int numThreads )
{
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( source.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology copy thread " + i )
- {
- @Override
- public void run()
- {
- final Cursor< T > sourceCursor = source.localizingCursor();
- sourceCursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > targetRandomAccess = target.randomAccess();
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- sourceCursor.fwd();
- targetRandomAccess.setPosition( sourceCursor );
- targetRandomAccess.get().set( sourceCursor.get() );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( source, target ).multithreaded().forEachPixel( ( s, t ) -> t.set( s ) );
+ } );
}
static < T extends Type< T > > void copy2( final RandomAccessible< T > source, final IterableInterval< T > target, final int numThreads )
{
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( target.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology copy2 thread " + i )
- {
- @Override
- public void run()
- {
- final Cursor< T > targetCursor = target.localizingCursor();
- targetCursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > sourceRandomAccess = source.randomAccess();
-
- // iterate over the input cursor
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- targetCursor.fwd();
- sourceRandomAccess.setPosition( targetCursor );
- targetCursor.get().set( sourceRandomAccess.get() );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( target, source ).multithreaded().forEachPixel( ( t, s ) -> t.set( s ) );
+ } );
}
static < T extends Type< T > > Img< T > copyCropped( final Img< T > largeSource, final Interval interval, final int numThreads )
@@ -356,32 +304,8 @@ static < T extends Type< T > > Img< T > copyCropped( final Img< T > largeSource,
offset[ d ] = ( largeSource.dimension( d ) - interval.dimension( d ) ) / 2;
}
final Img< T > create = largeSource.factory().create( interval );
-
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( create.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology copyCropped thread " + i )
- {
- @Override
- public void run()
- {
- final IntervalView< T > intervalView = Views.offset( largeSource, offset );
- final Cursor< T > cursor = create.cursor();
- cursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > randomAccess = intervalView.randomAccess();
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- cursor.fwd();
- randomAccess.setPosition( cursor );
- cursor.get().set( randomAccess.get() );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ final RandomAccessibleInterval< T > intervalView = Views.translateInverse( largeSource, offset );
+ copy2( intervalView, create, numThreads );
return create;
}
@@ -393,7 +317,7 @@ public void run()
* @param interval
* @return type instance
*/
- static < T extends Type< T >> T createVariable( final RandomAccessible< T > accessible, final Interval interval )
+ static < T extends Type< T > > T createVariable( final RandomAccessible< T > accessible, final Interval interval )
{
final RandomAccess< T > a = accessible.randomAccess();
interval.min( a );
@@ -405,7 +329,7 @@ public static final Neighborhood< BitType > getNeighborhood( final Shape shape,
final int numDims = space.numDimensions();
final long[] dimensions = Util.getArrayFromValue( 1l, numDims );
final ArrayImg< BitType, LongArray > img = ArrayImgs.bits( dimensions );
- final IterableInterval< Neighborhood< BitType >> neighborhoods = shape.neighborhoods( img );
+ final IterableInterval< Neighborhood< BitType > > neighborhoods = shape.neighborhoods( img );
final Neighborhood< BitType > neighborhood = neighborhoods.cursor().next();
return neighborhood;
}
@@ -418,12 +342,10 @@ public static final Neighborhood< BitType > getNeighborhood( final Shape shape,
* This method only prints the first 3 dimensions of the structuring
* element. Dimensions above 3 are skipped.
*
- * @param shape
- * the structuring element to print.
- * @param dimensionality
- * the dimensionality to cast it over. This is required as
- * {@link Shape} does not carry a dimensionality, and we need one
- * to generate a neighborhood to iterate.
+ * @param shape the structuring element to print.
+ * @param dimensionality the dimensionality to cast it over. This is required as
+ * {@link Shape} does not carry a dimensionality, and we need one
+ * to generate a neighborhood to iterate.
* @return a string representation of the structuring element.
*/
public static final String printNeighborhood( final Shape shape, final int dimensionality )
@@ -473,219 +395,85 @@ else if ( neighborhood.numDimensions() > 0 )
/**
* Does A = A - B. Writes the results in A.
*
- * @param A
- * A
- * @param B
- * B
+ * @param A A
+ * @param B B
* @param numThreads
*/
static < T extends Sub< T > > void subAAB( final RandomAccessible< T > A, final IterableInterval< T > B, final int numThreads )
{
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( B.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology subAAB thread " + i )
- {
- @Override
- public void run()
- {
- final Cursor< T > Bcursor = B.localizingCursor();
- Bcursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > Ara = A.randomAccess();
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- Bcursor.fwd();
- Ara.setPosition( Bcursor );
- Ara.get().sub( Bcursor.get() );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( B, A ).multithreaded().forEachPixel( ( b, a ) -> a.sub( b ) );
+ } );
}
-
/**
* Does A = A - B. Writes the results in A.
*
- * @param A
- * A
- * @param B
- * B
+ * @param A A
+ * @param B B
* @param numThreads
*/
static < T extends Sub< T > > void subAAB2( final IterableInterval< T > A, final RandomAccessible< T > B, final int numThreads )
{
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( A.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology subAAB2 thread " + i )
- {
- @Override
- public void run()
- {
- final Cursor< T > Acursor = A.localizingCursor();
- Acursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > Bra = B.randomAccess(); // LOL
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- Acursor.fwd();
- Bra.setPosition( Acursor );
- Acursor.get().sub( Bra.get() );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( A, B ).multithreaded().forEachPixel( ( a, b ) -> a.sub( b ) );
+ } );
}
-
/**
* Does A = B - A. Writes the results in A.
*
- * @param source
- * A
- * @param target
- * B
+ * @param A A
+ * @param B B
* @param numThreads
*/
- static < T extends Sub< T > & Type< T >> void subABA( final RandomAccessible< T > source, final IterableInterval< T > target, final int numThreads )
+ static < T extends Sub< T > & Type< T > > void subABA( final RandomAccessible< T > A, final IterableInterval< T > B, final int numThreads )
{
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( target.size(), numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology subABA thread " + i )
- {
- @Override
- public void run()
- {
- final T tmp = createVariable( source, target );
- final Cursor< T > targetCursor = target.localizingCursor();
- targetCursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > sourceRandomAccess = source.randomAccess();
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- targetCursor.fwd();
- sourceRandomAccess.setPosition( targetCursor );
-
- tmp.set( targetCursor.get() );
- tmp.sub( sourceRandomAccess.get() );
-
- sourceRandomAccess.get().set( tmp );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( B, A ).multithreaded().forEachChunk( chunk -> {
+ T tmp = createVariable( A, B );
+ chunk.forEachPixel( ( b, a ) -> {
+ tmp.set( b );
+ tmp.sub( a );
+ a.set( tmp );
+ } );
+ return null;
+ } );
+ } );
}
/**
* Does A = B - A. Writes the results in A.
*
- * @param source
- * A
- * @param target
- * B
+ * @param A A
+ * @param B B
* @param numThreads
*/
- static < T extends Sub< T > & Type< T >> void subABA2( final RandomAccessibleInterval< T > source, final RandomAccessible< T > target, final int numThreads )
+ static < T extends Sub< T > & Type< T > > void subABA2( final RandomAccessibleInterval< T > A, final RandomAccessible< T > B, final int numThreads )
{
- final long size = Intervals.numElements( source );
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( size, numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology subABA2 thread " + i )
- {
- @Override
- public void run()
- {
- final T tmp = createVariable( target, source );
- final Cursor< T > sourceCursor = Views.iterable( source ).localizingCursor();
- sourceCursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > targetRandomAccess = target.randomAccess( source );
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
-
- }
- while ( sourceCursor.hasNext() )
- {
- sourceCursor.fwd();
- targetRandomAccess.setPosition( sourceCursor );
-
- tmp.set( targetRandomAccess.get() );
- tmp.sub( sourceCursor.get() );
-
- targetRandomAccess.get().set( tmp );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ subABA( A, Views.interval( B, A ), numThreads );
}
/**
* Does B = A - B. Writes the results in B.
*
- * @param A
- * A
- * @param B
- * B
+ * @param A A
+ * @param B B
* @param numThreads
*/
static < T extends Type< T > & Sub< T > > void subBAB( final RandomAccessible< T > A, final IterableInterval< T > B, final int numThreads )
{
- final long size = Intervals.numElements( B );
- final Vector< Chunk > chunks = SimpleMultiThreading.divideIntoChunks( size, numThreads );
- final Thread[] threads = SimpleMultiThreading.newThreads( numThreads );
-
- for ( int i = 0; i < threads.length; i++ )
- {
- final Chunk chunk = chunks.get( i );
- threads[ i ] = new Thread( "Morphology subBAB thread " + i )
- {
- @Override
- public void run()
- {
- final T tmp = createVariable( A, B );
- final Cursor< T > BCursor = B.localizingCursor();
- BCursor.jumpFwd( chunk.getStartPosition() );
- final RandomAccess< T > Ara = A.randomAccess();
-
- for ( long step = 0; step < chunk.getLoopSize(); step++ )
- {
- BCursor.fwd();
- Ara.setPosition( BCursor );
-
- tmp.set( Ara.get() );
- tmp.sub( BCursor.get() );
-
- BCursor.get().set( tmp );
- }
- }
- };
- }
-
- SimpleMultiThreading.startAndJoin( threads );
+ Parallelization.runWithNumThreads( numThreads, () -> {
+ IterableLoopBuilder.setImages( B, A ).multithreaded().forEachChunk( chunk -> {
+ T tmp = createVariable( A, B );
+ chunk.forEachPixel( ( b, a ) -> {
+ tmp.set( a );
+ tmp.sub( b );
+ a.set( tmp );
+ } );
+ return null;
+ } );
+ } );
}
}
diff --git a/src/main/java/net/imglib2/algorithm/morphology/distance/DistanceTransform.java b/src/main/java/net/imglib2/algorithm/morphology/distance/DistanceTransform.java
index 19c1350cc..a73f3c9f4 100644
--- a/src/main/java/net/imglib2/algorithm/morphology/distance/DistanceTransform.java
+++ b/src/main/java/net/imglib2/algorithm/morphology/distance/DistanceTransform.java
@@ -34,17 +34,10 @@
package net.imglib2.algorithm.morphology.distance;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
import java.util.stream.DoubleStream;
import java.util.stream.IntStream;
-
-import net.imglib2.Cursor;
import net.imglib2.FinalInterval;
import net.imglib2.Interval;
import net.imglib2.RandomAccessible;
@@ -56,12 +49,14 @@
import net.imglib2.img.array.ArrayImgFactory;
import net.imglib2.img.cell.CellImg;
import net.imglib2.img.cell.CellImgFactory;
+import net.imglib2.loops.LoopBuilder;
+import net.imglib2.parallel.Parallelization;
+import net.imglib2.parallel.TaskExecutors;
import net.imglib2.type.BooleanType;
import net.imglib2.type.NativeType;
import net.imglib2.type.numeric.RealType;
import net.imglib2.type.numeric.integer.LongType;
import net.imglib2.type.numeric.real.DoubleType;
-import net.imglib2.util.Intervals;
import net.imglib2.util.Util;
import net.imglib2.util.ValuePair;
import net.imglib2.view.Views;
@@ -169,7 +164,8 @@ public static < T extends RealType< T > > void transform(
final int nTasks,
final double... weights ) throws InterruptedException, ExecutionException
{
- transform( source, source, distanceType, es, nTasks, weights );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, source, distanceType, weights ) );
}
/**
@@ -248,7 +244,8 @@ public static < T extends RealType< T >, U extends RealType< U > > void transfor
final int nTasks,
final double... weights ) throws InterruptedException, ExecutionException
{
- transform( source, target, target, distanceType, es, nTasks, weights );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, target, distanceType, weights ) );
}
/**
@@ -353,21 +350,8 @@ public static < T extends RealType< T >, U extends RealType< U >, V extends Real
final int nTasks,
final double... weights ) throws InterruptedException, ExecutionException
{
-
- final boolean isIsotropic = weights.length <= 1;
- final double[] w = weights.length == source.numDimensions() ? weights : DoubleStream.generate( () -> weights.length == 0 ? 1.0 : weights[ 0 ] ).limit( source.numDimensions() ).toArray();
-
- switch ( distanceType )
- {
- case EUCLIDIAN:
- transform( source, tmp, target, isIsotropic ? new EuclidianDistanceIsotropic( w[ 0 ] ) : new EuclidianDistanceAnisotropic( w ), es, nTasks );
- break;
- case L1:
- transformL1( source, tmp, target, es, nTasks, w );
- break;
- default:
- break;
- }
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, tmp, target, distanceType, weights ) );
}
/**
@@ -422,7 +406,8 @@ public static < T extends RealType< T > > void transform(
final ExecutorService es,
final int nTasks ) throws InterruptedException, ExecutionException
{
- transform( source, source, d, es, nTasks );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, d ) );
}
/**
@@ -487,7 +472,8 @@ public static < T extends RealType< T >, U extends RealType< U > > void transfor
final ExecutorService es,
final int nTasks ) throws InterruptedException, ExecutionException
{
- transform( source, target, target, d, es, nTasks );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, target, d ) );
}
/**
@@ -592,37 +578,8 @@ public static < T extends RealType< T >, U extends RealType< U >, V extends Real
final ExecutorService es,
final int nTasks ) throws InterruptedException, ExecutionException
{
-
- assert source.numDimensions() == target.numDimensions(): "Dimension mismatch";
- final int nDim = source.numDimensions();
- final int lastDim = nDim - 1;
-
- if ( nDim == 1 )
- {
- transformAlongDimensionParallel(
- ( RandomAccessible< T > ) Views.addDimension( source ),
- Views.interval( Views.addDimension( target ), new FinalInterval( target.dimension( 0 ), 1 ) ),
- d,
- 0,
- es,
- nTasks );
- }
- else
- {
- transformAlongDimensionParallel( source, tmp, d, 0, es, nTasks );
- }
-
- for ( int dim = 1; dim < nDim; ++dim )
- {
- if ( dim == lastDim )
- {
- transformAlongDimensionParallel( tmp, target, d, dim, es, nTasks );
- }
- else
- {
- transformAlongDimensionParallel( tmp, tmp, d, dim, es, nTasks );
- }
- }
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> transform( source, tmp, target, d ) );
}
/**
@@ -697,7 +654,8 @@ public static < B extends BooleanType< B >, U extends RealType< U > > void binar
final int nTasks,
final double... weights ) throws InterruptedException, ExecutionException
{
- binaryTransform( source, target, target, distanceType, es, nTasks, weights );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> binaryTransform( source, target, distanceType, weights ) );
}
/**
@@ -789,11 +747,8 @@ public static < B extends BooleanType< B >, U extends RealType< U >, V extends R
final int nTasks,
final double... weights ) throws InterruptedException, ExecutionException
{
- final U maxVal = Util.getTypeFromInterval( tmp ).createVariable();
- maxVal.setReal( maxVal.getMaxValue() );
- final Converter< B, U > converter = new BinaryMaskToCost<>( maxVal );
- final RandomAccessible< U > converted = Converters.convert( source, converter, maxVal.createVariable() );
- transform( converted, tmp, target, distanceType, es, nTasks, weights );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> binaryTransform( source, tmp, target, distanceType, weights ) );
}
/**
@@ -886,7 +841,8 @@ public static < B extends BooleanType< B >, U extends RealType< U > > void binar
final ExecutorService es,
final int nTasks ) throws InterruptedException, ExecutionException
{
- binaryTransform( source, target, target, d, es, nTasks );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> binaryTransform( source, target, d ) );
}
/**
@@ -963,11 +919,8 @@ public static < B extends BooleanType< B >, U extends RealType< U >, V extends R
final ExecutorService es,
final int nTasks ) throws InterruptedException, ExecutionException
{
- final U maxVal = Util.getTypeFromInterval( tmp ).createVariable();
- maxVal.setReal( maxVal.getMaxValue() );
- final Converter< B, U > converter = new BinaryMaskToCost<>( maxVal );
- final RandomAccessible< U > converted = Converters.convert( source, converter, maxVal.createVariable() );
- transform( converted, tmp, target, d, es, nTasks );
+ Parallelization.runWithExecutor( TaskExecutors.forExecutorServiceAndNumTasks( es, nTasks ),
+ () -> binaryTransform( source, tmp, target, d ) );
}
/**
@@ -986,7 +939,7 @@ public static < B extends BooleanType< B >, U extends RealType< U >, V extends R
* @param weights
* Individual weights for each dimension, balancing image values
* and L1 distance.
- * @param
+ * @param