8265029: Preserve SIZED characteristics on slice operations (skip, limit)

Reviewed-by: psandoz
This commit is contained in:
Tagir F. Valeev 2021-05-28 01:56:38 +00:00
parent 95b1fa7a88
commit 0c9daa7ed5
12 changed files with 704 additions and 89 deletions

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2016, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -466,7 +466,32 @@ abstract class AbstractPipeline<E_IN, E_OUT, S extends BaseStream<E_OUT, S>>
@Override
final <P_IN> long exactOutputSizeIfKnown(Spliterator<P_IN> spliterator) {
return StreamOpFlag.SIZED.isKnown(getStreamAndOpFlags()) ? spliterator.getExactSizeIfKnown() : -1;
int flags = getStreamAndOpFlags();
long size = StreamOpFlag.SIZED.isKnown(flags) ? spliterator.getExactSizeIfKnown() : -1;
// Currently, we have no stateless SIZE_ADJUSTING intermediate operations,
// so we can simply ignore SIZE_ADJUSTING in parallel streams, since adjustments
// are already accounted in the input spliterator.
//
// If we ever have a stateless SIZE_ADJUSTING intermediate operation,
// we would need step back until depth == 0, then call exactOutputSize() for
// the subsequent stages.
if (size != -1 && StreamOpFlag.SIZE_ADJUSTING.isKnown(flags) && !isParallel()) {
// Skip the source stage as it's never SIZE_ADJUSTING
for (AbstractPipeline<?, ?, ?> stage = sourceStage.nextStage; stage != null; stage = stage.nextStage) {
size = stage.exactOutputSize(size);
}
}
return size;
}
/**
* Returns the exact output size of the pipeline given the exact size reported by the previous stage.
*
* @param previousSize the exact size reported by the previous stage
* @return the output size of this stage
*/
long exactOutputSize(long previousSize) {
return previousSize;
}
@Override

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2015, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -82,7 +82,8 @@ abstract class PipelineHelper<P_OUT> {
* The exact output size is known if the {@code Spliterator} has the
* {@code SIZED} characteristic, and the operation flags
* {@link StreamOpFlag#SIZED} is known on the combined stream and operation
* flags.
* flags. The exact output size may differ from spliterator size,
* if pipeline contains a slice operation.
*
* @param spliterator the spliterator describing the relevant portion of the
* source data

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2015, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -252,16 +252,18 @@ final class ReduceOps {
@Override
public <P_IN> Long evaluateSequential(PipelineHelper<T> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateSequential(helper, spliterator);
}
@Override
public <P_IN> Long evaluateParallel(PipelineHelper<T> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateParallel(helper, spliterator);
}
@ -426,16 +428,18 @@ final class ReduceOps {
@Override
public <P_IN> Long evaluateSequential(PipelineHelper<Integer> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateSequential(helper, spliterator);
}
@Override
public <P_IN> Long evaluateParallel(PipelineHelper<Integer> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateParallel(helper, spliterator);
}
@ -600,16 +604,18 @@ final class ReduceOps {
@Override
public <P_IN> Long evaluateSequential(PipelineHelper<Long> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateSequential(helper, spliterator);
}
@Override
public <P_IN> Long evaluateParallel(PipelineHelper<Long> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateParallel(helper, spliterator);
}
@ -774,16 +780,18 @@ final class ReduceOps {
@Override
public <P_IN> Long evaluateSequential(PipelineHelper<Double> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateSequential(helper, spliterator);
}
@Override
public <P_IN> Long evaluateParallel(PipelineHelper<Double> helper,
Spliterator<P_IN> spliterator) {
if (StreamOpFlag.SIZED.isKnown(helper.getStreamAndOpFlags()))
return spliterator.getExactSizeIfKnown();
long size = helper.exactOutputSizeIfKnown(spliterator);
if (size != -1)
return size;
return super.evaluateParallel(helper, spliterator);
}

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2016, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -50,7 +50,7 @@ final class SliceOps {
* @return the sliced size
*/
private static long calcSize(long size, long skip, long limit) {
return size >= 0 ? Math.max(-1, Math.min(size - skip, limit)) : -1;
return size >= 0 ? Math.max(0, Math.min(size - skip, limit)) : -1;
}
/**
@ -72,28 +72,23 @@ final class SliceOps {
* spliterator type. Requires that the underlying Spliterator
* be SUBSIZED.
*/
@SuppressWarnings("unchecked")
private static <P_IN> Spliterator<P_IN> sliceSpliterator(StreamShape shape,
Spliterator<P_IN> s,
long skip, long limit) {
assert s.hasCharacteristics(Spliterator.SUBSIZED);
long sliceFence = calcSliceFence(skip, limit);
switch (shape) {
case REFERENCE:
return new StreamSpliterators
.SliceSpliterator.OfRef<>(s, skip, sliceFence);
case INT_VALUE:
return (Spliterator<P_IN>) new StreamSpliterators
.SliceSpliterator.OfInt((Spliterator.OfInt) s, skip, sliceFence);
case LONG_VALUE:
return (Spliterator<P_IN>) new StreamSpliterators
.SliceSpliterator.OfLong((Spliterator.OfLong) s, skip, sliceFence);
case DOUBLE_VALUE:
return (Spliterator<P_IN>) new StreamSpliterators
.SliceSpliterator.OfDouble((Spliterator.OfDouble) s, skip, sliceFence);
default:
throw new IllegalStateException("Unknown shape " + shape);
}
@SuppressWarnings("unchecked")
Spliterator<P_IN> sliceSpliterator = (Spliterator<P_IN>) switch (shape) {
case REFERENCE
-> new StreamSpliterators.SliceSpliterator.OfRef<>(s, skip, sliceFence);
case INT_VALUE
-> new StreamSpliterators.SliceSpliterator.OfInt((Spliterator.OfInt) s, skip, sliceFence);
case LONG_VALUE
-> new StreamSpliterators.SliceSpliterator.OfLong((Spliterator.OfLong) s, skip, sliceFence);
case DOUBLE_VALUE
-> new StreamSpliterators.SliceSpliterator.OfDouble((Spliterator.OfDouble) s, skip, sliceFence);
};
return sliceSpliterator;
}
/**
@ -110,9 +105,15 @@ final class SliceOps {
long skip, long limit) {
if (skip < 0)
throw new IllegalArgumentException("Skip must be non-negative: " + skip);
long normalizedLimit = limit >= 0 ? limit : Long.MAX_VALUE;
return new ReferencePipeline.StatefulOp<T, T>(upstream, StreamShape.REFERENCE,
flags(limit)) {
@Override
long exactOutputSize(long previousSize) {
return calcSize(previousSize, skip, normalizedLimit);
}
Spliterator<T> unorderedSkipLimitSpliterator(Spliterator<T> s,
long skip, long limit, long sizeIfKnown) {
if (skip <= sizeIfKnown) {
@ -182,9 +183,9 @@ final class SliceOps {
@Override
Sink<T> opWrapSink(int flags, Sink<T> sink) {
return new Sink.ChainedReference<T, T>(sink) {
return new Sink.ChainedReference<>(sink) {
long n = skip;
long m = limit >= 0 ? limit : Long.MAX_VALUE;
long m = normalizedLimit;
@Override
public void begin(long size) {
@ -226,9 +227,15 @@ final class SliceOps {
long skip, long limit) {
if (skip < 0)
throw new IllegalArgumentException("Skip must be non-negative: " + skip);
long normalizedLimit = limit >= 0 ? limit : Long.MAX_VALUE;
return new IntPipeline.StatefulOp<Integer>(upstream, StreamShape.INT_VALUE,
flags(limit)) {
@Override
long exactOutputSize(long previousSize) {
return calcSize(previousSize, skip, normalizedLimit);
}
Spliterator.OfInt unorderedSkipLimitSpliterator(
Spliterator.OfInt s, long skip, long limit, long sizeIfKnown) {
if (skip <= sizeIfKnown) {
@ -291,9 +298,9 @@ final class SliceOps {
@Override
Sink<Integer> opWrapSink(int flags, Sink<Integer> sink) {
return new Sink.ChainedInt<Integer>(sink) {
return new Sink.ChainedInt<>(sink) {
long n = skip;
long m = limit >= 0 ? limit : Long.MAX_VALUE;
long m = normalizedLimit;
@Override
public void begin(long size) {
@ -335,9 +342,15 @@ final class SliceOps {
long skip, long limit) {
if (skip < 0)
throw new IllegalArgumentException("Skip must be non-negative: " + skip);
long normalizedLimit = limit >= 0 ? limit : Long.MAX_VALUE;
return new LongPipeline.StatefulOp<Long>(upstream, StreamShape.LONG_VALUE,
flags(limit)) {
@Override
long exactOutputSize(long previousSize) {
return calcSize(previousSize, skip, normalizedLimit);
}
Spliterator.OfLong unorderedSkipLimitSpliterator(
Spliterator.OfLong s, long skip, long limit, long sizeIfKnown) {
if (skip <= sizeIfKnown) {
@ -400,9 +413,9 @@ final class SliceOps {
@Override
Sink<Long> opWrapSink(int flags, Sink<Long> sink) {
return new Sink.ChainedLong<Long>(sink) {
return new Sink.ChainedLong<>(sink) {
long n = skip;
long m = limit >= 0 ? limit : Long.MAX_VALUE;
long m = normalizedLimit;
@Override
public void begin(long size) {
@ -444,9 +457,15 @@ final class SliceOps {
long skip, long limit) {
if (skip < 0)
throw new IllegalArgumentException("Skip must be non-negative: " + skip);
long normalizedLimit = limit >= 0 ? limit : Long.MAX_VALUE;
return new DoublePipeline.StatefulOp<Double>(upstream, StreamShape.DOUBLE_VALUE,
flags(limit)) {
@Override
long exactOutputSize(long previousSize) {
return calcSize(previousSize, skip, normalizedLimit);
}
Spliterator.OfDouble unorderedSkipLimitSpliterator(
Spliterator.OfDouble s, long skip, long limit, long sizeIfKnown) {
if (skip <= sizeIfKnown) {
@ -509,9 +528,9 @@ final class SliceOps {
@Override
Sink<Double> opWrapSink(int flags, Sink<Double> sink) {
return new Sink.ChainedDouble<Double>(sink) {
return new Sink.ChainedDouble<>(sink) {
long n = skip;
long m = limit >= 0 ? limit : Long.MAX_VALUE;
long m = normalizedLimit;
@Override
public void begin(long size) {
@ -541,7 +560,7 @@ final class SliceOps {
}
private static int flags(long limit) {
return StreamOpFlag.NOT_SIZED | ((limit != -1) ? StreamOpFlag.IS_SHORT_CIRCUIT : 0);
return StreamOpFlag.IS_SIZE_ADJUSTING | ((limit != -1) ? StreamOpFlag.IS_SHORT_CIRCUIT : 0);
}
/**

View file

@ -325,12 +325,24 @@ enum StreamOpFlag {
*/
// 12, 0x01000000
SHORT_CIRCUIT(12,
set(Type.OP).set(Type.TERMINAL_OP));
set(Type.OP).set(Type.TERMINAL_OP)),
/**
* Characteristic value signifying that an operation may adjust the
* total size of the stream.
* <p>
* The flag, if present, is only valid when SIZED is present;
* and is only valid for sequential streams.
* <p>
* An intermediate operation can preserve or inject this value.
*/
// 13, 0x04000000
SIZE_ADJUSTING(13,
set(Type.OP));
// The following 2 flags are currently undefined and a free for any further
// stream flags if/when required
//
// 13, 0x04000000
// 14, 0x10000000
// 15, 0x40000000
@ -629,6 +641,11 @@ enum StreamOpFlag {
*/
static final int IS_SHORT_CIRCUIT = SHORT_CIRCUIT.set;
/**
* The bit value to inject {@link #SIZE_ADJUSTING}.
*/
static final int IS_SIZE_ADJUSTING = SIZE_ADJUSTING.set;
private static int getMask(int flags) {
return (flags == 0)
? FLAG_MASK

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2017, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -217,19 +217,17 @@ class StreamSpliterators {
@Override
public final long estimateSize() {
init();
long exactSizeIfKnown = getExactSizeIfKnown();
// Use the estimate of the wrapped spliterator
// Note this may not be accurate if there are filter/flatMap
// operations filtering or adding elements to the stream
return spliterator.estimateSize();
return exactSizeIfKnown == -1 ? spliterator.estimateSize() : exactSizeIfKnown;
}
@Override
public final long getExactSizeIfKnown() {
init();
return StreamOpFlag.SIZED.isKnown(ph.getStreamAndOpFlags())
? spliterator.getExactSizeIfKnown()
: -1;
return ph.exactOutputSizeIfKnown(spliterator);
}
@Override

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2014, 2015, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2014, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -24,13 +24,14 @@
/**
* @test
* @summary Tests counting of streams
* @bug 8031187 8067969 8075307
* @bug 8031187 8067969 8075307 8265029
*/
package org.openjdk.tests.java.util.stream;
import java.util.HashSet;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.DoubleStream;
import java.util.stream.DoubleStreamTestDataProvider;
@ -56,6 +57,11 @@ public class CountTest extends OpTestCase {
expectedResult(expectedCount).
exercise();
withData(data).
terminal(s -> s.skip(1), Stream::count).
expectedResult(Math.max(0, expectedCount - 1)).
exercise();
// Test with an unknown sized stream
withData(data).
terminal(s -> s.filter(e -> true), Stream::count).
@ -91,6 +97,11 @@ public class CountTest extends OpTestCase {
expectedResult(expectedCount).
exercise();
withData(data).
terminal(s -> s.skip(1), IntStream::count).
expectedResult(Math.max(0, expectedCount - 1)).
exercise();
withData(data).
terminal(s -> s.filter(e -> true), IntStream::count).
expectedResult(expectedCount).
@ -116,6 +127,11 @@ public class CountTest extends OpTestCase {
expectedResult(expectedCount).
exercise();
withData(data).
terminal(s -> s.skip(1), LongStream::count).
expectedResult(Math.max(0, expectedCount - 1)).
exercise();
withData(data).
terminal(s -> s.filter(e -> true), LongStream::count).
expectedResult(expectedCount).
@ -141,6 +157,11 @@ public class CountTest extends OpTestCase {
expectedResult(expectedCount).
exercise();
withData(data).
terminal(s -> s.skip(1), DoubleStream::count).
expectedResult(Math.max(0, expectedCount - 1)).
exercise();
withData(data).
terminal(s -> s.filter(e -> true), DoubleStream::count).
expectedResult(expectedCount).
@ -157,41 +178,51 @@ public class CountTest extends OpTestCase {
exercise();
}
@Test
public void testNoEvaluationForSizedStream() {
{
checkStreamDoesNotConsumeElements(() -> Stream.of(1, 2, 3, 4), 4);
checkStreamDoesNotConsumeElements(() -> Stream.of(1, 2, 3, 4).skip(1).limit(2).skip(1), 1);
checkIntStreamDoesNotConsumeElements(() -> IntStream.of(1, 2, 3, 4), 4);
checkIntStreamDoesNotConsumeElements(() -> IntStream.of(1, 2, 3, 4).skip(1).limit(2).skip(1), 1);
checkLongStreamDoesNotConsumeElements(() -> LongStream.of(1, 2, 3, 4), 4);
checkLongStreamDoesNotConsumeElements(() -> LongStream.of(1, 2, 3, 4).skip(1).limit(2).skip(1), 1);
checkDoubleStreamDoesNotConsumeElements(() -> DoubleStream.of(1, 2, 3, 4), 4);
checkDoubleStreamDoesNotConsumeElements(() -> DoubleStream.of(1, 2, 3, 4).skip(1).limit(2).skip(1), 1);
}
private void checkStreamDoesNotConsumeElements(Supplier<Stream<?>> supplier, long expectedCount) {
AtomicInteger ai = new AtomicInteger();
Stream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).count(), expectedCount);
assertEquals(ai.get(), 0);
Stream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).parallel().count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).parallel().count(), expectedCount);
assertEquals(ai.get(), 0);
}
{
private void checkIntStreamDoesNotConsumeElements(Supplier<IntStream> supplier, long expectedCount) {
AtomicInteger ai = new AtomicInteger();
IntStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).count(), expectedCount);
assertEquals(ai.get(), 0);
IntStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).parallel().count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).parallel().count(), expectedCount);
assertEquals(ai.get(), 0);
}
{
private void checkLongStreamDoesNotConsumeElements(Supplier<LongStream> supplier, long expectedCount) {
AtomicInteger ai = new AtomicInteger();
LongStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).count(), expectedCount);
assertEquals(ai.get(), 0);
LongStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).parallel().count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).parallel().count(), expectedCount);
assertEquals(ai.get(), 0);
}
{
private void checkDoubleStreamDoesNotConsumeElements(Supplier<DoubleStream> supplier, long expectedCount) {
AtomicInteger ai = new AtomicInteger();
DoubleStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).count(), expectedCount);
assertEquals(ai.get(), 0);
DoubleStream.of(1, 2, 3, 4).peek(e -> ai.getAndIncrement()).parallel().count();
assertEquals(supplier.get().peek(e -> ai.getAndIncrement()).parallel().count(), expectedCount);
assertEquals(ai.get(), 0);
}
}
}

View file

@ -1,5 +1,5 @@
/*
* Copyright (c) 2012, 2016, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2012, 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@ -23,7 +23,7 @@
/*
* @test
* @bug 8148250
* @bug 8148250 8265029
*/
package org.openjdk.tests.java.util.stream;
@ -356,4 +356,26 @@ public class SliceOpTest extends OpTestCase {
assertEquals(LongStream.range(0, n).toArray(), actual);
}
}
public void testSliceOpsSpliteratorPreservesSized() {
var parSpliterator = IntStream.range(0, 1000).parallel().skip(50).limit(800).spliterator();
assertTrue(parSpliterator.hasCharacteristics(Spliterator.SIZED));
assertTrue(parSpliterator.hasCharacteristics(Spliterator.SUBSIZED));
assertEquals(parSpliterator.getExactSizeIfKnown(), 800);
// Original spliterator is split to [0..499] and [500..999] parts
// due to skip+limit, we have [50..499] and [500..849]
var prefix = parSpliterator.trySplit();
assertNotNull(prefix);
assertTrue(parSpliterator.hasCharacteristics(Spliterator.SIZED));
assertTrue(parSpliterator.hasCharacteristics(Spliterator.SUBSIZED));
assertEquals(parSpliterator.getExactSizeIfKnown(), 350);
assertTrue(prefix.hasCharacteristics(Spliterator.SIZED));
assertTrue(prefix.hasCharacteristics(Spliterator.SUBSIZED));
assertEquals(prefix.getExactSizeIfKnown(), 450);
var seqSpliterator = IntStream.range(0, 1000).skip(50).limit(800).spliterator();
assertTrue(seqSpliterator.hasCharacteristics(Spliterator.SIZED));
assertTrue(seqSpliterator.hasCharacteristics(Spliterator.SUBSIZED));
assertEquals(seqSpliterator.getExactSizeIfKnown(), 800);
}
}

View file

@ -0,0 +1,98 @@
/*
* Copyright (c) 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package org.openjdk.bench.java.util.stream.ops.ref;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.State;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;
/**
* Benchmark for limit()/skip() operation in sized streams.
*/
@BenchmarkMode(Mode.Throughput)
@OutputTimeUnit(TimeUnit.SECONDS)
@State(Scope.Thread)
public class SliceToList {
@Param("10000")
private int size;
@Benchmark
public List<String> seq_baseline() {
return IntStream.range(0, size)
.mapToObj(x -> "x")
.toList();
}
@Benchmark
public List<String> seq_limit() {
return IntStream.range(0, size * 2)
.mapToObj(x -> "x")
.limit(size)
.toList();
}
@Benchmark
public List<String> seq_skipLimit() {
return IntStream.range(0, size * 2)
.mapToObj(x -> "x")
.skip(1)
.limit(size)
.toList();
}
@Benchmark
public List<String> par_baseline() {
return IntStream.range(0, size)
.parallel()
.mapToObj(x -> "x")
.toList();
}
@Benchmark
public List<String> par_limit() {
return IntStream.range(0, size * 2)
.parallel()
.mapToObj(x -> "x")
.limit(size)
.toList();
}
@Benchmark
public List<String> par_skipLimit() {
return IntStream.range(0, size * 2)
.parallel()
.mapToObj(x -> "x")
.skip(1)
.limit(size)
.toList();
}
}

View file

@ -0,0 +1,152 @@
/*
* Copyright (c) 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package org.openjdk.bench.java.util.stream.ops.value;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.util.concurrent.TimeUnit;
import java.util.stream.DoubleStream;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
import java.util.stream.Stream;
/**
* Benchmark for count operation in sized streams.
*/
@Fork(5)
@Warmup(iterations = 10, time = 1000, timeUnit = TimeUnit.MILLISECONDS)
@Measurement(iterations = 20, time = 1000, timeUnit = TimeUnit.MILLISECONDS)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.NANOSECONDS)
@State(Scope.Thread)
public class SizedCount {
@Param("10000")
private int size;
@Param({"true", "false"})
private boolean polluteTypeProfile;
@Setup
public void setup() {
if (!polluteTypeProfile) return;
for(int i=0; i<10000; i++) {
IntStream.empty().skip(1).count();
LongStream.empty().skip(1).count();
DoubleStream.empty().skip(1).count();
Stream.empty().skip(1).count();
}
}
@Benchmark
public long count0() {
return IntStream.range(0, size)
.count();
}
@Benchmark
public long count2() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.count();
}
@Benchmark
public long count4() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.count();
}
@Benchmark
public long count6() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.count();
}
@Benchmark
public long count8() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.count();
}
@Benchmark
public long count10() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.count();
}
@Benchmark
public long count10Skip() {
return IntStream.range(0, size)
.skip(1)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.count();
}
}

View file

@ -0,0 +1,152 @@
/*
* Copyright (c) 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package org.openjdk.bench.java.util.stream.ops.value;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.util.concurrent.TimeUnit;
import java.util.stream.DoubleStream;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
import java.util.stream.Stream;
/**
* Benchmark for sum operation in sized streams.
*/
@Fork(5)
@Warmup(iterations = 10, time = 1000, timeUnit = TimeUnit.MILLISECONDS)
@Measurement(iterations = 20, time = 1000, timeUnit = TimeUnit.MILLISECONDS)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.NANOSECONDS)
@State(Scope.Thread)
public class SizedSum {
@Param({"5", "10"})
private int size;
@Param({"true", "false"})
private boolean polluteTypeProfile;
@Setup
public void setup() {
if (!polluteTypeProfile) return;
for(int i=0; i<10000; i++) {
IntStream.empty().skip(1).count();
LongStream.empty().skip(1).count();
DoubleStream.empty().skip(1).count();
Stream.empty().skip(1).count();
}
}
@Benchmark
public long sum0() {
return IntStream.range(0, size)
.sum();
}
@Benchmark
public long sum2() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.sum();
}
@Benchmark
public long sum4() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.sum();
}
@Benchmark
public long sum6() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.sum();
}
@Benchmark
public long sum8() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.sum();
}
@Benchmark
public long sum10() {
return IntStream.range(0, size)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.sum();
}
@Benchmark
public long sum10Skip() {
return IntStream.range(0, size)
.skip(1)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.map(x -> x)
.sum();
}
}

View file

@ -0,0 +1,92 @@
/*
* Copyright (c) 2021, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package org.openjdk.bench.java.util.stream.ops.value;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.State;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;
/**
* Benchmark for limit()/skip() operation in sized streams.
*/
@BenchmarkMode(Mode.Throughput)
@OutputTimeUnit(TimeUnit.SECONDS)
@State(Scope.Thread)
public class SliceToArray {
@Param("10000")
private int size;
@Benchmark
public int[] seq_baseline() {
return IntStream.range(0, size)
.toArray();
}
@Benchmark
public int[] seq_limit() {
return IntStream.range(0, size * 2)
.limit(size)
.toArray();
}
@Benchmark
public int[] seq_skipLimit() {
return IntStream.range(0, size * 2)
.skip(1)
.limit(size)
.toArray();
}
@Benchmark
public int[] par_baseline() {
return IntStream.range(0, size)
.parallel()
.toArray();
}
@Benchmark
public int[] par_limit() {
return IntStream.range(0, size * 2)
.parallel()
.limit(size)
.toArray();
}
@Benchmark
public int[] par_skipLimit() {
return IntStream.range(0, size * 2)
.parallel()
.skip(1)
.limit(size)
.toArray();
}
}