From a6ba7bf26d97c3a55a5a8cd9e9e3e00bb0fd12b3 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 6 Mar 2019 16:18:37 +1000 Subject: [PATCH 01/39] Add a basic pipeline framework. --- services/pipeline/build.gradle | 40 ++++++++++ .../services/pipeline/CompleterStage.java | 42 ++++++++++ .../pantheon/services/pipeline/InputPipe.java | 19 +++++ .../services/pipeline/OutputPipe.java | 21 +++++ .../pantheon/services/pipeline/Pipe.java | 74 ++++++++++++++++++ .../pantheon/services/pipeline/Pipeline.java | 34 +++++++++ .../services/pipeline/PipelineBuilder.java | 50 ++++++++++++ .../services/pipeline/ProcessStage.java | 42 ++++++++++ .../services/pipeline/SourceStage.java | 33 ++++++++ .../services/pipeline/CompleterStageTest.java | 39 ++++++++++ .../pipeline/PipelineBuilderTest.java | 76 +++++++++++++++++++ .../services/pipeline/ProcessStageTest.java | 56 ++++++++++++++ .../services/pipeline/SourceStageTest.java | 41 ++++++++++ .../pipeline/src/test/resources/log4j2.xml | 16 ++++ settings.gradle | 1 + 15 files changed, 584 insertions(+) create mode 100644 services/pipeline/build.gradle create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java create mode 100644 services/pipeline/src/test/resources/log4j2.xml diff --git a/services/pipeline/build.gradle b/services/pipeline/build.gradle new file mode 100644 index 0000000000..4d2b6df9f2 --- /dev/null +++ b/services/pipeline/build.gradle @@ -0,0 +1,40 @@ +/* + * Copyright 2018 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +apply plugin: 'java-library' + +jar { + baseName 'pantheon-pipeline' + manifest { + attributes( + 'Specification-Title': baseName, + 'Specification-Version': project.version, + 'Implementation-Title': baseName, + 'Implementation-Version': calculateVersion() + ) + } +} + +dependencies { + api project(':util') + implementation project(':metrics') + + implementation 'org.apache.logging.log4j:log4j-api' + implementation 'com.google.guava:guava' + + runtime 'org.apache.logging.log4j:log4j-core' + + testImplementation 'junit:junit' + testImplementation 'org.assertj:assertj-core' + testImplementation 'org.mockito:mockito-core' +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java new file mode 100644 index 0000000000..e4d3aa7f37 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +class CompleterStage implements Runnable { + private final InputPipe input; + private final Consumer completer; + private final CompletableFuture future = new CompletableFuture<>(); + + CompleterStage(final InputPipe input, final Consumer completer) { + this.input = input; + this.completer = completer; + } + + @Override + public void run() { + while (input.hasMore()) { + final T value = input.get(); + if (value != null) { + completer.accept(value); + } + } + future.complete(null); + } + + public CompletableFuture getFuture() { + return future; + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java new file mode 100644 index 0000000000..24086dc03b --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +public interface InputPipe { + boolean hasMore(); + + T get(); +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java new file mode 100644 index 0000000000..8a1e40ffda --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -0,0 +1,21 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +public interface OutputPipe { + boolean isOpen(); + + void put(T value); + + void close(); +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java new file mode 100644 index 0000000000..62a5d4ff05 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -0,0 +1,74 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +class Pipe implements InputPipe, OutputPipe { + private static final Logger LOG = LogManager.getLogger(); + private final BlockingQueue queue; + private final AtomicBoolean closed = new AtomicBoolean(); + + public Pipe(final int capacity) { + queue = new ArrayBlockingQueue<>(capacity); + } + + @Override + public boolean isOpen() { + return !closed.get(); + } + + @Override + public void close() { + closed.set(true); + } + + @Override + public boolean hasMore() { + return !closed.get() || !queue.isEmpty(); + } + + @Override + public T get() { + while (isOpen() || !queue.isEmpty()) { + try { + final T value = queue.poll(1, TimeUnit.SECONDS); + if (value != null) { + return value; + } + } catch (final InterruptedException e) { + LOG.trace("Interrupted while waiting for next item", e); + } + } + return null; + } + + @Override + public void put(final T value) { + while (isOpen()) { + try { + if (queue.offer(value, 1, TimeUnit.SECONDS)) { + return; + } + } catch (final InterruptedException e) { + LOG.trace("Interrupted while waiting to add to output", e); + } + } + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java new file mode 100644 index 0000000000..f31a8bf829 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; + +public class Pipeline { + + private final Collection stages; + private final CompleterStage completerStage; + + public Pipeline(final Collection stages, final CompleterStage completerStage) { + this.stages = stages; + this.completerStage = completerStage; + } + + public CompletableFuture start(final ExecutorService executorService) { + stages.forEach(executorService::submit); + executorService.submit(completerStage); + return completerStage.getFuture(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java new file mode 100644 index 0000000000..6c922d9f16 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static java.util.Collections.singleton; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.function.Consumer; +import java.util.function.Function; + +public class PipelineBuilder { + + private final Collection stages; + private final InputPipe pipeEnd; + + public PipelineBuilder(final Collection stages, final InputPipe pipeEnd) { + this.stages = stages; + this.pipeEnd = pipeEnd; + } + + public static PipelineBuilder createPipelineFrom(final Iterator taskStream) { + final Pipe pipe = new Pipe<>(10); + final SourceStage sourceStage = new SourceStage<>(taskStream, pipe); + return new PipelineBuilder<>(singleton(sourceStage), pipe); + } + + public PipelineBuilder thenProcessWith(final Function processor) { + final ArrayList newStages = new ArrayList<>(stages); + final Pipe outputPipe = new Pipe<>(10); + final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); + newStages.add(processStage); + return new PipelineBuilder<>(newStages, outputPipe); + } + + public Pipeline andFinishWith(final Consumer completer) { + return new Pipeline(stages, new CompleterStage<>(pipeEnd, completer)); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java new file mode 100644 index 0000000000..de845a545f --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.function.Function; + +class ProcessStage implements Runnable { + + private final InputPipe inputPipe; + private final OutputPipe outputPipe; + private final Function processor; + + public ProcessStage( + final InputPipe inputPipe, + final OutputPipe outputPipe, + final Function processor) { + this.inputPipe = inputPipe; + this.outputPipe = outputPipe; + this.processor = processor; + } + + @Override + public void run() { + while (inputPipe.hasMore()) { + final I value = inputPipe.get(); + if (value != null) { + outputPipe.put(processor.apply(value)); + } + } + outputPipe.close(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java new file mode 100644 index 0000000000..5d3eebefd3 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java @@ -0,0 +1,33 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.Iterator; + +class SourceStage implements Runnable { + private final Iterator source; + private final OutputPipe output; + + SourceStage(final Iterator source, final OutputPipe output) { + this.source = source; + this.output = output; + } + + @Override + public void run() { + while (output.isOpen() && source.hasNext()) { + output.put(source.next()); + } + output.close(); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java new file mode 100644 index 0000000000..f6e1828862 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java @@ -0,0 +1,39 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.ArrayList; +import java.util.List; + +import org.junit.Test; + +public class CompleterStageTest { + + private final Pipe pipe = new Pipe<>(10); + private final List output = new ArrayList<>(); + private final CompleterStage stage = new CompleterStage<>(pipe, output::add); + + @Test + public void shouldAddItemsToOutputUntilPipeHasNoMore() { + pipe.put("a"); + pipe.put("b"); + pipe.put("c"); + pipe.close(); + + stage.run(); + + assertThat(output).containsExactly("a", "b", "c"); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java new file mode 100644 index 0000000000..3d8e903613 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -0,0 +1,76 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static com.google.common.primitives.Ints.asList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.AfterClass; +import org.junit.Test; + +public class PipelineBuilderTest { + + private static final ThreadFactory THREAD_FACTORY = + new ThreadFactoryBuilder() + .setNameFormat(PipelineBuilderTest.class.getSimpleName() + "-%d") + .setDaemon(true) + .build(); + private final Iterator tasks = + asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15).iterator(); + + private static final ExecutorService executorService = + Executors.newCachedThreadPool(THREAD_FACTORY); + + @AfterClass + public static void afterClass() throws Exception { + executorService.shutdownNow(); + if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) { + fail("Executor service did not shut down cleanly"); + } + } + + @Test + public void shouldPipeTasksFromSupplierToCompleter() throws Exception { + final List output = new ArrayList<>(); + final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks).andFinishWith(output::add); + final CompletableFuture result = pipeline.start(executorService); + result.get(10, TimeUnit.SECONDS); + assertThat(output).containsExactly(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); + } + + @Test + public void shouldPassInputThroughIntermediateStage() throws Exception { + final List output = new ArrayList<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks) + .thenProcessWith(Object::toString) + .andFinishWith(output::add); + + final CompletableFuture result = pipeline.start(executorService); + result.get(10, TimeUnit.SECONDS); + assertThat(output) + .containsExactly( + "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12", "13", "14", "15"); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java new file mode 100644 index 0000000000..936a06d273 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Test; + +public class ProcessStageTest { + + private final Pipe input = new Pipe<>(10); + private final Pipe output = new Pipe<>(10); + + private final ProcessStage stage = + new ProcessStage<>(input, output, String::toLowerCase); + + @Test + public void shouldCloseOutputPipeWhenInputIsClosed() { + input.put("A"); + input.put("B"); + input.put("C"); + input.close(); + + stage.run(); + + assertThat(output.isOpen()).isFalse(); + } + + @Test + public void shouldApplyFunctionToItems() { + input.put("A"); + input.put("B"); + input.put("C"); + input.close(); + + stage.run(); + + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("a"); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("b"); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("c"); + assertThat(output.hasMore()).isFalse(); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java new file mode 100644 index 0000000000..ce3cc7fd8e --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java @@ -0,0 +1,41 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.google.common.collect.Iterators; +import org.junit.Test; + +public class SourceStageTest { + + private final Pipe output = new Pipe<>(10); + + private final SourceStage stage = + new SourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); + + @Test + public void shouldOutputEntriesThenClosePipe() { + stage.run(); + assertThat(output.isOpen()).isFalse(); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("a"); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("b"); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("c"); + assertThat(output.hasMore()).isTrue(); + assertThat(output.get()).isEqualTo("d"); + assertThat(output.hasMore()).isFalse(); + } +} diff --git a/services/pipeline/src/test/resources/log4j2.xml b/services/pipeline/src/test/resources/log4j2.xml new file mode 100644 index 0000000000..f6fd6801df --- /dev/null +++ b/services/pipeline/src/test/resources/log4j2.xml @@ -0,0 +1,16 @@ + + + + INFO + + + + + + + + + + + + diff --git a/settings.gradle b/settings.gradle index d9818d185d..c690988d7a 100644 --- a/settings.gradle +++ b/settings.gradle @@ -35,6 +35,7 @@ include 'ethereum:trie' include 'metrics' include 'pantheon' include 'services:kvstore' +include 'services:pipeline' include 'services:queue' include 'services:util' include 'testutil' From 2a75bd91688d30c7f68032e907e64e825b3f4ca3 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 6 Mar 2019 16:40:48 +1000 Subject: [PATCH 02/39] Support introducing batches into the pipeline. --- .../services/pipeline/BatchingStage.java | 42 ++++++++++++++++ .../pantheon/services/pipeline/InputPipe.java | 4 ++ .../pantheon/services/pipeline/Pipe.java | 15 ++++++ .../services/pipeline/PipelineBuilder.java | 23 +++++++-- .../services/pipeline/BatchingStageTest.java | 48 +++++++++++++++++++ .../pipeline/PipelineBuilderTest.java | 38 +++++++++++++-- 6 files changed, 163 insertions(+), 7 deletions(-) create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java new file mode 100644 index 0000000000..c8e9c9ed5d --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java @@ -0,0 +1,42 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.List; + +public class BatchingStage implements Runnable { + + private final InputPipe inputPipe; + private final OutputPipe> outputPipe; + private final int maximumBatchSize; + + public BatchingStage( + final InputPipe inputPipe, + final OutputPipe> outputPipe, + final int maximumBatchSize) { + this.inputPipe = inputPipe; + this.outputPipe = outputPipe; + this.maximumBatchSize = maximumBatchSize; + } + + @Override + public void run() { + while (inputPipe.hasMore()) { + final List batch = inputPipe.getBatch(maximumBatchSize); + if (!batch.isEmpty()) { + outputPipe.put(batch); + } + } + outputPipe.close(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java index 24086dc03b..967313a32b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -12,8 +12,12 @@ */ package tech.pegasys.pantheon.services.pipeline; +import java.util.List; + public interface InputPipe { boolean hasMore(); T get(); + + List getBatch(int maximumBatchSize); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index 62a5d4ff05..f769503c70 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -12,6 +12,9 @@ */ package tech.pegasys.pantheon.services.pipeline; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; @@ -59,6 +62,18 @@ public T get() { return null; } + @Override + public List getBatch(final int maximumBatchSize) { + final T nextItem = get(); + if (nextItem == null) { + return Collections.emptyList(); + } + final List batch = new ArrayList<>(); + batch.add(nextItem); + queue.drainTo(batch, maximumBatchSize - 1); + return batch; + } + @Override public void put(final T value) { while (isOpen()) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 6c922d9f16..a6f97cf52c 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -12,11 +12,13 @@ */ package tech.pegasys.pantheon.services.pipeline; +import static java.util.Collections.emptyList; import static java.util.Collections.singleton; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; +import java.util.List; import java.util.function.Consumer; import java.util.function.Function; @@ -36,15 +38,30 @@ public static PipelineBuilder createPipelineFrom(final Iterator taskSt return new PipelineBuilder<>(singleton(sourceStage), pipe); } + public static PipelineBuilder createPipelineFrom(final InputPipe pipe) { + return new PipelineBuilder<>(emptyList(), pipe); + } + public PipelineBuilder thenProcessWith(final Function processor) { - final ArrayList newStages = new ArrayList<>(stages); final Pipe outputPipe = new Pipe<>(10); final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); - newStages.add(processStage); - return new PipelineBuilder<>(newStages, outputPipe); + return addStage(processStage, outputPipe); + } + + public PipelineBuilder> inBatches(final int maximumBatchSize) { + final Pipe> outputPipe = new Pipe<>(10); + final BatchingStage batchingStage = + new BatchingStage<>(pipeEnd, outputPipe, maximumBatchSize); + return addStage(batchingStage, outputPipe); } public Pipeline andFinishWith(final Consumer completer) { return new Pipeline(stages, new CompleterStage<>(pipeEnd, completer)); } + + private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { + final ArrayList newStages = new ArrayList<>(stages); + newStages.add(stage); + return new PipelineBuilder<>(newStages, outputPipe); + } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java new file mode 100644 index 0000000000..110ac6b7a0 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static java.util.Arrays.asList; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; + +import org.junit.Test; + +public class BatchingStageTest { + + private final Pipe inputPipe = new Pipe<>(10); + private final Pipe> outputPipe = new Pipe<>(10); + private final BatchingStage stage = new BatchingStage<>(inputPipe, outputPipe, 3); + + @Test + public void shouldCreateBatches() { + for (int i = 1; i <= 8; i++) { + inputPipe.put(i); + } + inputPipe.close(); + + stage.run(); + + assertThat(outputPipe.hasMore()).isTrue(); + assertThat(outputPipe.get()).isEqualTo(asList(1, 2, 3)); + + assertThat(outputPipe.hasMore()).isTrue(); + assertThat(outputPipe.get()).isEqualTo(asList(4, 5, 6)); + + assertThat(outputPipe.hasMore()).isTrue(); + assertThat(outputPipe.get()).isEqualTo(asList(7, 8)); + + assertThat(outputPipe.hasMore()).isFalse(); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 3d8e903613..2db55476f1 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -13,17 +13,19 @@ package tech.pegasys.pantheon.services.pipeline; import static com.google.common.primitives.Ints.asList; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.junit.AfterClass; @@ -45,7 +47,7 @@ public class PipelineBuilderTest { @AfterClass public static void afterClass() throws Exception { executorService.shutdownNow(); - if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) { + if (!executorService.awaitTermination(10, SECONDS)) { fail("Executor service did not shut down cleanly"); } } @@ -55,7 +57,7 @@ public void shouldPipeTasksFromSupplierToCompleter() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks).andFinishWith(output::add); final CompletableFuture result = pipeline.start(executorService); - result.get(10, TimeUnit.SECONDS); + result.get(10, SECONDS); assertThat(output).containsExactly(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); } @@ -68,9 +70,37 @@ public void shouldPassInputThroughIntermediateStage() throws Exception { .andFinishWith(output::add); final CompletableFuture result = pipeline.start(executorService); - result.get(10, TimeUnit.SECONDS); + result.get(10, SECONDS); assertThat(output) .containsExactly( "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12", "13", "14", "15"); } + + @Test + public void shouldCombineIntoBatches() throws Exception { + final Pipe input = new Pipe<>(20); + tasks.forEachRemaining(input::put); + final BlockingQueue> output = new ArrayBlockingQueue<>(10); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(input).inBatches(6).andFinishWith(output::offer); + + final CompletableFuture result = pipeline.start(executorService); + + assertThat(output.poll(10, SECONDS)).containsExactly(1, 2, 3, 4, 5, 6); + assertThat(output.poll(10, SECONDS)).containsExactly(7, 8, 9, 10, 11, 12); + assertThat(output.poll(10, SECONDS)).containsExactly(13, 14, 15); + + assertThat(output).isEmpty(); + assertThat(result).isNotDone(); + + // Should not wait to fill the batch. + input.put(16); + assertThat(output.poll(10, SECONDS)).containsExactly(16); + input.put(17); + assertThat(output.poll(10, SECONDS)).containsExactly(17); + + input.close(); + result.get(10, SECONDS); + assertThat(output).isEmpty(); + } } From 1eecb21b755ade0642f0bcc04f05a7f1bfd64146 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 6 Mar 2019 16:45:25 +1000 Subject: [PATCH 03/39] Make buffer size configurable. --- .../services/pipeline/PipelineBuilder.java | 23 +++++++++++-------- .../pipeline/PipelineBuilderTest.java | 7 +++--- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index a6f97cf52c..8377f6e4d7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -26,30 +26,35 @@ public class PipelineBuilder { private final Collection stages; private final InputPipe pipeEnd; + private final int bufferSize; - public PipelineBuilder(final Collection stages, final InputPipe pipeEnd) { + public PipelineBuilder( + final Collection stages, final InputPipe pipeEnd, final int bufferSize) { this.stages = stages; this.pipeEnd = pipeEnd; + this.bufferSize = bufferSize; } - public static PipelineBuilder createPipelineFrom(final Iterator taskStream) { - final Pipe pipe = new Pipe<>(10); + public static PipelineBuilder createPipelineFrom( + final Iterator taskStream, final int bufferSize) { + final Pipe pipe = new Pipe<>(bufferSize); final SourceStage sourceStage = new SourceStage<>(taskStream, pipe); - return new PipelineBuilder<>(singleton(sourceStage), pipe); + return new PipelineBuilder<>(singleton(sourceStage), pipe, bufferSize); } - public static PipelineBuilder createPipelineFrom(final InputPipe pipe) { - return new PipelineBuilder<>(emptyList(), pipe); + public static PipelineBuilder createPipelineFrom( + final InputPipe pipe, final int bufferSize) { + return new PipelineBuilder<>(emptyList(), pipe, bufferSize); } public PipelineBuilder thenProcessWith(final Function processor) { - final Pipe outputPipe = new Pipe<>(10); + final Pipe outputPipe = new Pipe<>(bufferSize); final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); return addStage(processStage, outputPipe); } public PipelineBuilder> inBatches(final int maximumBatchSize) { - final Pipe> outputPipe = new Pipe<>(10); + final Pipe> outputPipe = new Pipe<>(bufferSize / maximumBatchSize + 1); final BatchingStage batchingStage = new BatchingStage<>(pipeEnd, outputPipe, maximumBatchSize); return addStage(batchingStage, outputPipe); @@ -62,6 +67,6 @@ public Pipeline andFinishWith(final Consumer completer) { private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { final ArrayList newStages = new ArrayList<>(stages); newStages.add(stage); - return new PipelineBuilder<>(newStages, outputPipe); + return new PipelineBuilder<>(newStages, outputPipe, bufferSize); } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 2db55476f1..eead095aeb 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -55,7 +55,8 @@ public static void afterClass() throws Exception { @Test public void shouldPipeTasksFromSupplierToCompleter() throws Exception { final List output = new ArrayList<>(); - final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks).andFinishWith(output::add); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10).andFinishWith(output::add); final CompletableFuture result = pipeline.start(executorService); result.get(10, SECONDS); assertThat(output).containsExactly(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); @@ -65,7 +66,7 @@ public void shouldPipeTasksFromSupplierToCompleter() throws Exception { public void shouldPassInputThroughIntermediateStage() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks) + PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcessWith(Object::toString) .andFinishWith(output::add); @@ -82,7 +83,7 @@ public void shouldCombineIntoBatches() throws Exception { tasks.forEachRemaining(input::put); final BlockingQueue> output = new ArrayBlockingQueue<>(10); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(input).inBatches(6).andFinishWith(output::offer); + PipelineBuilder.createPipelineFrom(input, 20).inBatches(6).andFinishWith(output::offer); final CompletableFuture result = pipeline.start(executorService); From fa06ace60f0ee2169166826a09966649a7da00b5 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Thu, 7 Mar 2019 19:44:30 +1000 Subject: [PATCH 04/39] Add async processing and parallel processing support to pipelines. --- services/pipeline/build.gradle | 1 + .../services/pipeline/AsyncProcessStage.java | 95 +++++++++++++++++++ .../services/pipeline/FlatMapStage.java | 31 ++++++ .../pantheon/services/pipeline/InputPipe.java | 2 + .../services/pipeline/OutputPipe.java | 2 + .../pantheon/services/pipeline/Pipe.java | 18 +++- .../pantheon/services/pipeline/Pipeline.java | 4 +- .../services/pipeline/PipelineBuilder.java | 28 +++++- .../services/pipeline/SourceStage.java | 5 +- .../pipeline/PipelineBuilderTest.java | 62 +++++++++++- .../services/pipeline/ProcessStageTest.java | 4 +- 11 files changed, 242 insertions(+), 10 deletions(-) create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java diff --git a/services/pipeline/build.gradle b/services/pipeline/build.gradle index 4d2b6df9f2..f5f687727e 100644 --- a/services/pipeline/build.gradle +++ b/services/pipeline/build.gradle @@ -36,5 +36,6 @@ dependencies { testImplementation 'junit:junit' testImplementation 'org.assertj:assertj-core' + testImplementation 'org.awaitility:awaitility' testImplementation 'org.mockito:mockito-core' } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java new file mode 100644 index 0000000000..95b66a2b29 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java @@ -0,0 +1,95 @@ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class AsyncProcessStage implements Runnable { + private static final Logger LOG = LogManager.getLogger(); + private final InputPipe inputPipe; + private final OutputPipe outputPipe; + private final Function> processor; + private final Collection> inProgress; + private final int maxConcurrency; + + public AsyncProcessStage( + final InputPipe inputPipe, + final OutputPipe outputPipe, + final Function> processor, + final int maxConcurrency) { + this.inputPipe = inputPipe; + this.outputPipe = outputPipe; + this.processor = processor; + this.maxConcurrency = maxConcurrency; + this.inProgress = new ArrayList<>(maxConcurrency); + } + + @Override + public void run() { + final Thread stageThread = Thread.currentThread(); + while (inputPipe.hasMore()) { + if (inProgress.size() < maxConcurrency) { + final I value = inputPipe.get(); + if (value != null) { + final CompletableFuture future = processor.apply(value); + // When the future completes, interrupt so if we're waiting for new input we wake up and + // schedule the output. + future.whenComplete((result, error) -> stageThread.interrupt()); + inProgress.add(future); + } + + outputCompletedTasks(0); + } else { + outputNextCompletedTask(); + } + } + + while (!inProgress.isEmpty()) { + outputNextCompletedTask(); + } + + outputPipe.close(); + } + + private void outputNextCompletedTask() { + try { + waitForAnyFutureToComplete(); + outputCompletedTasks(1); + } catch (final InterruptedException e) { + LOG.trace("Interrupted while waiting for processing to complete", e); + } catch (final ExecutionException e) { + LOG.error("Processing failed and we don't handle exceptions properly yet", e); + } catch (final TimeoutException e) { + // Ignore and go back around the loop. + } + } + + @SuppressWarnings("rawtypes") + private void waitForAnyFutureToComplete() + throws InterruptedException, ExecutionException, TimeoutException { + CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])) + .get(1, TimeUnit.SECONDS); + } + + private void outputCompletedTasks(final int minTasksToOutput) { + int outputTasks = 0; + for (final Iterator> i = inProgress.iterator(); + i.hasNext() && (outputTasks < minTasksToOutput || outputPipe.hasRemainingCapacity()); ) { + final CompletableFuture process = i.next(); + final O result = process.getNow(null); + if (result != null) { + outputPipe.put(result); + i.remove(); + outputTasks++; + } + } + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java new file mode 100644 index 0000000000..30e44bb76b --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java @@ -0,0 +1,31 @@ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.function.Function; +import java.util.stream.Stream; + +public class FlatMapStage implements Runnable { + + private final InputPipe inputPipe; + private final OutputPipe outputPipe; + private final Function> mapper; + + public FlatMapStage( + final InputPipe inputPipe, + final OutputPipe outputPipe, + final Function> mapper) { + this.inputPipe = inputPipe; + this.outputPipe = outputPipe; + this.mapper = mapper; + } + + @Override + public void run() { + while (inputPipe.hasMore()) { + final I value = inputPipe.get(); + if (value != null) { + mapper.apply(value).forEach(outputPipe::put); + } + } + outputPipe.close(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java index 967313a32b..db61a950a2 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -19,5 +19,7 @@ public interface InputPipe { T get(); + T poll(); + List getBatch(int maximumBatchSize); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java index 8a1e40ffda..c0e37d0d7a 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -18,4 +18,6 @@ public interface OutputPipe { void put(T value); void close(); + + boolean hasRemainingCapacity(); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index f769503c70..cbd12b973e 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -42,6 +42,11 @@ public void close() { closed.set(true); } + @Override + public boolean hasRemainingCapacity() { + return queue.remainingCapacity() > 0; + } + @Override public boolean hasMore() { return !closed.get() || !queue.isEmpty(); @@ -49,19 +54,24 @@ public boolean hasMore() { @Override public T get() { - while (isOpen() || !queue.isEmpty()) { - try { + try { + while (isOpen() || !queue.isEmpty()) { final T value = queue.poll(1, TimeUnit.SECONDS); if (value != null) { return value; } - } catch (final InterruptedException e) { - LOG.trace("Interrupted while waiting for next item", e); } + } catch (final InterruptedException e) { + LOG.trace("Interrupted while waiting for next item", e); } return null; } + @Override + public T poll() { + return queue.poll(); + } + @Override public List getBatch(final int maximumBatchSize) { final T nextItem = get(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index f31a8bf829..0384ddec4a 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -27,8 +27,8 @@ public Pipeline(final Collection stages, final CompleterStage compl } public CompletableFuture start(final ExecutorService executorService) { - stages.forEach(executorService::submit); - executorService.submit(completerStage); + stages.forEach(executorService::execute); + executorService.execute(completerStage); return completerStage.getFuture(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 8377f6e4d7..8ed77f06eb 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -19,8 +19,10 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import java.util.function.Function; +import java.util.stream.Stream; public class PipelineBuilder { @@ -47,12 +49,30 @@ public static PipelineBuilder createPipelineFrom( return new PipelineBuilder<>(emptyList(), pipe, bufferSize); } - public PipelineBuilder thenProcessWith(final Function processor) { + public PipelineBuilder thenProcess(final Function processor) { final Pipe outputPipe = new Pipe<>(bufferSize); final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); return addStage(processStage, outputPipe); } + public PipelineBuilder thenProcessInParallel(final Function processor, final int numberOfThreads) { + final Pipe outputPipe = new Pipe<>(bufferSize); + final ArrayList newStages = new ArrayList<>(stages); + for (int i = 0; i < numberOfThreads; i++) { + final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); + newStages.add(processStage); + } + return new PipelineBuilder<>(newStages, outputPipe, bufferSize); + } + + public PipelineBuilder thenProcessAsync( + final Function> processor, final int maxConcurrency) { + final Pipe outputPipe = new Pipe<>(bufferSize); + final AsyncProcessStage stage = + new AsyncProcessStage<>(pipeEnd, outputPipe, processor, maxConcurrency); + return addStage(stage, outputPipe); + } + public PipelineBuilder> inBatches(final int maximumBatchSize) { final Pipe> outputPipe = new Pipe<>(bufferSize / maximumBatchSize + 1); final BatchingStage batchingStage = @@ -60,6 +80,12 @@ public PipelineBuilder> inBatches(final int maximumBatchSize) { return addStage(batchingStage, outputPipe); } + public PipelineBuilder theFlatMap(final Function> mapper, final int newBufferSize) { + final Pipe outputPipe = new Pipe<>(newBufferSize); + final FlatMapStage stage = new FlatMapStage<>(pipeEnd, outputPipe, mapper); + return addStage(stage, outputPipe); + } + public Pipeline andFinishWith(final Consumer completer) { return new Pipeline(stages, new CompleterStage<>(pipeEnd, completer)); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java index 5d3eebefd3..dd39472700 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java @@ -26,7 +26,10 @@ class SourceStage implements Runnable { @Override public void run() { while (output.isOpen() && source.hasNext()) { - output.put(source.next()); + final T value = source.next(); + if (value != null) { + output.put(value); + } } output.close(); } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index eead095aeb..fe6712e0db 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -13,21 +13,25 @@ package tech.pegasys.pantheon.services.pipeline; import static com.google.common.primitives.Ints.asList; +import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.awaitility.Awaitility; import org.junit.AfterClass; import org.junit.Test; @@ -67,7 +71,7 @@ public void shouldPassInputThroughIntermediateStage() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) - .thenProcessWith(Object::toString) + .thenProcess(Object::toString) .andFinishWith(output::add); final CompletableFuture result = pipeline.start(executorService); @@ -104,4 +108,60 @@ public void shouldCombineIntoBatches() throws Exception { result.get(10, SECONDS); assertThat(output).isEmpty(); } + + @Test + public void shouldProcessAsync() throws Exception { + final List output = new ArrayList<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .thenProcessAsync(value -> completedFuture(Integer.toString(value)), 3) + .andFinishWith(output::add); + final CompletableFuture result = pipeline.start(executorService); + result.get(10, SECONDS); + assertThat(output) + .containsExactlyInAnyOrder( + "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12", "13", "14", "15"); + } + + @Test + public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { + final List output = new ArrayList<>(); + final List> futures = new CopyOnWriteArrayList<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(asList(1, 2, 3, 4, 5, 6, 7).iterator(), 10) + .thenProcessAsync( + value -> { + final CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + return future; + }, + 3) + .andFinishWith(output::add); + final CompletableFuture result = pipeline.start(executorService); + + waitForSize(futures, 3); + + assertThat(result).isNotDone(); + + // Completing one task should cause another to be started. + futures.get(1).complete("2"); + waitForSize(futures, 4); + + futures.get(0).complete("1"); + futures.get(2).complete("3"); + futures.get(3).complete("4"); + + waitForSize(futures, 7); + futures.get(4).complete("5"); + futures.get(5).complete("6"); + futures.get(6).complete("7"); + + result.get(10, SECONDS); + assertThat(output).containsExactly("2", "1", "3", "4", "5", "6", "7"); + } + + private void waitForSize(final Collection collection, final int targetSize) { + Awaitility.waitAtMost(10, SECONDS) + .untilAsserted(() -> assertThat(collection).hasSize(targetSize)); + } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java index 936a06d273..851ef2e744 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java @@ -14,6 +14,8 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Locale; + import org.junit.Test; public class ProcessStageTest { @@ -22,7 +24,7 @@ public class ProcessStageTest { private final Pipe output = new Pipe<>(10); private final ProcessStage stage = - new ProcessStage<>(input, output, String::toLowerCase); + new ProcessStage<>(input, output, value -> value.toLowerCase(Locale.UK)); @Test public void shouldCloseOutputPipeWhenInputIsClosed() { From 7fa7c592be2ec937fc97faa90b57d3265366d681 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 11:32:43 +1000 Subject: [PATCH 05/39] Extract common looping code. --- .../services/pipeline/AsyncProcessStage.java | 15 ++++++- .../services/pipeline/BatchingStage.java | 23 ++++------- .../services/pipeline/FlatMapStage.java | 34 +++++++-------- .../services/pipeline/InputOutputStage.java | 37 +++++++++++++++++ .../services/pipeline/PipelineBuilder.java | 20 +++++---- .../services/pipeline/ProcessStage.java | 22 +++------- .../services/pipeline/SingleStepStage.java | 17 ++++++++ .../services/pipeline/BatchingStageTest.java | 29 +++++++++---- .../services/pipeline/ProcessStageTest.java | 41 ++++++++----------- 9 files changed, 148 insertions(+), 90 deletions(-) create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java index 95b66a2b29..bfd13fd269 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java @@ -1,3 +1,15 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package tech.pegasys.pantheon.services.pipeline; import java.util.ArrayList; @@ -75,8 +87,7 @@ private void outputNextCompletedTask() { @SuppressWarnings("rawtypes") private void waitForAnyFutureToComplete() throws InterruptedException, ExecutionException, TimeoutException { - CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])) - .get(1, TimeUnit.SECONDS); + CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])).get(1, TimeUnit.SECONDS); } private void outputCompletedTasks(final int minTasksToOutput) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java index c8e9c9ed5d..021bf21199 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java @@ -14,29 +14,20 @@ import java.util.List; -public class BatchingStage implements Runnable { +public class BatchingStage implements SingleStepStage> { - private final InputPipe inputPipe; - private final OutputPipe> outputPipe; private final int maximumBatchSize; - public BatchingStage( - final InputPipe inputPipe, - final OutputPipe> outputPipe, - final int maximumBatchSize) { - this.inputPipe = inputPipe; - this.outputPipe = outputPipe; + public BatchingStage(final int maximumBatchSize) { this.maximumBatchSize = maximumBatchSize; } @Override - public void run() { - while (inputPipe.hasMore()) { - final List batch = inputPipe.getBatch(maximumBatchSize); - if (!batch.isEmpty()) { - outputPipe.put(batch); - } + public void processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { + + final List batch = inputPipe.getBatch(maximumBatchSize); + if (!batch.isEmpty()) { + outputPipe.put(batch); } - outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java index 30e44bb76b..74b084fc70 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java @@ -1,31 +1,33 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ package tech.pegasys.pantheon.services.pipeline; import java.util.function.Function; import java.util.stream.Stream; -public class FlatMapStage implements Runnable { +public class FlatMapStage implements SingleStepStage { - private final InputPipe inputPipe; - private final OutputPipe outputPipe; private final Function> mapper; - public FlatMapStage( - final InputPipe inputPipe, - final OutputPipe outputPipe, - final Function> mapper) { - this.inputPipe = inputPipe; - this.outputPipe = outputPipe; + public FlatMapStage(final Function> mapper) { this.mapper = mapper; } @Override - public void run() { - while (inputPipe.hasMore()) { - final I value = inputPipe.get(); - if (value != null) { - mapper.apply(value).forEach(outputPipe::put); - } + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + final I value = inputPipe.get(); + if (value != null) { + mapper.apply(value).forEach(outputPipe::put); } - outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java new file mode 100644 index 0000000000..d42fe4c17a --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +public class InputOutputStage implements Runnable { + + private final InputPipe inputPipe; + private final OutputPipe outputPipe; + private final SingleStepStage singleStepStage; + + public InputOutputStage( + final InputPipe inputPipe, + final OutputPipe outputPipe, + final SingleStepStage singleStepStage) { + this.inputPipe = inputPipe; + this.outputPipe = outputPipe; + this.singleStepStage = singleStepStage; + } + + @Override + public void run() { + while (inputPipe.hasMore()) { + singleStepStage.processNextInput(inputPipe, outputPipe); + } + outputPipe.close(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 8ed77f06eb..fab6d3d25c 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -51,15 +51,18 @@ public static PipelineBuilder createPipelineFrom( public PipelineBuilder thenProcess(final Function processor) { final Pipe outputPipe = new Pipe<>(bufferSize); - final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); + final Runnable processStage = + new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); return addStage(processStage, outputPipe); } - public PipelineBuilder thenProcessInParallel(final Function processor, final int numberOfThreads) { + public PipelineBuilder thenProcessInParallel( + final Function processor, final int numberOfThreads) { final Pipe outputPipe = new Pipe<>(bufferSize); final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { - final ProcessStage processStage = new ProcessStage<>(pipeEnd, outputPipe, processor); + final Runnable processStage = + new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); newStages.add(processStage); } return new PipelineBuilder<>(newStages, outputPipe, bufferSize); @@ -75,14 +78,15 @@ public PipelineBuilder thenProcessAsync( public PipelineBuilder> inBatches(final int maximumBatchSize) { final Pipe> outputPipe = new Pipe<>(bufferSize / maximumBatchSize + 1); - final BatchingStage batchingStage = - new BatchingStage<>(pipeEnd, outputPipe, maximumBatchSize); - return addStage(batchingStage, outputPipe); + final Runnable stage = + new InputOutputStage<>(pipeEnd, outputPipe, new BatchingStage<>(maximumBatchSize)); + return addStage(stage, outputPipe); } - public PipelineBuilder theFlatMap(final Function> mapper, final int newBufferSize) { + public PipelineBuilder thenFlatMap( + final Function> mapper, final int newBufferSize) { final Pipe outputPipe = new Pipe<>(newBufferSize); - final FlatMapStage stage = new FlatMapStage<>(pipeEnd, outputPipe, mapper); + final Runnable stage = new InputOutputStage<>(pipeEnd, outputPipe, new FlatMapStage<>(mapper)); return addStage(stage, outputPipe); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java index de845a545f..079d5ea559 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java @@ -14,29 +14,19 @@ import java.util.function.Function; -class ProcessStage implements Runnable { +class ProcessStage implements SingleStepStage { - private final InputPipe inputPipe; - private final OutputPipe outputPipe; private final Function processor; - public ProcessStage( - final InputPipe inputPipe, - final OutputPipe outputPipe, - final Function processor) { - this.inputPipe = inputPipe; - this.outputPipe = outputPipe; + public ProcessStage(final Function processor) { this.processor = processor; } @Override - public void run() { - while (inputPipe.hasMore()) { - final I value = inputPipe.get(); - if (value != null) { - outputPipe.put(processor.apply(value)); - } + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + final I value = inputPipe.get(); + if (value != null) { + outputPipe.put(processor.apply(value)); } - outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java new file mode 100644 index 0000000000..844a8beb02 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java @@ -0,0 +1,17 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +public interface SingleStepStage { + void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java index 110ac6b7a0..3da767037b 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java @@ -14,6 +14,8 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verifyZeroInteractions; import java.util.List; @@ -23,7 +25,7 @@ public class BatchingStageTest { private final Pipe inputPipe = new Pipe<>(10); private final Pipe> outputPipe = new Pipe<>(10); - private final BatchingStage stage = new BatchingStage<>(inputPipe, outputPipe, 3); + private final BatchingStage stage = new BatchingStage<>(3); @Test public void shouldCreateBatches() { @@ -32,17 +34,26 @@ public void shouldCreateBatches() { } inputPipe.close(); - stage.run(); + stage.processNextInput(inputPipe, outputPipe); - assertThat(outputPipe.hasMore()).isTrue(); - assertThat(outputPipe.get()).isEqualTo(asList(1, 2, 3)); + assertThat(outputPipe.poll()).isEqualTo(asList(1, 2, 3)); + assertThat(outputPipe.poll()).isNull(); - assertThat(outputPipe.hasMore()).isTrue(); - assertThat(outputPipe.get()).isEqualTo(asList(4, 5, 6)); + stage.processNextInput(inputPipe, outputPipe); + assertThat(outputPipe.poll()).isEqualTo(asList(4, 5, 6)); + assertThat(outputPipe.poll()).isNull(); - assertThat(outputPipe.hasMore()).isTrue(); - assertThat(outputPipe.get()).isEqualTo(asList(7, 8)); + stage.processNextInput(inputPipe, outputPipe); + assertThat(outputPipe.poll()).isEqualTo(asList(7, 8)); + assertThat(outputPipe.poll()).isNull(); + } - assertThat(outputPipe.hasMore()).isFalse(); + @Test + public void shouldNotOutputItemWhenInputIsClosed() { + @SuppressWarnings("unchecked") + final OutputPipe> outputPipe = mock(OutputPipe.class); + inputPipe.close(); + stage.processNextInput(inputPipe, outputPipe); + verifyZeroInteractions(outputPipe); } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java index 851ef2e744..56620c5f75 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java @@ -13,8 +13,12 @@ package tech.pegasys.pantheon.services.pipeline; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; -import java.util.Locale; +import java.util.function.Function; import org.junit.Test; @@ -23,36 +27,27 @@ public class ProcessStageTest { private final Pipe input = new Pipe<>(10); private final Pipe output = new Pipe<>(10); - private final ProcessStage stage = - new ProcessStage<>(input, output, value -> value.toLowerCase(Locale.UK)); + @SuppressWarnings("unchecked") + private final Function processor = mock(Function.class); - @Test - public void shouldCloseOutputPipeWhenInputIsClosed() { - input.put("A"); - input.put("B"); - input.put("C"); - input.close(); - - stage.run(); - - assertThat(output.isOpen()).isFalse(); - } + private final ProcessStage stage = new ProcessStage<>(processor); @Test public void shouldApplyFunctionToItems() { + when(processor.apply("A")).thenReturn("a"); input.put("A"); - input.put("B"); - input.put("C"); - input.close(); - stage.run(); + stage.processNextInput(input, output); assertThat(output.hasMore()).isTrue(); assertThat(output.get()).isEqualTo("a"); - assertThat(output.hasMore()).isTrue(); - assertThat(output.get()).isEqualTo("b"); - assertThat(output.hasMore()).isTrue(); - assertThat(output.get()).isEqualTo("c"); - assertThat(output.hasMore()).isFalse(); + verify(processor).apply("A"); + } + + @Test + public void shouldSkipProcessingWhenInputIsClosed() { + input.close(); + stage.processNextInput(input, output); + verifyZeroInteractions(processor); } } From 91b53caaa83a4ae8ef94c110427ed33277fc9177 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 11:45:22 +1000 Subject: [PATCH 06/39] Introduce finalize step to make InputOutputStep more useful. --- .../services/pipeline/InputOutputStage.java | 1 + .../services/pipeline/SingleStepStage.java | 2 + .../pipeline/InputOutputStageTest.java | 76 +++++++++++++++++++ 3 files changed, 79 insertions(+) create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java index d42fe4c17a..6bcadc39ed 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java @@ -32,6 +32,7 @@ public void run() { while (inputPipe.hasMore()) { singleStepStage.processNextInput(inputPipe, outputPipe); } + singleStepStage.finalize(outputPipe); outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java index 844a8beb02..7164f81885 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java @@ -14,4 +14,6 @@ public interface SingleStepStage { void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); + + default void finalize(final OutputPipe outputPipe) {} } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java new file mode 100644 index 0000000000..457d356e63 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java @@ -0,0 +1,76 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; + +import java.util.Locale; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class InputOutputStageTest { + + private final Pipe inputPipe = new Pipe<>(10); + private final Pipe outputPipe = new Pipe<>(10); + @Mock private SingleStepStage singleStep; + private InputOutputStage stage; + + @Before + public void setUp() { + stage = new InputOutputStage<>(inputPipe, outputPipe, singleStep); + doAnswer( + invocation -> { + outputPipe.put(inputPipe.get().toLowerCase(Locale.UK)); + return null; + }) + .when(singleStep) + .processNextInput(inputPipe, outputPipe); + } + + @Test + public void shouldCallSingleStepStageForEachInput() { + inputPipe.put("A"); + inputPipe.put("B"); + inputPipe.put("C"); + inputPipe.close(); + + stage.run(); + + assertThat(outputPipe.poll()).isEqualTo("a"); + assertThat(outputPipe.poll()).isEqualTo("b"); + assertThat(outputPipe.poll()).isEqualTo("c"); + assertThat(outputPipe.poll()).isNull(); + + verify(singleStep, times(3)).processNextInput(inputPipe, outputPipe); + } + + @Test + public void shouldFinalizeSingleStepStageAndCloseOutputPipeWhenInputCloses() { + inputPipe.close(); + + stage.run(); + + verify(singleStep).finalize(outputPipe); + verifyNoMoreInteractions(singleStep); + assertThat(outputPipe.isOpen()).isFalse(); + } +} From a3970c1f33a0c4fd1d8e6b66fb56ad28a187d69f Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 11:49:06 +1000 Subject: [PATCH 07/39] Move AsyncProcessStage over to SingleStepStage. --- .../services/pipeline/AsyncProcessStage.java | 54 ++++++++----------- .../services/pipeline/PipelineBuilder.java | 5 +- 2 files changed, 26 insertions(+), 33 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java index bfd13fd269..1c5f8b7b6c 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java @@ -24,57 +24,49 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -public class AsyncProcessStage implements Runnable { +public class AsyncProcessStage implements SingleStepStage { private static final Logger LOG = LogManager.getLogger(); - private final InputPipe inputPipe; - private final OutputPipe outputPipe; private final Function> processor; private final Collection> inProgress; private final int maxConcurrency; public AsyncProcessStage( - final InputPipe inputPipe, - final OutputPipe outputPipe, - final Function> processor, - final int maxConcurrency) { - this.inputPipe = inputPipe; - this.outputPipe = outputPipe; + final Function> processor, final int maxConcurrency) { this.processor = processor; this.maxConcurrency = maxConcurrency; this.inProgress = new ArrayList<>(maxConcurrency); } @Override - public void run() { - final Thread stageThread = Thread.currentThread(); - while (inputPipe.hasMore()) { - if (inProgress.size() < maxConcurrency) { - final I value = inputPipe.get(); - if (value != null) { - final CompletableFuture future = processor.apply(value); - // When the future completes, interrupt so if we're waiting for new input we wake up and - // schedule the output. - future.whenComplete((result, error) -> stageThread.interrupt()); - inProgress.add(future); - } - - outputCompletedTasks(0); - } else { - outputNextCompletedTask(); + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + if (inProgress.size() < maxConcurrency) { + final I value = inputPipe.get(); + if (value != null) { + final CompletableFuture future = processor.apply(value); + // When the future completes, interrupt so if we're waiting for new input we wake up and + // schedule the output. + final Thread stageThread = Thread.currentThread(); + future.whenComplete((result, error) -> stageThread.interrupt()); + inProgress.add(future); } + + outputCompletedTasks(0, outputPipe); + } else { + outputNextCompletedTask(outputPipe); } + } + @Override + public void finalize(final OutputPipe outputPipe) { while (!inProgress.isEmpty()) { - outputNextCompletedTask(); + outputNextCompletedTask(outputPipe); } - - outputPipe.close(); } - private void outputNextCompletedTask() { + private void outputNextCompletedTask(final OutputPipe outputPipe) { try { waitForAnyFutureToComplete(); - outputCompletedTasks(1); + outputCompletedTasks(1, outputPipe); } catch (final InterruptedException e) { LOG.trace("Interrupted while waiting for processing to complete", e); } catch (final ExecutionException e) { @@ -90,7 +82,7 @@ private void waitForAnyFutureToComplete() CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])).get(1, TimeUnit.SECONDS); } - private void outputCompletedTasks(final int minTasksToOutput) { + private void outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { int outputTasks = 0; for (final Iterator> i = inProgress.iterator(); i.hasNext() && (outputTasks < minTasksToOutput || outputPipe.hasRemainingCapacity()); ) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index fab6d3d25c..4a2635c1a8 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -71,8 +71,9 @@ public PipelineBuilder thenProcessInParallel( public PipelineBuilder thenProcessAsync( final Function> processor, final int maxConcurrency) { final Pipe outputPipe = new Pipe<>(bufferSize); - final AsyncProcessStage stage = - new AsyncProcessStage<>(pipeEnd, outputPipe, processor, maxConcurrency); + final Runnable stage = + new InputOutputStage<>( + pipeEnd, outputPipe, new AsyncProcessStage<>(processor, maxConcurrency)); return addStage(stage, outputPipe); } From c9b23f1f8a408969bc09c665dd685b2ec045ea78 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 11:56:43 +1000 Subject: [PATCH 08/39] Reduce duplication. --- .../services/pipeline/PipelineBuilder.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 4a2635c1a8..a1b8c07d1a 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -50,10 +50,8 @@ public static PipelineBuilder createPipelineFrom( } public PipelineBuilder thenProcess(final Function processor) { - final Pipe outputPipe = new Pipe<>(bufferSize); - final Runnable processStage = - new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); - return addStage(processStage, outputPipe); + final SingleStepStage singleStepStage = new ProcessStage<>(processor); + return addStage(singleStepStage); } public PipelineBuilder thenProcessInParallel( @@ -70,33 +68,35 @@ public PipelineBuilder thenProcessInParallel( public PipelineBuilder thenProcessAsync( final Function> processor, final int maxConcurrency) { - final Pipe outputPipe = new Pipe<>(bufferSize); - final Runnable stage = - new InputOutputStage<>( - pipeEnd, outputPipe, new AsyncProcessStage<>(processor, maxConcurrency)); - return addStage(stage, outputPipe); + return addStage(new AsyncProcessStage<>(processor, maxConcurrency)); } public PipelineBuilder> inBatches(final int maximumBatchSize) { - final Pipe> outputPipe = new Pipe<>(bufferSize / maximumBatchSize + 1); - final Runnable stage = - new InputOutputStage<>(pipeEnd, outputPipe, new BatchingStage<>(maximumBatchSize)); - return addStage(stage, outputPipe); + return addStage(new BatchingStage<>(maximumBatchSize), bufferSize / maximumBatchSize + 1); } public PipelineBuilder thenFlatMap( final Function> mapper, final int newBufferSize) { - final Pipe outputPipe = new Pipe<>(newBufferSize); - final Runnable stage = new InputOutputStage<>(pipeEnd, outputPipe, new FlatMapStage<>(mapper)); - return addStage(stage, outputPipe); + return addStage(new FlatMapStage<>(mapper), newBufferSize); } public Pipeline andFinishWith(final Consumer completer) { return new Pipeline(stages, new CompleterStage<>(pipeEnd, completer)); } + private PipelineBuilder addStage(final SingleStepStage singleStepStage) { + return addStage(singleStepStage, bufferSize); + } + + private PipelineBuilder addStage( + final SingleStepStage singleStepStage, final int newBufferSize) { + final Pipe outputPipe = new Pipe<>(newBufferSize); + final Runnable processStage = new InputOutputStage<>(pipeEnd, outputPipe, singleStepStage); + return addStage(processStage, outputPipe); + } + private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { - final ArrayList newStages = new ArrayList<>(stages); + final List newStages = new ArrayList<>(stages); newStages.add(stage); return new PipelineBuilder<>(newStages, outputPipe, bufferSize); } From 2ed23fc21ab6c88cb5be6e424f9cc8a854b0c7eb Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 12:17:30 +1000 Subject: [PATCH 09/39] Add tests for flat map. --- .../services/pipeline/FlatMapStageTest.java | 56 +++++++++++++++++++ .../pipeline/PipelineBuilderTest.java | 17 ++++++ 2 files changed, 73 insertions(+) create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java new file mode 100644 index 0000000000..da07bac1ba --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +import java.util.function.Function; +import java.util.stream.Stream; + +import org.junit.Test; + +public class FlatMapStageTest { + + private final Pipe input = new Pipe<>(10); + private final Pipe output = new Pipe<>(10); + + @SuppressWarnings("unchecked") + private final Function> mapper = mock(Function.class); + + private final FlatMapStage stage = new FlatMapStage<>(mapper); + + @Test + public void shouldOutputItemsFromReturnedStreamIndividually() { + when(mapper.apply("A")).thenReturn(Stream.of("a", "b", "c")); + input.put("A"); + + stage.processNextInput(input, output); + + assertThat(output.poll()).isEqualTo("a"); + assertThat(output.poll()).isEqualTo("b"); + assertThat(output.poll()).isEqualTo("c"); + assertThat(output.poll()).isNull(); + verify(mapper).apply("A"); + } + + @Test + public void shouldSkipProcessingWhenInputIsClosed() { + input.close(); + stage.processNextInput(input, output); + verifyZeroInteractions(mapper); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index fe6712e0db..a48a368525 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -29,6 +29,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; +import java.util.stream.Stream; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.awaitility.Awaitility; @@ -160,6 +161,22 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { assertThat(output).containsExactly("2", "1", "3", "4", "5", "6", "7"); } + @Test + public void shouldFlatMapItems() throws Exception { + final List output = new ArrayList<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .thenFlatMap(input -> Stream.of(input, input * 2), 20) + .andFinishWith(output::add); + + pipeline.start(executorService).get(10, SECONDS); + + assertThat(output) + .containsExactly( + 1, 2, 2, 4, 3, 6, 4, 8, 5, 10, 6, 12, 7, 14, 8, 16, 9, 18, 10, 20, 11, 22, 12, 24, 13, + 26, 14, 28, 15, 30); + } + private void waitForSize(final Collection collection, final int targetSize) { Awaitility.waitAtMost(10, SECONDS) .untilAsserted(() -> assertThat(collection).hasSize(targetSize)); From 2f2ce8a4fcf4d00159c454fc80bbd038f2df7f99 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 12:59:32 +1000 Subject: [PATCH 10/39] Fix issue where output pipe from a parallel processing stage was closed before all threads finished processing their last entry. --- .../services/pipeline/PipelineBuilder.java | 5 +- .../services/pipeline/SharedOutputPipe.java | 47 +++++++++++++++++++ .../pipeline/PipelineBuilderTest.java | 36 ++++++++++++++ .../pipeline/SharedOutputPipeTest.java | 40 ++++++++++++++++ 4 files changed, 126 insertions(+), 2 deletions(-) create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index a1b8c07d1a..6e14d5d303 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -56,14 +56,15 @@ public PipelineBuilder thenProcess(final Function processor) { public PipelineBuilder thenProcessInParallel( final Function processor, final int numberOfThreads) { - final Pipe outputPipe = new Pipe<>(bufferSize); + final Pipe newPipeEnd = new Pipe<>(bufferSize); + final OutputPipe outputPipe = new SharedOutputPipe<>(newPipeEnd, numberOfThreads); final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { final Runnable processStage = new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); newStages.add(processStage); } - return new PipelineBuilder<>(newStages, outputPipe, bufferSize); + return new PipelineBuilder<>(newStages, newPipeEnd, bufferSize); } public PipelineBuilder thenProcessAsync( diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java new file mode 100644 index 0000000000..111a5d1c0b --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import java.util.concurrent.atomic.AtomicInteger; + +public class SharedOutputPipe implements OutputPipe { + private final OutputPipe delegate; + private final AtomicInteger remainingClosesRequired; + + public SharedOutputPipe(final OutputPipe delegate, final int closesRequired) { + this.delegate = delegate; + this.remainingClosesRequired = new AtomicInteger(closesRequired); + } + + @Override + public boolean isOpen() { + return delegate.isOpen(); + } + + @Override + public void put(final T value) { + delegate.put(value); + } + + @Override + public void close() { + if (remainingClosesRequired.decrementAndGet() == 0) { + delegate.close(); + } + } + + @Override + public boolean hasRemainingCapacity() { + return delegate.hasRemainingCapacity(); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index a48a368525..16c0024af2 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -13,6 +13,7 @@ package tech.pegasys.pantheon.services.pipeline; import static com.google.common.primitives.Ints.asList; +import static java.util.Collections.synchronizedList; import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; @@ -26,6 +27,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -177,6 +179,40 @@ public void shouldFlatMapItems() throws Exception { 26, 14, 28, 15, 30); } + @Test + public void shouldProcessInParallel() throws Exception { + final List output = synchronizedList(new ArrayList<>()); + final CountDownLatch latch = new CountDownLatch(1); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .thenProcessInParallel( + value -> { + if (value == 2) { + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return value.toString(); + }, + 2) + .andFinishWith(output::add); + final CompletableFuture result = pipeline.start(executorService); + + // One thread will block but the other should process the remaining entries. + waitForSize(output, 14); + assertThat(result).isNotDone(); + + latch.countDown(); + + result.get(10, SECONDS); + + assertThat(output) + .containsExactly( + "1", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12", "13", "14", "15", "2"); + } + private void waitForSize(final Collection collection, final int targetSize) { Awaitility.waitAtMost(10, SECONDS) .untilAsserted(() -> assertThat(collection).hasSize(targetSize)); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java new file mode 100644 index 0000000000..cf6bdd3a8c --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; + +import org.junit.Test; + +public class SharedOutputPipeTest { + + private static final int CLOSES_REQUIRED = 3; + + @SuppressWarnings("unchecked") + private final OutputPipe delegate = mock(OutputPipe.class); + + private final SharedOutputPipe pipe = new SharedOutputPipe<>(delegate, CLOSES_REQUIRED); + + @Test + public void shouldOnlyCloseDelegatePipeWhenCloseCalledSpecifiedNumberOfTimes() { + for (int i = 0; i < CLOSES_REQUIRED - 1; i++) { + pipe.close(); + verifyZeroInteractions(delegate); + } + + pipe.close(); + verify(delegate).close(); + } +} From ea8e0c4fdfbe03b3dbddc25e205012f5f913e7ac Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 14:52:08 +1000 Subject: [PATCH 11/39] Support aborting pipelines. --- .../pipeline/ExternalPipelineSource.java | 32 +++++++++ .../services/pipeline/OutputPipe.java | 4 +- .../pantheon/services/pipeline/Pipe.java | 23 +++++-- .../pantheon/services/pipeline/Pipeline.java | 50 ++++++++++++-- .../services/pipeline/PipelineBuilder.java | 40 ++++++++--- .../services/pipeline/PipelineSource.java | 19 +++++ .../services/pipeline/SharedOutputPipe.java | 5 ++ ...ourceStage.java => StreamSourceStage.java} | 24 +++++-- .../pantheon/services/pipeline/PipeTest.java | 69 +++++++++++++++++++ .../pipeline/PipelineBuilderTest.java | 44 ++++++++++-- ...geTest.java => StreamSourceStageTest.java} | 6 +- 11 files changed, 282 insertions(+), 34 deletions(-) create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java create mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{SourceStage.java => StreamSourceStage.java} (67%) create mode 100644 services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{SourceStageTest.java => StreamSourceStageTest.java} (88%) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java new file mode 100644 index 0000000000..3487ca0b64 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java @@ -0,0 +1,32 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +class ExternalPipelineSource implements PipelineSource { + + private final Pipe pipe; + + public ExternalPipelineSource(final Pipe pipe) { + this.pipe = pipe; + } + + @Override + public InputPipe getPipe() { + return pipe; + } + + @Override + public void abort() { + pipe.abort(); + } +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java index c0e37d0d7a..44267011e3 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -17,7 +17,9 @@ public interface OutputPipe { void put(T value); + boolean hasRemainingCapacity(); + void close(); - boolean hasRemainingCapacity(); + void abort(); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index cbd12b973e..44c3497a0d 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -26,15 +26,27 @@ class Pipe implements InputPipe, OutputPipe { private static final Logger LOG = LogManager.getLogger(); private final BlockingQueue queue; + private final int capacity; private final AtomicBoolean closed = new AtomicBoolean(); + private final AtomicBoolean aborted = new AtomicBoolean(); public Pipe(final int capacity) { queue = new ArrayBlockingQueue<>(capacity); + this.capacity = capacity; } @Override public boolean isOpen() { - return !closed.get(); + return !closed.get() && !aborted.get(); + } + + public int getCapacity() { + return capacity; + } + + @Override + public boolean hasRemainingCapacity() { + return queue.remainingCapacity() > 0; } @Override @@ -43,19 +55,22 @@ public void close() { } @Override - public boolean hasRemainingCapacity() { - return queue.remainingCapacity() > 0; + public void abort() { + aborted.set(true); } @Override public boolean hasMore() { + if (aborted.get()) { + return false; + } return !closed.get() || !queue.isEmpty(); } @Override public T get() { try { - while (isOpen() || !queue.isEmpty()) { + while (hasMore()) { final T value = queue.poll(1, TimeUnit.SECONDS); if (value != null) { return value; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 0384ddec4a..dbdc0ab2e8 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -12,23 +12,65 @@ */ package tech.pegasys.pantheon.services.pipeline; +import static java.util.stream.Collectors.toList; + import java.util.Collection; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Stream; -public class Pipeline { +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +public class Pipeline { + private static final Logger LOG = LogManager.getLogger(); + private final PipelineSource pipelineSource; private final Collection stages; + private final Collection> pipes; private final CompleterStage completerStage; + private final AtomicBoolean started = new AtomicBoolean(false); + private volatile List> futures; - public Pipeline(final Collection stages, final CompleterStage completerStage) { + public Pipeline( + final PipelineSource pipelineSource, + final Collection stages, + final Collection> pipes, + final CompleterStage completerStage) { + this.pipelineSource = pipelineSource; this.stages = stages; + this.pipes = pipes; this.completerStage = completerStage; } public CompletableFuture start(final ExecutorService executorService) { - stages.forEach(executorService::execute); - executorService.execute(completerStage); + if (!started.compareAndSet(false, true)) { + throw new IllegalStateException("Already started"); + } + futures = + Stream.concat(stages.stream(), Stream.of(completerStage)) + .map(task -> runWithErrorLogging(executorService, task)) + .collect(toList()); return completerStage.getFuture(); } + + private Future runWithErrorLogging( + final ExecutorService executorService, final Runnable task) { + return executorService.submit( + () -> { + try { + task.run(); + } catch (final Throwable t) { + LOG.error("Unhandled exception in pipeline", t); + } + }); + } + + public void abort() { + pipelineSource.abort(); + pipes.forEach(Pipe::abort); + futures.forEach(future -> future.cancel(true)); + } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 6e14d5d303..f4a9f76260 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -12,6 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; +import static com.google.common.base.Preconditions.checkArgument; import static java.util.Collections.emptyList; import static java.util.Collections.singleton; @@ -26,13 +27,22 @@ public class PipelineBuilder { + private final PipelineSource pipelineSource; private final Collection stages; + private final Collection> pipes; private final InputPipe pipeEnd; private final int bufferSize; public PipelineBuilder( - final Collection stages, final InputPipe pipeEnd, final int bufferSize) { + final PipelineSource pipelineSource, + final Collection stages, + final Collection> pipes, + final InputPipe pipeEnd, + final int bufferSize) { + checkArgument(!pipes.isEmpty(), "Must have at least one pipe in a pipeline"); + this.pipelineSource = pipelineSource; this.stages = stages; + this.pipes = pipes; this.pipeEnd = pipeEnd; this.bufferSize = bufferSize; } @@ -40,13 +50,14 @@ public PipelineBuilder( public static PipelineBuilder createPipelineFrom( final Iterator taskStream, final int bufferSize) { final Pipe pipe = new Pipe<>(bufferSize); - final SourceStage sourceStage = new SourceStage<>(taskStream, pipe); - return new PipelineBuilder<>(singleton(sourceStage), pipe, bufferSize); + final StreamSourceStage sourceStage = new StreamSourceStage<>(taskStream, pipe); + return new PipelineBuilder<>( + sourceStage, singleton(sourceStage), singleton(pipe), pipe, bufferSize); } - public static PipelineBuilder createPipelineFrom( - final InputPipe pipe, final int bufferSize) { - return new PipelineBuilder<>(emptyList(), pipe, bufferSize); + public static PipelineBuilder createPipelineFrom(final Pipe pipe) { + return new PipelineBuilder<>( + new ExternalPipelineSource<>(pipe), emptyList(), singleton(pipe), pipe, pipe.getCapacity()); } public PipelineBuilder thenProcess(final Function processor) { @@ -64,7 +75,8 @@ public PipelineBuilder thenProcessInParallel( new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); newStages.add(processStage); } - return new PipelineBuilder<>(newStages, newPipeEnd, bufferSize); + return new PipelineBuilder<>( + pipelineSource, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); } public PipelineBuilder thenProcessAsync( @@ -82,7 +94,7 @@ public PipelineBuilder thenFlatMap( } public Pipeline andFinishWith(final Consumer completer) { - return new Pipeline(stages, new CompleterStage<>(pipeEnd, completer)); + return new Pipeline(pipelineSource, stages, pipes, new CompleterStage<>(pipeEnd, completer)); } private PipelineBuilder addStage(final SingleStepStage singleStepStage) { @@ -97,8 +109,14 @@ private PipelineBuilder addStage( } private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { - final List newStages = new ArrayList<>(stages); - newStages.add(stage); - return new PipelineBuilder<>(newStages, outputPipe, bufferSize); + final List newStages = concat(stages, stage); + return new PipelineBuilder<>( + pipelineSource, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); + } + + private List concat(final Collection existing, final X newItem) { + final List newList = new ArrayList<>(existing); + newList.add(newItem); + return newList; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java new file mode 100644 index 0000000000..bc8d798a49 --- /dev/null +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +public interface PipelineSource { + InputPipe getPipe(); + + void abort(); +} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java index 111a5d1c0b..5fea2ee323 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java @@ -40,6 +40,11 @@ public void close() { } } + @Override + public void abort() { + delegate.abort(); + } + @Override public boolean hasRemainingCapacity() { return delegate.hasRemainingCapacity(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java similarity index 67% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java index dd39472700..93385e0ecc 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SourceStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java @@ -14,23 +14,33 @@ import java.util.Iterator; -class SourceStage implements Runnable { +class StreamSourceStage implements PipelineSource, Runnable { private final Iterator source; - private final OutputPipe output; + private final Pipe pipe; - SourceStage(final Iterator source, final OutputPipe output) { + StreamSourceStage(final Iterator source, final Pipe pipe) { this.source = source; - this.output = output; + this.pipe = pipe; } @Override public void run() { - while (output.isOpen() && source.hasNext()) { + while (pipe.isOpen() && source.hasNext()) { final T value = source.next(); if (value != null) { - output.put(value); + pipe.put(value); } } - output.close(); + pipe.close(); + } + + @Override + public InputPipe getPipe() { + return pipe; + } + + @Override + public void abort() { + pipe.abort(); } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java new file mode 100644 index 0000000000..82ae5ec554 --- /dev/null +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package tech.pegasys.pantheon.services.pipeline; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Test; + +public class PipeTest { + private final Pipe pipe = new Pipe<>(5); + + @Test + public void shouldNotHaveMoreWhenEmptyAndClosed() { + pipe.close(); + assertThat(pipe.hasMore()).isFalse(); + } + + @Test + public void shouldHaveMoreWhenNotEmptyAndClosed() { + pipe.put("A"); + pipe.close(); + + assertThat(pipe.hasMore()).isTrue(); + + pipe.get(); + + assertThat(pipe.hasMore()).isFalse(); + } + + @Test + public void shouldNotHaveMoreWhenAbortedEvenIfNotEmpty() { + pipe.put("A"); + pipe.abort(); + + assertThat(pipe.hasMore()).isFalse(); + } + + @Test + public void shouldLimitBatchMaximumSize() { + pipe.put("a"); + pipe.put("b"); + pipe.put("c"); + pipe.put("d"); + + assertThat(pipe.getBatch(3)).containsExactly("a", "b", "c"); + } + + @Test + public void shouldNotWaitToReachMaximumSizeBeforeReturningBatch() { + pipe.put("a"); + assertThat(pipe.getBatch(3)).containsExactly("a"); + } + + @Test + public void shouldNotBeOpenAfterAbort() { + pipe.abort(); + assertThat(pipe.isOpen()).isFalse(); + } +} diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 16c0024af2..3243207fe0 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -18,6 +18,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; +import static org.awaitility.Awaitility.waitAtMost; import java.util.ArrayList; import java.util.Collection; @@ -31,10 +32,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Stream; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.awaitility.Awaitility; import org.junit.AfterClass; import org.junit.Test; @@ -90,7 +92,7 @@ public void shouldCombineIntoBatches() throws Exception { tasks.forEachRemaining(input::put); final BlockingQueue> output = new ArrayBlockingQueue<>(10); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(input, 20).inBatches(6).andFinishWith(output::offer); + PipelineBuilder.createPipelineFrom(input).inBatches(6).andFinishWith(output::offer); final CompletableFuture result = pipeline.start(executorService); @@ -213,8 +215,42 @@ public void shouldProcessInParallel() throws Exception { "1", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12", "13", "14", "15", "2"); } + @Test + public void shouldAbortPipeline() throws Exception { + final int allowProcessingUpTo = 5; + final AtomicBoolean processorInterrupted = new AtomicBoolean(false); + final List output = synchronizedList(new ArrayList<>()); + final CountDownLatch startedProcessingValueSix = new CountDownLatch(1); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .thenProcess( + value -> { + if (value > allowProcessingUpTo) { + try { + startedProcessingValueSix.countDown(); + Thread.sleep(TimeUnit.MINUTES.toNanos(2)); + } catch (final InterruptedException e) { + processorInterrupted.set(true); + } + } + return value; + }) + .andFinishWith(output::add); + + final CompletableFuture result = pipeline.start(executorService); + + startedProcessingValueSix.await(10, SECONDS); + waitForSize(output, allowProcessingUpTo); + + pipeline.abort(); + + result.get(10, SECONDS); + assertThat(output).containsExactly(1, 2, 3, 4, 5); + + waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(processorInterrupted).isTrue()); + } + private void waitForSize(final Collection collection, final int targetSize) { - Awaitility.waitAtMost(10, SECONDS) - .untilAsserted(() -> assertThat(collection).hasSize(targetSize)); + waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(collection).hasSize(targetSize)); } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java similarity index 88% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java index ce3cc7fd8e..aef79f7b00 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SourceStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java @@ -17,12 +17,12 @@ import com.google.common.collect.Iterators; import org.junit.Test; -public class SourceStageTest { +public class StreamSourceStageTest { private final Pipe output = new Pipe<>(10); - private final SourceStage stage = - new SourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); + private final StreamSourceStage stage = + new StreamSourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); @Test public void shouldOutputEntriesThenClosePipe() { From 55ccbcc7d820fd423442a39a69650f675055e042 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 15:15:38 +1000 Subject: [PATCH 12/39] Abort the pipeline if any stage throws an unhandled exception. --- .../pantheon/services/pipeline/Pipeline.java | 35 +++++++++++++++---- .../pipeline/PipelineBuilderTest.java | 27 +++++++++++++- 2 files changed, 55 insertions(+), 7 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index dbdc0ab2e8..e2ba744c44 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -16,6 +16,7 @@ import java.util.Collection; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -32,6 +33,7 @@ public class Pipeline { private final Collection> pipes; private final CompleterStage completerStage; private final AtomicBoolean started = new AtomicBoolean(false); + private final CompletableFuture overallFuture = new CompletableFuture<>(); private volatile List> futures; public Pipeline( @@ -47,13 +49,25 @@ public Pipeline( public CompletableFuture start(final ExecutorService executorService) { if (!started.compareAndSet(false, true)) { - throw new IllegalStateException("Already started"); + return overallFuture; } futures = Stream.concat(stages.stream(), Stream.of(completerStage)) .map(task -> runWithErrorLogging(executorService, task)) .collect(toList()); - return completerStage.getFuture(); + completerStage + .getFuture() + .whenComplete( + (result, error) -> { + synchronized (this) { + if (error != null) { + overallFuture.completeExceptionally(error); + } else { + overallFuture.complete(null); + } + } + }); + return overallFuture; } private Future runWithErrorLogging( @@ -63,14 +77,23 @@ private Future runWithErrorLogging( try { task.run(); } catch (final Throwable t) { - LOG.error("Unhandled exception in pipeline", t); + LOG.debug("Unhandled exception in pipeline. Aborting.", t); + abort(t); } }); } public void abort() { - pipelineSource.abort(); - pipes.forEach(Pipe::abort); - futures.forEach(future -> future.cancel(true)); + final CancellationException exception = new CancellationException("Pipeline aborted"); + abort(exception); + } + + private void abort(final Throwable error) { + synchronized (this) { + pipelineSource.abort(); + pipes.forEach(Pipe::abort); + futures.forEach(future -> future.cancel(true)); + overallFuture.completeExceptionally(error); + } } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 3243207fe0..b73bb6eb29 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -17,6 +17,7 @@ import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; import static org.awaitility.Awaitility.waitAtMost; @@ -26,6 +27,7 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -244,12 +246,35 @@ public void shouldAbortPipeline() throws Exception { pipeline.abort(); - result.get(10, SECONDS); + assertThatThrownBy(() -> result.get(10, SECONDS)).isInstanceOf(CancellationException.class); assertThat(output).containsExactly(1, 2, 3, 4, 5); waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(processorInterrupted).isTrue()); } + @Test + public void shouldAbortPipelineWhenProcessorThrowsException() { + final RuntimeException expectedError = new RuntimeException("Oops"); + final List output = new ArrayList<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .thenProcess( + value -> { + if (value == 4) { + throw expectedError; + } + return value; + }) + .andFinishWith(output::add); + + final CompletableFuture result = pipeline.start(executorService); + + assertThatThrownBy(() -> result.get(10, SECONDS)) + .extracting(Throwable::getCause) + .isSameAs(expectedError); + assertThat(output).containsExactly(1, 2, 3); + } + private void waitForSize(final Collection collection, final int targetSize) { waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(collection).hasSize(targetSize)); } From 68112153e7071b491eda23192591dfbd624cb566 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 15:20:31 +1000 Subject: [PATCH 13/39] Tidy up access levels. --- .../services/pipeline/AsyncProcessStage.java | 2 +- .../pantheon/services/pipeline/BatchingStage.java | 2 +- .../pantheon/services/pipeline/FlatMapStage.java | 2 +- .../services/pipeline/InputOutputStage.java | 2 +- .../pantheon/services/pipeline/InputPipe.java | 2 +- .../pantheon/services/pipeline/OutputPipe.java | 2 +- .../pegasys/pantheon/services/pipeline/Pipe.java | 2 +- .../pantheon/services/pipeline/Pipeline.java | 14 +++++++------- .../services/pipeline/SharedOutputPipe.java | 2 +- .../services/pipeline/SingleStepStage.java | 2 +- 10 files changed, 16 insertions(+), 16 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java index 1c5f8b7b6c..4ca4fd0361 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java @@ -24,7 +24,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -public class AsyncProcessStage implements SingleStepStage { +class AsyncProcessStage implements SingleStepStage { private static final Logger LOG = LogManager.getLogger(); private final Function> processor; private final Collection> inProgress; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java index 021bf21199..b673739740 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java @@ -14,7 +14,7 @@ import java.util.List; -public class BatchingStage implements SingleStepStage> { +class BatchingStage implements SingleStepStage> { private final int maximumBatchSize; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java index 74b084fc70..a2fb2c55a7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java @@ -15,7 +15,7 @@ import java.util.function.Function; import java.util.stream.Stream; -public class FlatMapStage implements SingleStepStage { +class FlatMapStage implements SingleStepStage { private final Function> mapper; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java index 6bcadc39ed..2800e22f14 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java @@ -12,7 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; -public class InputOutputStage implements Runnable { +class InputOutputStage implements Runnable { private final InputPipe inputPipe; private final OutputPipe outputPipe; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java index db61a950a2..dea26399b4 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -14,7 +14,7 @@ import java.util.List; -public interface InputPipe { +interface InputPipe { boolean hasMore(); T get(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java index 44267011e3..d4310a3ed9 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -12,7 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; -public interface OutputPipe { +interface OutputPipe { boolean isOpen(); void put(T value); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index 44c3497a0d..b8507946c8 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -23,7 +23,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -class Pipe implements InputPipe, OutputPipe { +public class Pipe implements InputPipe, OutputPipe { private static final Logger LOG = LogManager.getLogger(); private final BlockingQueue queue; private final int capacity; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index e2ba744c44..185c1e5e88 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -53,7 +53,7 @@ public CompletableFuture start(final ExecutorService executorService) { } futures = Stream.concat(stages.stream(), Stream.of(completerStage)) - .map(task -> runWithErrorLogging(executorService, task)) + .map(task -> runWithErrorHandling(executorService, task)) .collect(toList()); completerStage .getFuture() @@ -70,7 +70,12 @@ public CompletableFuture start(final ExecutorService executorService) { return overallFuture; } - private Future runWithErrorLogging( + public void abort() { + final CancellationException exception = new CancellationException("Pipeline aborted"); + abort(exception); + } + + private Future runWithErrorHandling( final ExecutorService executorService, final Runnable task) { return executorService.submit( () -> { @@ -83,11 +88,6 @@ private Future runWithErrorLogging( }); } - public void abort() { - final CancellationException exception = new CancellationException("Pipeline aborted"); - abort(exception); - } - private void abort(final Throwable error) { synchronized (this) { pipelineSource.abort(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java index 5fea2ee323..5ee880b78e 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java @@ -14,7 +14,7 @@ import java.util.concurrent.atomic.AtomicInteger; -public class SharedOutputPipe implements OutputPipe { +class SharedOutputPipe implements OutputPipe { private final OutputPipe delegate; private final AtomicInteger remainingClosesRequired; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java index 7164f81885..9115892ff7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java @@ -12,7 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; -public interface SingleStepStage { +interface SingleStepStage { void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); default void finalize(final OutputPipe outputPipe) {} From 02e4aa1c5d1d4fb12e4f66626def98f5a3dfb7a5 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 16:41:37 +1000 Subject: [PATCH 14/39] Add some javadoc. --- .../pantheon/services/pipeline/InputPipe.java | 5 + .../services/pipeline/OutputPipe.java | 5 + .../pantheon/services/pipeline/Pipeline.java | 16 ++++ .../services/pipeline/PipelineBuilder.java | 96 ++++++++++++++++++- 4 files changed, 120 insertions(+), 2 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java index dea26399b4..9e11189afd 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -14,6 +14,11 @@ import java.util.List; +/** + * The end of the pipe that stages read input from. + * + * @param the type of input. + */ interface InputPipe { boolean hasMore(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java index d4310a3ed9..aea535b788 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -12,6 +12,11 @@ */ package tech.pegasys.pantheon.services.pipeline; +/** + * The end of the pipe that stages write their output to. + * + * @param the type of output. + */ interface OutputPipe { boolean isOpen(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 185c1e5e88..7ccf4115b8 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -47,6 +47,14 @@ public Pipeline( this.completerStage = completerStage; } + /** + * Starts execution of the pipeline. Each stage in the pipeline requires a dedicated thread from + * the supplied executor service. + * + * @param executorService the {@link ExecutorService} to execute each stage in. + * @return a future that will be completed when the pipeline completes. If the pipeline fails or + * is aborted the returned future will be completed exceptionally. + */ public CompletableFuture start(final ExecutorService executorService) { if (!started.compareAndSet(false, true)) { return overallFuture; @@ -70,6 +78,14 @@ public CompletableFuture start(final ExecutorService executorService) { return overallFuture; } + /** + * Abort execution of this pipeline. The future returned by {@link #start(ExecutorService)} will + * be completed with a {@link CancellationException}. + * + *

A best effort is made to halt all processing by the pipeline immediately by interrupting + * each execution thread and pipes connecting each stage will no longer accept or provide further + * items. + */ public void abort() { final CancellationException exception = new CancellationException("Pipeline aborted"); abort(exception); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index f4a9f76260..bf9486bf6b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -25,6 +25,16 @@ import java.util.function.Function; import java.util.stream.Stream; +/** + * Supports building a new pipeline. Pipelines are comprised of a source, various processing stages + * and a consumer, each of which run in their own thread. + * + *

The pipeline completes when all items from the source have passed through each stage and are + * received by the consumer. The pipeline will halt immediately if an exception is thrown from any + * processing stage. + * + * @param the output type of the last stage in the pipeline. + */ public class PipelineBuilder { private final PipelineSource pipelineSource; @@ -47,24 +57,60 @@ public PipelineBuilder( this.bufferSize = bufferSize; } + /** + * Create a new pipeline that processes inputs from source. The pipeline completes when + * source returns false from {@link Iterator#hasNext()} and the last item has + * been reached the end of the pipeline. + * + * @param source the source to pull items from for processing. + * @param bufferSize the number of items to be buffered between each stage in the pipeline. + * @param the type of items input into the pipeline. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public static PipelineBuilder createPipelineFrom( - final Iterator taskStream, final int bufferSize) { + final Iterator source, final int bufferSize) { final Pipe pipe = new Pipe<>(bufferSize); - final StreamSourceStage sourceStage = new StreamSourceStage<>(taskStream, pipe); + final StreamSourceStage sourceStage = new StreamSourceStage<>(source, pipe); return new PipelineBuilder<>( sourceStage, singleton(sourceStage), singleton(pipe), pipe, bufferSize); } + /** + * Create a new pipeline that processes inputs added to pipe. The pipeline completes when + * pipe is closed and the last item has been reached the end of the pipeline. + * + * @param pipe the pipe feeding the pipeline. + * @param the type of items input into the pipeline. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public static PipelineBuilder createPipelineFrom(final Pipe pipe) { return new PipelineBuilder<>( new ExternalPipelineSource<>(pipe), emptyList(), singleton(pipe), pipe, pipe.getCapacity()); } + /** + * Adds a 1-to-1 processing stage to the pipeline. A single thread processes each item in the + * pipeline with processor outputting its return value to the next stage. + * + * @param processor the processing to apply to each item. + * @param the output type for this processing step. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public PipelineBuilder thenProcess(final Function processor) { final SingleStepStage singleStepStage = new ProcessStage<>(processor); return addStage(singleStepStage); } + /** + * Adds a 1-to-1 processing stage to the pipeline. Multiple threads processes items in the + * pipeline concurrently with processor outputting its return value to the next stage. + * + *

Note: The order of items is not preserved. + * + * @param processor the processing to apply to each item. + * @param the output type for this processing step. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public PipelineBuilder thenProcessInParallel( final Function processor, final int numberOfThreads) { final Pipe newPipeEnd = new Pipe<>(bufferSize); @@ -79,20 +125,66 @@ public PipelineBuilder thenProcessInParallel( pipelineSource, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); } + /** + * Adds a 1-to-1, asynchronous processing stage to the pipeline. A single thread reads items from + * the input and calls processor to begin processing. While a single thread is used to + * begin processing, up to maxConcurrency items may be in progress concurrently. When the + * returned {@link CompletableFuture} completes successfully the result is passed to the next + * stage. + * + *

If the returned {@link CompletableFuture} completes exceptionally the pipeline will abort. + * + *

Note: The order of items is not preserved. + * + * @param processor the processing to apply to each item. + * @param the output type for this processing step. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public PipelineBuilder thenProcessAsync( final Function> processor, final int maxConcurrency) { return addStage(new AsyncProcessStage<>(processor, maxConcurrency)); } + /** + * Batches items into groups of at most maximumBatchSize. Batches are created eagerly to + * minimize delay so may not be full. + * + *

Order of items is preserved. + * + *

The output buffer size is reduced to bufferSize / maximumBatchSize + 1. + * + * @param maximumBatchSize the maximum number of items to include in a batch. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public PipelineBuilder> inBatches(final int maximumBatchSize) { return addStage(new BatchingStage<>(maximumBatchSize), bufferSize / maximumBatchSize + 1); } + /** + * Adds a 1-to-many processing stage to the pipeline. For each item in the stream, mapper + * is called and each item of the {@link Stream} it returns is output as an individual item. The + * returned Stream may be empty to remove an item. + * + *

This can be used to reverse the effect of {@link #inBatches(int)} with: + * + *

thenFlatMap(List::stream, newBufferSize)
+ * + * @param mapper the function to process each item with. + * @param newBufferSize the output buffer size to use from this stage onwards. + * @param the type of items to be output from this stage. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ public PipelineBuilder thenFlatMap( final Function> mapper, final int newBufferSize) { return addStage(new FlatMapStage<>(mapper), newBufferSize); } + /** + * End the pipeline with a {@link Consumer} that is the last stage of the pipeline. + * + * @param completer the {@link Consumer} that accepts the final output of the pipeline. + * @return the constructed pipeline ready to execute. + */ public Pipeline andFinishWith(final Consumer completer) { return new Pipeline(pipelineSource, stages, pipes, new CompleterStage<>(pipeEnd, completer)); } From ebf0feb1f7f1d634db64b9040cfc79126bebce08 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 17:57:08 +1000 Subject: [PATCH 15/39] Exceptions cause the pipeline to abort quite violently so we can't assume that earlier entries have completed processing by then. --- .../pantheon/services/pipeline/PipelineBuilderTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index b73bb6eb29..5c1d0f2a4f 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -255,7 +255,6 @@ public void shouldAbortPipeline() throws Exception { @Test public void shouldAbortPipelineWhenProcessorThrowsException() { final RuntimeException expectedError = new RuntimeException("Oops"); - final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcess( @@ -265,14 +264,13 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { } return value; }) - .andFinishWith(output::add); + .andFinishWith(new ArrayList()::add); final CompletableFuture result = pipeline.start(executorService); assertThatThrownBy(() -> result.get(10, SECONDS)) .extracting(Throwable::getCause) .isSameAs(expectedError); - assertThat(output).containsExactly(1, 2, 3); } private void waitForSize(final Collection collection, final int targetSize) { From be717e08c3cf69e89b95fe4cb54c329ee5efd800 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 20:20:08 +1000 Subject: [PATCH 16/39] Add missing javadoc parameters. --- .../pegasys/pantheon/services/pipeline/PipelineBuilder.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index bf9486bf6b..2e6fd7fb20 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -108,6 +108,7 @@ public PipelineBuilder thenProcess(final Function processor) { *

Note: The order of items is not preserved. * * @param processor the processing to apply to each item. + * @param numberOfThreads the number of threads to use for processing. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ @@ -137,6 +138,7 @@ public PipelineBuilder thenProcessInParallel( *

Note: The order of items is not preserved. * * @param processor the processing to apply to each item. + * @param maxConcurrency the maximum number of items being processed concurrently. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ From 4b8c544ed515bb230dd43943f07dc43218d2fba4 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 20:37:38 +1000 Subject: [PATCH 17/39] Don't use a static executor service. --- .../pantheon/services/pipeline/PipelineBuilderTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 5c1d0f2a4f..284145d1ef 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -39,7 +39,7 @@ import java.util.stream.Stream; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.AfterClass; +import org.junit.After; import org.junit.Test; public class PipelineBuilderTest { @@ -52,11 +52,11 @@ public class PipelineBuilderTest { private final Iterator tasks = asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15).iterator(); - private static final ExecutorService executorService = + private final ExecutorService executorService = Executors.newCachedThreadPool(THREAD_FACTORY); - @AfterClass - public static void afterClass() throws Exception { + @After + public void afterClass() throws Exception { executorService.shutdownNow(); if (!executorService.awaitTermination(10, SECONDS)) { fail("Executor service did not shut down cleanly"); From 8181abb12576b280b6e141b9e8069bb7980804eb Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 20:42:10 +1000 Subject: [PATCH 18/39] Simplify test & fix spotless. --- .../services/pipeline/PipelineBuilderTest.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 284145d1ef..8e4ef5e348 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -36,6 +36,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; import java.util.stream.Stream; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -52,8 +53,7 @@ public class PipelineBuilderTest { private final Iterator tasks = asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15).iterator(); - private final ExecutorService executorService = - Executors.newCachedThreadPool(THREAD_FACTORY); + private final ExecutorService executorService = Executors.newCachedThreadPool(THREAD_FACTORY); @After public void afterClass() throws Exception { @@ -258,12 +258,10 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcess( - value -> { - if (value == 4) { - throw expectedError; - } - return value; - }) + (Function) + value -> { + throw expectedError; + }) .andFinishWith(new ArrayList()::add); final CompletableFuture result = pipeline.start(executorService); From 7b9961f5c5490b5842ccaedef358cff765903f2a Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 8 Mar 2019 20:59:55 +1000 Subject: [PATCH 19/39] Ignore PipelineBuilderTest to see if that's what's causing the build to stall. --- .../pegasys/pantheon/services/pipeline/PipelineBuilderTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 8e4ef5e348..7e7ccd5f00 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -41,8 +41,10 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.junit.After; +import org.junit.Ignore; import org.junit.Test; +@Ignore public class PipelineBuilderTest { private static final ThreadFactory THREAD_FACTORY = From 6a7818168217320536f199157ad69a6fdd8e6ae0 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 10:30:50 +1000 Subject: [PATCH 20/39] Try to find the one test case that's stalling. --- .../pantheon/services/pipeline/PipelineBuilderTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 7e7ccd5f00..cd220c5ab0 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -44,7 +44,6 @@ import org.junit.Ignore; import org.junit.Test; -@Ignore public class PipelineBuilderTest { private static final ThreadFactory THREAD_FACTORY = @@ -133,6 +132,7 @@ public void shouldProcessAsync() throws Exception { } @Test + @Ignore public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { final List output = new ArrayList<>(); final List> futures = new CopyOnWriteArrayList<>(); @@ -170,6 +170,7 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { } @Test + @Ignore public void shouldFlatMapItems() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = @@ -186,6 +187,7 @@ public void shouldFlatMapItems() throws Exception { } @Test + @Ignore public void shouldProcessInParallel() throws Exception { final List output = synchronizedList(new ArrayList<>()); final CountDownLatch latch = new CountDownLatch(1); @@ -220,6 +222,7 @@ public void shouldProcessInParallel() throws Exception { } @Test + @Ignore public void shouldAbortPipeline() throws Exception { final int allowProcessingUpTo = 5; final AtomicBoolean processorInterrupted = new AtomicBoolean(false); @@ -255,6 +258,7 @@ public void shouldAbortPipeline() throws Exception { } @Test + @Ignore public void shouldAbortPipelineWhenProcessorThrowsException() { final RuntimeException expectedError = new RuntimeException("Oops"); final Pipeline pipeline = From e3f0f842da604595c774d717432bd3e41ffde850 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 10:46:53 +1000 Subject: [PATCH 21/39] Unignore some more tests. --- .../pantheon/services/pipeline/PipelineBuilderTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index cd220c5ab0..d5a59c61c8 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -132,7 +132,6 @@ public void shouldProcessAsync() throws Exception { } @Test - @Ignore public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { final List output = new ArrayList<>(); final List> futures = new CopyOnWriteArrayList<>(); @@ -170,7 +169,6 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { } @Test - @Ignore public void shouldFlatMapItems() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = @@ -187,7 +185,6 @@ public void shouldFlatMapItems() throws Exception { } @Test - @Ignore public void shouldProcessInParallel() throws Exception { final List output = synchronizedList(new ArrayList<>()); final CountDownLatch latch = new CountDownLatch(1); From 1e3a6b0e58c37e47a271be1ce27513e562e2ac59 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 12:08:28 +1000 Subject: [PATCH 22/39] Unignore one more test. --- .../pegasys/pantheon/services/pipeline/PipelineBuilderTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index d5a59c61c8..fe1fa8e314 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -219,7 +219,6 @@ public void shouldProcessInParallel() throws Exception { } @Test - @Ignore public void shouldAbortPipeline() throws Exception { final int allowProcessingUpTo = 5; final AtomicBoolean processorInterrupted = new AtomicBoolean(false); From 0a4f8cf2ccf6496f4d5c5a86321d9fe49bd8fa2b Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 12:28:21 +1000 Subject: [PATCH 23/39] Unignore final test. --- .../pegasys/pantheon/services/pipeline/PipelineBuilderTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index fe1fa8e314..8e4ef5e348 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -41,7 +41,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.junit.After; -import org.junit.Ignore; import org.junit.Test; public class PipelineBuilderTest { @@ -254,7 +253,6 @@ public void shouldAbortPipeline() throws Exception { } @Test - @Ignore public void shouldAbortPipelineWhenProcessorThrowsException() { final RuntimeException expectedError = new RuntimeException("Oops"); final Pipeline pipeline = From e92fce27b7f90fec44462bbf096d93196f2d2f90 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 12:58:56 +1000 Subject: [PATCH 24/39] Synchronize on overallFuture. --- .../tech/pegasys/pantheon/services/pipeline/Pipeline.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 7ccf4115b8..4fc834b348 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -67,7 +67,7 @@ public CompletableFuture start(final ExecutorService executorService) { .getFuture() .whenComplete( (result, error) -> { - synchronized (this) { + synchronized (overallFuture) { if (error != null) { overallFuture.completeExceptionally(error); } else { @@ -105,7 +105,7 @@ private Future runWithErrorHandling( } private void abort(final Throwable error) { - synchronized (this) { + synchronized (overallFuture) { pipelineSource.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); From 3515376d136e0cdeb44aa2f9ca0dc890520f77e7 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 13:21:46 +1000 Subject: [PATCH 25/39] Add more logging. --- .../java/tech/pegasys/pantheon/services/pipeline/Pipeline.java | 2 ++ services/pipeline/src/test/resources/log4j2.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 4fc834b348..540aa0c9ec 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -68,6 +68,7 @@ public CompletableFuture start(final ExecutorService executorService) { .whenComplete( (result, error) -> { synchronized (overallFuture) { + LOG.debug("Completer stage future complete", error); if (error != null) { overallFuture.completeExceptionally(error); } else { @@ -106,6 +107,7 @@ private Future runWithErrorHandling( private void abort(final Throwable error) { synchronized (overallFuture) { + LOG.info("Aborting build", error); pipelineSource.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); diff --git a/services/pipeline/src/test/resources/log4j2.xml b/services/pipeline/src/test/resources/log4j2.xml index f6fd6801df..18e6565bb7 100644 --- a/services/pipeline/src/test/resources/log4j2.xml +++ b/services/pipeline/src/test/resources/log4j2.xml @@ -1,7 +1,7 @@ - INFO + DEBUG From c9e671e2b02d331e59e803d8c6d0452166aeacdf Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 14:39:35 +1000 Subject: [PATCH 26/39] Remove unused executor. --- .../main/java/tech/pegasys/pantheon/Runner.java | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java b/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java index cf2203bc1d..6eeaa82390 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java @@ -25,9 +25,6 @@ import java.nio.file.Path; import java.util.Optional; import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import io.vertx.core.Vertx; import org.apache.logging.log4j.LogManager; @@ -39,8 +36,6 @@ public class Runner implements AutoCloseable { private final Vertx vertx; - private final ExecutorService exec = Executors.newCachedThreadPool(); - private final NetworkRunner networkRunner; private final Optional jsonRpc; private final Optional websocketRpc; @@ -98,20 +93,15 @@ public void close() throws Exception { networkRunner.stop(); networkRunner.awaitStop(); - exec.shutdown(); try { jsonRpc.ifPresent(service -> service.stop().join()); websocketRpc.ifPresent(service -> service.stop().join()); metrics.ifPresent(service -> service.stop().join()); } finally { try { - exec.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS); + vertx.close(); } finally { - try { - vertx.close(); - } finally { - pantheonController.close(); - } + pantheonController.close(); } } } From 5f9899a2efd9eb6dcdadaac054b7957286270037 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 14:55:13 +1000 Subject: [PATCH 27/39] Don't wait forever for vertx services to shutdown. Log warnings if services take an unusually long time to start. --- .../java/tech/pegasys/pantheon/Runner.java | 48 ++++++++++++++++--- 1 file changed, 42 insertions(+), 6 deletions(-) diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java b/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java index 6eeaa82390..5121891174 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/Runner.java @@ -25,6 +25,10 @@ import java.nio.file.Path; import java.util.Optional; import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import io.vertx.core.Vertx; import org.apache.logging.log4j.LogManager; @@ -68,9 +72,9 @@ public void start() { if (networkRunner.getNetwork().isP2pEnabled()) { pantheonController.getSynchronizer().start(); } - jsonRpc.ifPresent(service -> service.start().join()); - websocketRpc.ifPresent(service -> service.start().join()); - metrics.ifPresent(service -> service.start().join()); + jsonRpc.ifPresent(service -> waitForServiceToStart("jsonRpc", service.start())); + websocketRpc.ifPresent(service -> waitForServiceToStop("websocketRpc", service.start())); + metrics.ifPresent(service -> waitForServiceToStart("metrics", service.start())); LOG.info("Ethereum main loop is up."); writePantheonPortsToFile(); } catch (final Exception ex) { @@ -94,9 +98,9 @@ public void close() throws Exception { networkRunner.awaitStop(); try { - jsonRpc.ifPresent(service -> service.stop().join()); - websocketRpc.ifPresent(service -> service.stop().join()); - metrics.ifPresent(service -> service.stop().join()); + jsonRpc.ifPresent(service -> waitForServiceToStop("jsonRpc", service.stop())); + websocketRpc.ifPresent(service -> waitForServiceToStop("websocketRpc", service.stop())); + metrics.ifPresent(service -> waitForServiceToStop("metrics", service.stop())); } finally { try { vertx.close(); @@ -106,6 +110,38 @@ public void close() throws Exception { } } + private void waitForServiceToStop( + final String serviceName, final CompletableFuture stopFuture) { + try { + stopFuture.get(30, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + LOG.debug("Interrupted while waiting for service to complete", e); + Thread.currentThread().interrupt(); + } catch (final ExecutionException e) { + LOG.error("Service " + serviceName + " failed to shutdown", e); + } catch (final TimeoutException e) { + LOG.error("Service {} did not shut down cleanly", serviceName); + } + } + + private void waitForServiceToStart( + final String serviceName, final CompletableFuture startFuture) { + while (!startFuture.isDone()) { + try { + startFuture.get(60, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + LOG.debug("Interrupted while waiting for service to start", e); + Thread.currentThread().interrupt(); + return; + } catch (final ExecutionException e) { + LOG.error("Service " + serviceName + " failed to start", e); + throw new IllegalStateException(e); + } catch (final TimeoutException e) { + LOG.warn("Service {} is taking an unusually long time to start", serviceName); + } + } + } + private void writePantheonPortsToFile() { final Properties properties = new Properties(); From 38b3b6606bba4bbd659320665e2406868d2a59df Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 11 Mar 2019 15:40:42 +1000 Subject: [PATCH 28/39] Revert "Add more logging." This reverts commit 3515376d136e0cdeb44aa2f9ca0dc890520f77e7. --- .../java/tech/pegasys/pantheon/services/pipeline/Pipeline.java | 2 -- services/pipeline/src/test/resources/log4j2.xml | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 540aa0c9ec..4fc834b348 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -68,7 +68,6 @@ public CompletableFuture start(final ExecutorService executorService) { .whenComplete( (result, error) -> { synchronized (overallFuture) { - LOG.debug("Completer stage future complete", error); if (error != null) { overallFuture.completeExceptionally(error); } else { @@ -107,7 +106,6 @@ private Future runWithErrorHandling( private void abort(final Throwable error) { synchronized (overallFuture) { - LOG.info("Aborting build", error); pipelineSource.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); diff --git a/services/pipeline/src/test/resources/log4j2.xml b/services/pipeline/src/test/resources/log4j2.xml index 18e6565bb7..f6fd6801df 100644 --- a/services/pipeline/src/test/resources/log4j2.xml +++ b/services/pipeline/src/test/resources/log4j2.xml @@ -1,7 +1,7 @@ - DEBUG + INFO From 3d8c4477a3741756f52018b3924ef8b5794974cb Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 05:56:21 +1000 Subject: [PATCH 29/39] Rename PipelineSource to ProducingStage. --- ...lineSource.java => ExternalProducingStage.java} | 4 ++-- .../pantheon/services/pipeline/Pipeline.java | 8 ++++---- .../services/pipeline/PipelineBuilder.java | 14 +++++++------- .../{PipelineSource.java => ProducingStage.java} | 2 +- .../services/pipeline/StreamSourceStage.java | 2 +- 5 files changed, 15 insertions(+), 15 deletions(-) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{ExternalPipelineSource.java => ExternalProducingStage.java} (87%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{PipelineSource.java => ProducingStage.java} (94%) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java similarity index 87% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java index 3487ca0b64..2bac61f0d2 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalPipelineSource.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java @@ -12,11 +12,11 @@ */ package tech.pegasys.pantheon.services.pipeline; -class ExternalPipelineSource implements PipelineSource { +class ExternalProducingStage implements ProducingStage { private final Pipe pipe; - public ExternalPipelineSource(final Pipe pipe) { + public ExternalProducingStage(final Pipe pipe) { this.pipe = pipe; } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 4fc834b348..489ed27634 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -28,7 +28,7 @@ public class Pipeline { private static final Logger LOG = LogManager.getLogger(); - private final PipelineSource pipelineSource; + private final ProducingStage producingStage; private final Collection stages; private final Collection> pipes; private final CompleterStage completerStage; @@ -37,11 +37,11 @@ public class Pipeline { private volatile List> futures; public Pipeline( - final PipelineSource pipelineSource, + final ProducingStage producingStage, final Collection stages, final Collection> pipes, final CompleterStage completerStage) { - this.pipelineSource = pipelineSource; + this.producingStage = producingStage; this.stages = stages; this.pipes = pipes; this.completerStage = completerStage; @@ -106,7 +106,7 @@ private Future runWithErrorHandling( private void abort(final Throwable error) { synchronized (overallFuture) { - pipelineSource.abort(); + producingStage.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); overallFuture.completeExceptionally(error); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 2e6fd7fb20..fde436558a 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -37,20 +37,20 @@ */ public class PipelineBuilder { - private final PipelineSource pipelineSource; + private final ProducingStage producingStage; private final Collection stages; private final Collection> pipes; private final InputPipe pipeEnd; private final int bufferSize; public PipelineBuilder( - final PipelineSource pipelineSource, + final ProducingStage producingStage, final Collection stages, final Collection> pipes, final InputPipe pipeEnd, final int bufferSize) { checkArgument(!pipes.isEmpty(), "Must have at least one pipe in a pipeline"); - this.pipelineSource = pipelineSource; + this.producingStage = producingStage; this.stages = stages; this.pipes = pipes; this.pipeEnd = pipeEnd; @@ -85,7 +85,7 @@ public static PipelineBuilder createPipelineFrom( */ public static PipelineBuilder createPipelineFrom(final Pipe pipe) { return new PipelineBuilder<>( - new ExternalPipelineSource<>(pipe), emptyList(), singleton(pipe), pipe, pipe.getCapacity()); + new ExternalProducingStage<>(pipe), emptyList(), singleton(pipe), pipe, pipe.getCapacity()); } /** @@ -123,7 +123,7 @@ public PipelineBuilder thenProcessInParallel( newStages.add(processStage); } return new PipelineBuilder<>( - pipelineSource, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); + producingStage, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); } /** @@ -188,7 +188,7 @@ public PipelineBuilder thenFlatMap( * @return the constructed pipeline ready to execute. */ public Pipeline andFinishWith(final Consumer completer) { - return new Pipeline(pipelineSource, stages, pipes, new CompleterStage<>(pipeEnd, completer)); + return new Pipeline(producingStage, stages, pipes, new CompleterStage<>(pipeEnd, completer)); } private PipelineBuilder addStage(final SingleStepStage singleStepStage) { @@ -205,7 +205,7 @@ private PipelineBuilder addStage( private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { final List newStages = concat(stages, stage); return new PipelineBuilder<>( - pipelineSource, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); + producingStage, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); } private List concat(final Collection existing, final X newItem) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java similarity index 94% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java index bc8d798a49..5d66c8f1ed 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineSource.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java @@ -12,7 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; -public interface PipelineSource { +interface ProducingStage { InputPipe getPipe(); void abort(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java index 93385e0ecc..f20c86d101 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java @@ -14,7 +14,7 @@ import java.util.Iterator; -class StreamSourceStage implements PipelineSource, Runnable { +class StreamSourceStage implements ProducingStage, Runnable { private final Iterator source; private final Pipe pipe; From ca65957ddb8eeca5bef6d4ae3f99e84ade107b1e Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 07:34:53 +1000 Subject: [PATCH 30/39] Fix race condition when aborting. --- .../pantheon/services/pipeline/Pipeline.java | 25 +++++++++++++++---- .../pipeline/PipelineBuilderTest.java | 3 +++ 2 files changed, 23 insertions(+), 5 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 489ed27634..76f83bf1a7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -33,6 +33,14 @@ public class Pipeline { private final Collection> pipes; private final CompleterStage completerStage; private final AtomicBoolean started = new AtomicBoolean(false); + + /** + * Flags that the pipeline is being completed so that when we abort we can close the streams + * without the completion stage then marking the future successful before we finish the abort + * process and mark it as exceptionally completed. We can't just use synchronized because it + * winds up being the same thread coming in via a callback so already has the lock. + */ + private final AtomicBoolean completing = new AtomicBoolean(false); private final CompletableFuture overallFuture = new CompletableFuture<>(); private volatile List> futures; @@ -55,7 +63,7 @@ public Pipeline( * @return a future that will be completed when the pipeline completes. If the pipeline fails or * is aborted the returned future will be completed exceptionally. */ - public CompletableFuture start(final ExecutorService executorService) { + public synchronized CompletableFuture start(final ExecutorService executorService) { if (!started.compareAndSet(false, true)) { return overallFuture; } @@ -67,7 +75,7 @@ public CompletableFuture start(final ExecutorService executorService) { .getFuture() .whenComplete( (result, error) -> { - synchronized (overallFuture) { + if (completing.compareAndSet(false, true)) { if (error != null) { overallFuture.completeExceptionally(error); } else { @@ -99,13 +107,20 @@ private Future runWithErrorHandling( task.run(); } catch (final Throwable t) { LOG.debug("Unhandled exception in pipeline. Aborting.", t); - abort(t); + try { + abort(t); + } catch (final Throwable t2) { + // Seems excessive but exceptions that propagate out of this method won't be logged + // because the executor just completes the future exceptionally and we never + // need to call get on it which would normally expose the error. + LOG.error("Failed to abort pipeline after error", t2); + } } }); } - private void abort(final Throwable error) { - synchronized (overallFuture) { + private synchronized void abort(final Throwable error) { + if (completing.compareAndSet(false, true)) { producingStage.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 8e4ef5e348..1a910fb72e 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -31,6 +31,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -267,6 +268,8 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { final CompletableFuture result = pipeline.start(executorService); assertThatThrownBy(() -> result.get(10, SECONDS)) + .isInstanceOf(ExecutionException.class) + .hasRootCauseExactlyInstanceOf(RuntimeException.class) .extracting(Throwable::getCause) .isSameAs(expectedError); } From 803784e71d7530b222f144c845d5bad2aae64d2e Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 07:43:51 +1000 Subject: [PATCH 31/39] Spotless. --- .../tech/pegasys/pantheon/services/pipeline/Pipeline.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 76f83bf1a7..a6181dff31 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -37,10 +37,11 @@ public class Pipeline { /** * Flags that the pipeline is being completed so that when we abort we can close the streams * without the completion stage then marking the future successful before we finish the abort - * process and mark it as exceptionally completed. We can't just use synchronized because it - * winds up being the same thread coming in via a callback so already has the lock. + * process and mark it as exceptionally completed. We can't just use synchronized because it winds + * up being the same thread coming in via a callback so already has the lock. */ private final AtomicBoolean completing = new AtomicBoolean(false); + private final CompletableFuture overallFuture = new CompletableFuture<>(); private volatile List> futures; From 7c799c090b1c5a035b16c5d7f3db4a52c1469f61 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 08:07:04 +1000 Subject: [PATCH 32/39] Add javadoc. --- .../pantheon/services/pipeline/InputPipe.java | 33 ++++++++++++++++++- .../services/pipeline/OutputPipe.java | 25 +++++++++++++- .../pantheon/services/pipeline/Pipe.java | 16 +++++++++ 3 files changed, 72 insertions(+), 2 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java index 9e11189afd..87242cc45d 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java @@ -19,12 +19,43 @@ * * @param the type of input. */ -interface InputPipe { +public interface InputPipe { + + /** + * Determines if this pipe has more items to be read. The pipe is considered to have no more items + * when it has either been aborted with {@link OutputPipe#abort()} or if all queued items have + * been read and the pipe has been closed with {@link OutputPipe#close()}. + * + * @return true if there are more items to process, otherwise false. + */ boolean hasMore(); + /** + * Get the next item from this pipe. This method will block until the next item is available but + * may still return null if the pipe is closed or the thread interrupted while + * waiting. + * + * @return the next item or null if the pipe is closed or the thread interrupted. + */ T get(); + /** + * Get the next item from this pipe without blocking if it is available. + * + * @return the next item or null if the pipe is empty. + */ T poll(); + /** + * Get a batch of values from the pipe containing at most maximumBatchSize items. + * This method will block until at least one item is available but will not wait until the batch + * is full. + * + *

An empty list will be returned if the queue is closed or the thread interrupted while + * waiting for the next value. + * + * @param maximumBatchSize the maximum number of items to read. + * @return the batch that was read. + */ List getBatch(int maximumBatchSize); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java index aea535b788..8628494780 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java @@ -17,14 +17,37 @@ * * @param the type of output. */ -interface OutputPipe { +public interface OutputPipe { + + /** + * Determine if this pipe is still open and accepting output. + * + * @return true if and only if the pipe is open. + */ boolean isOpen(); + /** + * Adds a new item to the pipe. This method will block until capacity is available in the pipe. + * The item will be discarded if the pipe is closed befoer capacity becomes available. + * + * @param value the value to add to the pipe. + */ void put(T value); + /** + * Determine if this pipe has capacity to accept another item. + * + * @return true if the pipe has capacity to accept one more item. + */ boolean hasRemainingCapacity(); + /** + * Close this output pipe indicating that no further data will be published to it. When reading + * from the other end of this pipe {@link InputPipe#hasMore()} will continue to return true until + * all the already queued data has been drained. + */ void close(); + /** Abort this pipe. The pipe is closed and any queued data is discarded. */ void abort(); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index b8507946c8..ad4e9e233e 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -23,6 +23,17 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +/** + * Forms the connection between two pipeline stages. A pipe is essentially a blocking queue with the + * added ability to signal when no further input is available because the pipe has been closed or + * the pipeline aborted. + * + *

In most cases a Pipe is used through one of two narrower interfaces it supports {@link + * InputPipe} and {@link OutputPipe}. These are designed to expose only the operations relevant to + * objects either reading from or publishing to the pipe respectively. + * + * @param the type of item that flows through the pipe. + */ public class Pipe implements InputPipe, OutputPipe { private static final Logger LOG = LogManager.getLogger(); private final BlockingQueue queue; @@ -40,6 +51,11 @@ public boolean isOpen() { return !closed.get() && !aborted.get(); } + /** + * Get the number of items that can be queued inside this pipe. + * + * @return the pipe's capacity. + */ public int getCapacity() { return capacity; } From dc87314acbe393084ee42a42097713251ae50f62 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 08:42:01 +1000 Subject: [PATCH 33/39] More clearly separate stages (which are runnable) from processors (which are not). --- ...tage.java => AsyncOperationProcessor.java} | 4 +- ...chingStage.java => BatchingProcessor.java} | 5 +-- .../pipeline/ExternalProducingStage.java | 32 ---------------- ...latMapStage.java => FlatMapProcessor.java} | 4 +- ...rceStage.java => IteratorSourceStage.java} | 14 +------ .../{ProcessStage.java => MapProcessor.java} | 4 +- .../pantheon/services/pipeline/Pipeline.java | 8 ++-- .../services/pipeline/PipelineBuilder.java | 38 +++++++++---------- ...tOutputStage.java => ProcessingStage.java} | 14 +++---- .../{SingleStepStage.java => Processor.java} | 2 +- .../services/pipeline/ProducingStage.java | 19 ---------- .../services/pipeline/SharedOutputPipe.java | 14 +++++++ ...geTest.java => BatchingProcessorTest.java} | 4 +- ...ageTest.java => FlatMapProcessorTest.java} | 4 +- ...Test.java => IteratorSourceStageTest.java} | 6 +-- ...ssStageTest.java => MapProcessorTest.java} | 4 +- ...tageTest.java => ProcessingStageTest.java} | 8 ++-- 17 files changed, 67 insertions(+), 117 deletions(-) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{AsyncProcessStage.java => AsyncOperationProcessor.java} (97%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{BatchingStage.java => BatchingProcessor.java} (89%) delete mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{FlatMapStage.java => FlatMapProcessor.java} (89%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{StreamSourceStage.java => IteratorSourceStage.java} (79%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{ProcessStage.java => MapProcessor.java} (89%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{InputOutputStage.java => ProcessingStage.java} (74%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{SingleStepStage.java => Processor.java} (95%) delete mode 100644 services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{BatchingStageTest.java => BatchingProcessorTest.java} (94%) rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{FlatMapStageTest.java => FlatMapProcessorTest.java} (93%) rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{StreamSourceStageTest.java => IteratorSourceStageTest.java} (88%) rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{ProcessStageTest.java => MapProcessorTest.java} (93%) rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{InputOutputStageTest.java => ProcessingStageTest.java} (91%) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java similarity index 97% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java index 4ca4fd0361..039b57be30 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java @@ -24,13 +24,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -class AsyncProcessStage implements SingleStepStage { +class AsyncOperationProcessor implements Processor { private static final Logger LOG = LogManager.getLogger(); private final Function> processor; private final Collection> inProgress; private final int maxConcurrency; - public AsyncProcessStage( + public AsyncOperationProcessor( final Function> processor, final int maxConcurrency) { this.processor = processor; this.maxConcurrency = maxConcurrency; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java similarity index 89% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java index b673739740..015809919f 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java @@ -14,17 +14,16 @@ import java.util.List; -class BatchingStage implements SingleStepStage> { +class BatchingProcessor implements Processor> { private final int maximumBatchSize; - public BatchingStage(final int maximumBatchSize) { + public BatchingProcessor(final int maximumBatchSize) { this.maximumBatchSize = maximumBatchSize; } @Override public void processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { - final List batch = inputPipe.getBatch(maximumBatchSize); if (!batch.isEmpty()) { outputPipe.put(batch); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java deleted file mode 100644 index 2bac61f0d2..0000000000 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ExternalProducingStage.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Copyright 2019 ConsenSys AG. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package tech.pegasys.pantheon.services.pipeline; - -class ExternalProducingStage implements ProducingStage { - - private final Pipe pipe; - - public ExternalProducingStage(final Pipe pipe) { - this.pipe = pipe; - } - - @Override - public InputPipe getPipe() { - return pipe; - } - - @Override - public void abort() { - pipe.abort(); - } -} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java similarity index 89% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java index a2fb2c55a7..17e0b0bc47 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java @@ -15,11 +15,11 @@ import java.util.function.Function; import java.util.stream.Stream; -class FlatMapStage implements SingleStepStage { +class FlatMapProcessor implements Processor { private final Function> mapper; - public FlatMapStage(final Function> mapper) { + public FlatMapProcessor(final Function> mapper) { this.mapper = mapper; } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStage.java similarity index 79% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStage.java index f20c86d101..a393476e8b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStage.java @@ -14,11 +14,11 @@ import java.util.Iterator; -class StreamSourceStage implements ProducingStage, Runnable { +class IteratorSourceStage implements Runnable { private final Iterator source; private final Pipe pipe; - StreamSourceStage(final Iterator source, final Pipe pipe) { + IteratorSourceStage(final Iterator source, final Pipe pipe) { this.source = source; this.pipe = pipe; } @@ -33,14 +33,4 @@ public void run() { } pipe.close(); } - - @Override - public InputPipe getPipe() { - return pipe; - } - - @Override - public void abort() { - pipe.abort(); - } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java similarity index 89% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java index 079d5ea559..aae818a4f7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java @@ -14,11 +14,11 @@ import java.util.function.Function; -class ProcessStage implements SingleStepStage { +class MapProcessor implements Processor { private final Function processor; - public ProcessStage(final Function processor) { + public MapProcessor(final Function processor) { this.processor = processor; } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index a6181dff31..61c508ba40 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -28,7 +28,7 @@ public class Pipeline { private static final Logger LOG = LogManager.getLogger(); - private final ProducingStage producingStage; + private final Pipe inputPipe; private final Collection stages; private final Collection> pipes; private final CompleterStage completerStage; @@ -46,11 +46,11 @@ public class Pipeline { private volatile List> futures; public Pipeline( - final ProducingStage producingStage, + final Pipe inputPipe, final Collection stages, final Collection> pipes, final CompleterStage completerStage) { - this.producingStage = producingStage; + this.inputPipe = inputPipe; this.stages = stages; this.pipes = pipes; this.completerStage = completerStage; @@ -122,7 +122,7 @@ private Future runWithErrorHandling( private synchronized void abort(final Throwable error) { if (completing.compareAndSet(false, true)) { - producingStage.abort(); + inputPipe.abort(); pipes.forEach(Pipe::abort); futures.forEach(future -> future.cancel(true)); overallFuture.completeExceptionally(error); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index fde436558a..954f4ec276 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -37,20 +37,20 @@ */ public class PipelineBuilder { - private final ProducingStage producingStage; + private final Pipe inputPipe; private final Collection stages; private final Collection> pipes; private final InputPipe pipeEnd; private final int bufferSize; public PipelineBuilder( - final ProducingStage producingStage, + final Pipe inputPipe, final Collection stages, final Collection> pipes, final InputPipe pipeEnd, final int bufferSize) { checkArgument(!pipes.isEmpty(), "Must have at least one pipe in a pipeline"); - this.producingStage = producingStage; + this.inputPipe = inputPipe; this.stages = stages; this.pipes = pipes; this.pipeEnd = pipeEnd; @@ -70,9 +70,8 @@ public PipelineBuilder( public static PipelineBuilder createPipelineFrom( final Iterator source, final int bufferSize) { final Pipe pipe = new Pipe<>(bufferSize); - final StreamSourceStage sourceStage = new StreamSourceStage<>(source, pipe); - return new PipelineBuilder<>( - sourceStage, singleton(sourceStage), singleton(pipe), pipe, bufferSize); + final IteratorSourceStage sourceStage = new IteratorSourceStage<>(source, pipe); + return new PipelineBuilder<>(pipe, singleton(sourceStage), singleton(pipe), pipe, bufferSize); } /** @@ -84,8 +83,7 @@ public static PipelineBuilder createPipelineFrom( * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public static PipelineBuilder createPipelineFrom(final Pipe pipe) { - return new PipelineBuilder<>( - new ExternalProducingStage<>(pipe), emptyList(), singleton(pipe), pipe, pipe.getCapacity()); + return new PipelineBuilder<>(pipe, emptyList(), singleton(pipe), pipe, pipe.getCapacity()); } /** @@ -97,7 +95,7 @@ public static PipelineBuilder createPipelineFrom(final Pipe pipe) { * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder thenProcess(final Function processor) { - final SingleStepStage singleStepStage = new ProcessStage<>(processor); + final Processor singleStepStage = new MapProcessor<>(processor); return addStage(singleStepStage); } @@ -119,11 +117,11 @@ public PipelineBuilder thenProcessInParallel( final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { final Runnable processStage = - new InputOutputStage<>(pipeEnd, outputPipe, new ProcessStage<>(processor)); + new ProcessingStage<>(pipeEnd, outputPipe, new MapProcessor<>(processor)); newStages.add(processStage); } return new PipelineBuilder<>( - producingStage, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); + inputPipe, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); } /** @@ -144,7 +142,7 @@ public PipelineBuilder thenProcessInParallel( */ public PipelineBuilder thenProcessAsync( final Function> processor, final int maxConcurrency) { - return addStage(new AsyncProcessStage<>(processor, maxConcurrency)); + return addStage(new AsyncOperationProcessor<>(processor, maxConcurrency)); } /** @@ -159,7 +157,7 @@ public PipelineBuilder thenProcessAsync( * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder> inBatches(final int maximumBatchSize) { - return addStage(new BatchingStage<>(maximumBatchSize), bufferSize / maximumBatchSize + 1); + return addStage(new BatchingProcessor<>(maximumBatchSize), bufferSize / maximumBatchSize + 1); } /** @@ -178,7 +176,7 @@ public PipelineBuilder> inBatches(final int maximumBatchSize) { */ public PipelineBuilder thenFlatMap( final Function> mapper, final int newBufferSize) { - return addStage(new FlatMapStage<>(mapper), newBufferSize); + return addStage(new FlatMapProcessor<>(mapper), newBufferSize); } /** @@ -188,24 +186,24 @@ public PipelineBuilder thenFlatMap( * @return the constructed pipeline ready to execute. */ public Pipeline andFinishWith(final Consumer completer) { - return new Pipeline(producingStage, stages, pipes, new CompleterStage<>(pipeEnd, completer)); + return new Pipeline(inputPipe, stages, pipes, new CompleterStage<>(pipeEnd, completer)); } - private PipelineBuilder addStage(final SingleStepStage singleStepStage) { - return addStage(singleStepStage, bufferSize); + private PipelineBuilder addStage(final Processor processor) { + return addStage(processor, bufferSize); } private PipelineBuilder addStage( - final SingleStepStage singleStepStage, final int newBufferSize) { + final Processor processor, final int newBufferSize) { final Pipe outputPipe = new Pipe<>(newBufferSize); - final Runnable processStage = new InputOutputStage<>(pipeEnd, outputPipe, singleStepStage); + final Runnable processStage = new ProcessingStage<>(pipeEnd, outputPipe, processor); return addStage(processStage, outputPipe); } private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { final List newStages = concat(stages, stage); return new PipelineBuilder<>( - producingStage, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); + inputPipe, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); } private List concat(final Collection existing, final X newItem) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java similarity index 74% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java index 2800e22f14..614f04859b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputOutputStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java @@ -12,27 +12,27 @@ */ package tech.pegasys.pantheon.services.pipeline; -class InputOutputStage implements Runnable { +class ProcessingStage implements Runnable { private final InputPipe inputPipe; private final OutputPipe outputPipe; - private final SingleStepStage singleStepStage; + private final Processor processor; - public InputOutputStage( + public ProcessingStage( final InputPipe inputPipe, final OutputPipe outputPipe, - final SingleStepStage singleStepStage) { + final Processor processor) { this.inputPipe = inputPipe; this.outputPipe = outputPipe; - this.singleStepStage = singleStepStage; + this.processor = processor; } @Override public void run() { while (inputPipe.hasMore()) { - singleStepStage.processNextInput(inputPipe, outputPipe); + processor.processNextInput(inputPipe, outputPipe); } - singleStepStage.finalize(outputPipe); + processor.finalize(outputPipe); outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java similarity index 95% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java index 9115892ff7..729f4544e4 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SingleStepStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java @@ -12,7 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; -interface SingleStepStage { +interface Processor { void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); default void finalize(final OutputPipe outputPipe) {} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java deleted file mode 100644 index 5d66c8f1ed..0000000000 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProducingStage.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Copyright 2019 ConsenSys AG. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package tech.pegasys.pantheon.services.pipeline; - -interface ProducingStage { - InputPipe getPipe(); - - void abort(); -} diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java index 5ee880b78e..0553b6be7b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java @@ -14,10 +14,24 @@ import java.util.concurrent.atomic.AtomicInteger; +/** + * A wrapper around an {@link OutputPipe} which allows multiple stages to share the same output + * pipe. Most operations simply pass through to the underlying pipe but the underlying pipe is only + * closed when all stages have signalled this pipe should close. + * + * @param the type of item in the pipe. + */ class SharedOutputPipe implements OutputPipe { private final OutputPipe delegate; private final AtomicInteger remainingClosesRequired; + /** + * Creates a new SharedOutputPipe. + * + * @param delegate the pipe to wrap. + * @param closesRequired the number of stages this output pipe will be shared with. The underlying + * pipe will only be closed when {@link #close()} is called this many times. + */ public SharedOutputPipe(final OutputPipe delegate, final int closesRequired) { this.delegate = delegate; this.remainingClosesRequired = new AtomicInteger(closesRequired); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java similarity index 94% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java index 3da767037b..460cbef294 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java @@ -21,11 +21,11 @@ import org.junit.Test; -public class BatchingStageTest { +public class BatchingProcessorTest { private final Pipe inputPipe = new Pipe<>(10); private final Pipe> outputPipe = new Pipe<>(10); - private final BatchingStage stage = new BatchingStage<>(3); + private final BatchingProcessor stage = new BatchingProcessor<>(3); @Test public void shouldCreateBatches() { diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java similarity index 93% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java index da07bac1ba..8f89b748ba 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java @@ -23,7 +23,7 @@ import org.junit.Test; -public class FlatMapStageTest { +public class FlatMapProcessorTest { private final Pipe input = new Pipe<>(10); private final Pipe output = new Pipe<>(10); @@ -31,7 +31,7 @@ public class FlatMapStageTest { @SuppressWarnings("unchecked") private final Function> mapper = mock(Function.class); - private final FlatMapStage stage = new FlatMapStage<>(mapper); + private final FlatMapProcessor stage = new FlatMapProcessor<>(mapper); @Test public void shouldOutputItemsFromReturnedStreamIndividually() { diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java similarity index 88% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java index aef79f7b00..6cb5a1b718 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/StreamSourceStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java @@ -17,12 +17,12 @@ import com.google.common.collect.Iterators; import org.junit.Test; -public class StreamSourceStageTest { +public class IteratorSourceStageTest { private final Pipe output = new Pipe<>(10); - private final StreamSourceStage stage = - new StreamSourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); + private final IteratorSourceStage stage = + new IteratorSourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); @Test public void shouldOutputEntriesThenClosePipe() { diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java similarity index 93% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java index 56620c5f75..814219012f 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java @@ -22,7 +22,7 @@ import org.junit.Test; -public class ProcessStageTest { +public class MapProcessorTest { private final Pipe input = new Pipe<>(10); private final Pipe output = new Pipe<>(10); @@ -30,7 +30,7 @@ public class ProcessStageTest { @SuppressWarnings("unchecked") private final Function processor = mock(Function.class); - private final ProcessStage stage = new ProcessStage<>(processor); + private final MapProcessor stage = new MapProcessor<>(processor); @Test public void shouldApplyFunctionToItems() { diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java similarity index 91% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java index 457d356e63..4f81a1d751 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/InputOutputStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java @@ -27,16 +27,16 @@ import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) -public class InputOutputStageTest { +public class ProcessingStageTest { private final Pipe inputPipe = new Pipe<>(10); private final Pipe outputPipe = new Pipe<>(10); - @Mock private SingleStepStage singleStep; - private InputOutputStage stage; + @Mock private Processor singleStep; + private ProcessingStage stage; @Before public void setUp() { - stage = new InputOutputStage<>(inputPipe, outputPipe, singleStep); + stage = new ProcessingStage<>(inputPipe, outputPipe, singleStep); doAnswer( invocation -> { outputPipe.put(inputPipe.get().toLowerCase(Locale.UK)); From 714d62cd0a02a6991d5ac2476ade022dcf7ac070 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 10:54:46 +1000 Subject: [PATCH 34/39] Add a counter to track metrics for each stage output count. --- .../pipeline/AsyncOperationProcessor.java | 20 +++-- .../services/pipeline/BatchingProcessor.java | 4 +- .../services/pipeline/CompleterStage.java | 8 +- .../services/pipeline/FlatMapProcessor.java | 11 ++- .../services/pipeline/MapProcessor.java | 4 +- .../services/pipeline/PipelineBuilder.java | 82 ++++++++++++++----- .../services/pipeline/ProcessingStage.java | 13 ++- .../pantheon/services/pipeline/Processor.java | 6 +- .../pipeline/BatchingProcessorTest.java | 12 +++ .../services/pipeline/CompleterStageTest.java | 10 ++- .../pipeline/PipelineBuilderTest.java | 76 ++++++++++++++--- .../pipeline/ProcessingStageTest.java | 9 +- 12 files changed, 200 insertions(+), 55 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java index 039b57be30..39f5238db2 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java @@ -38,7 +38,7 @@ public AsyncOperationProcessor( } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { if (inProgress.size() < maxConcurrency) { final I value = inputPipe.get(); if (value != null) { @@ -50,23 +50,25 @@ public void processNextInput(final InputPipe inputPipe, final OutputPipe o inProgress.add(future); } - outputCompletedTasks(0, outputPipe); + return outputCompletedTasks(0, outputPipe); } else { - outputNextCompletedTask(outputPipe); + return outputNextCompletedTask(outputPipe); } } @Override - public void finalize(final OutputPipe outputPipe) { + public int finalize(final OutputPipe outputPipe) { + int outputCount = 0; while (!inProgress.isEmpty()) { - outputNextCompletedTask(outputPipe); + outputCount += outputNextCompletedTask(outputPipe); } + return outputCount; } - private void outputNextCompletedTask(final OutputPipe outputPipe) { + private int outputNextCompletedTask(final OutputPipe outputPipe) { try { waitForAnyFutureToComplete(); - outputCompletedTasks(1, outputPipe); + return outputCompletedTasks(1, outputPipe); } catch (final InterruptedException e) { LOG.trace("Interrupted while waiting for processing to complete", e); } catch (final ExecutionException e) { @@ -74,6 +76,7 @@ private void outputNextCompletedTask(final OutputPipe outputPipe) { } catch (final TimeoutException e) { // Ignore and go back around the loop. } + return 0; } @SuppressWarnings("rawtypes") @@ -82,7 +85,7 @@ private void waitForAnyFutureToComplete() CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])).get(1, TimeUnit.SECONDS); } - private void outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { + private int outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { int outputTasks = 0; for (final Iterator> i = inProgress.iterator(); i.hasNext() && (outputTasks < minTasksToOutput || outputPipe.hasRemainingCapacity()); ) { @@ -94,5 +97,6 @@ private void outputCompletedTasks(final int minTasksToOutput, final OutputPipe inputPipe, final OutputPipe> outputPipe) { + public int processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { final List batch = inputPipe.getBatch(maximumBatchSize); if (!batch.isEmpty()) { outputPipe.put(batch); + return 1; } + return 0; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java index e4d3aa7f37..1fc61af67e 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java @@ -12,17 +12,22 @@ */ package tech.pegasys.pantheon.services.pipeline; +import tech.pegasys.pantheon.metrics.Counter; + import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; class CompleterStage implements Runnable { private final InputPipe input; private final Consumer completer; + private final Counter outputCounter; private final CompletableFuture future = new CompletableFuture<>(); - CompleterStage(final InputPipe input, final Consumer completer) { + CompleterStage( + final InputPipe input, final Consumer completer, final Counter outputCounter) { this.input = input; this.completer = completer; + this.outputCounter = outputCounter; } @Override @@ -31,6 +36,7 @@ public void run() { final T value = input.get(); if (value != null) { completer.accept(value); + outputCounter.inc(); } } future.complete(null); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java index 17e0b0bc47..e1005602eb 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java @@ -12,6 +12,7 @@ */ package tech.pegasys.pantheon.services.pipeline; +import java.util.Iterator; import java.util.function.Function; import java.util.stream.Stream; @@ -24,10 +25,16 @@ public FlatMapProcessor(final Function> mapper) { } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + int outputCount = 0; final I value = inputPipe.get(); if (value != null) { - mapper.apply(value).forEach(outputPipe::put); + final Iterator outputs = mapper.apply(value).iterator(); + while (outputs.hasNext()) { + outputPipe.put(outputs.next()); + outputCount++; + } } + return outputCount; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java index aae818a4f7..9338a540c1 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java @@ -23,10 +23,12 @@ public MapProcessor(final Function processor) { } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { final I value = inputPipe.get(); if (value != null) { outputPipe.put(processor.apply(value)); + return 1; } + return 0; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 954f4ec276..f059470f43 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -15,6 +15,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Collections.emptyList; import static java.util.Collections.singleton; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_LABELLED_COUNTER; + +import tech.pegasys.pantheon.metrics.Counter; +import tech.pegasys.pantheon.metrics.LabelledMetric; import java.util.ArrayList; import java.util.Collection; @@ -42,13 +46,16 @@ public class PipelineBuilder { private final Collection> pipes; private final InputPipe pipeEnd; private final int bufferSize; + private final LabelledMetric outputCounter; public PipelineBuilder( final Pipe inputPipe, final Collection stages, final Collection> pipes, final InputPipe pipeEnd, - final int bufferSize) { + final int bufferSize, + final LabelledMetric outputCounter) { + this.outputCounter = outputCounter; checkArgument(!pipes.isEmpty(), "Must have at least one pipe in a pipeline"); this.inputPipe = inputPipe; this.stages = stages; @@ -71,7 +78,8 @@ public static PipelineBuilder createPipelineFrom( final Iterator source, final int bufferSize) { final Pipe pipe = new Pipe<>(bufferSize); final IteratorSourceStage sourceStage = new IteratorSourceStage<>(source, pipe); - return new PipelineBuilder<>(pipe, singleton(sourceStage), singleton(pipe), pipe, bufferSize); + return new PipelineBuilder<>( + pipe, singleton(sourceStage), singleton(pipe), pipe, bufferSize, NO_OP_LABELLED_COUNTER); } /** @@ -83,20 +91,36 @@ public static PipelineBuilder createPipelineFrom( * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public static PipelineBuilder createPipelineFrom(final Pipe pipe) { - return new PipelineBuilder<>(pipe, emptyList(), singleton(pipe), pipe, pipe.getCapacity()); + return new PipelineBuilder<>( + pipe, emptyList(), singleton(pipe), pipe, pipe.getCapacity(), NO_OP_LABELLED_COUNTER); + } + + /** + * Stages added after this method is called will increment the counter for every task they + * complete processing and add to their output queue. Note that if the output queue is full the + * counter will not increment until space is available and the task has actually been output. + * + * @param outputCounter the counter to increment for each output of a stage. Must have a single + * label which will be filled with the stage name. + * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. + */ + public PipelineBuilder trackOutputCount(final LabelledMetric outputCounter) { + return new PipelineBuilder<>(inputPipe, stages, pipes, pipeEnd, bufferSize, outputCounter); } /** * Adds a 1-to-1 processing stage to the pipeline. A single thread processes each item in the * pipeline with processor outputting its return value to the next stage. * + * @param stageName the name of this stage. Used as the label for the task count metric. * @param processor the processing to apply to each item. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ - public PipelineBuilder thenProcess(final Function processor) { + public PipelineBuilder thenProcess( + final String stageName, final Function processor) { final Processor singleStepStage = new MapProcessor<>(processor); - return addStage(singleStepStage); + return addStage(singleStepStage, stageName); } /** @@ -105,23 +129,25 @@ public PipelineBuilder thenProcess(final Function processor) { * *

Note: The order of items is not preserved. * + * @param stageName the name of this stage. Used as the label for the task count metric. * @param processor the processing to apply to each item. * @param numberOfThreads the number of threads to use for processing. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder thenProcessInParallel( - final Function processor, final int numberOfThreads) { + final String stageName, final Function processor, final int numberOfThreads) { final Pipe newPipeEnd = new Pipe<>(bufferSize); final OutputPipe outputPipe = new SharedOutputPipe<>(newPipeEnd, numberOfThreads); final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { final Runnable processStage = - new ProcessingStage<>(pipeEnd, outputPipe, new MapProcessor<>(processor)); + new ProcessingStage<>( + pipeEnd, outputPipe, new MapProcessor<>(processor), outputCounter.labels(stageName)); newStages.add(processStage); } return new PipelineBuilder<>( - inputPipe, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize); + inputPipe, newStages, concat(pipes, newPipeEnd), newPipeEnd, bufferSize, outputCounter); } /** @@ -135,14 +161,17 @@ public PipelineBuilder thenProcessInParallel( * *

Note: The order of items is not preserved. * + * @param stageName the name of this stage. Used as the label for the task count metric. * @param processor the processing to apply to each item. * @param maxConcurrency the maximum number of items being processed concurrently. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder thenProcessAsync( - final Function> processor, final int maxConcurrency) { - return addStage(new AsyncOperationProcessor<>(processor, maxConcurrency)); + final String stageName, + final Function> processor, + final int maxConcurrency) { + return addStage(new AsyncOperationProcessor<>(processor, maxConcurrency), stageName); } /** @@ -153,11 +182,13 @@ public PipelineBuilder thenProcessAsync( * *

The output buffer size is reduced to bufferSize / maximumBatchSize + 1. * + * @param stageName the name of this stage. Used as the label for the task count metric. * @param maximumBatchSize the maximum number of items to include in a batch. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ - public PipelineBuilder> inBatches(final int maximumBatchSize) { - return addStage(new BatchingProcessor<>(maximumBatchSize), bufferSize / maximumBatchSize + 1); + public PipelineBuilder> inBatches(final String stageName, final int maximumBatchSize) { + return addStage( + new BatchingProcessor<>(maximumBatchSize), bufferSize / maximumBatchSize + 1, stageName); } /** @@ -165,45 +196,52 @@ public PipelineBuilder> inBatches(final int maximumBatchSize) { * is called and each item of the {@link Stream} it returns is output as an individual item. The * returned Stream may be empty to remove an item. * - *

This can be used to reverse the effect of {@link #inBatches(int)} with: + *

This can be used to reverse the effect of {@link #inBatches(String, int)} with: * *

thenFlatMap(List::stream, newBufferSize)
* + * @param stageName the name of this stage. Used as the label for the task count metric. * @param mapper the function to process each item with. * @param newBufferSize the output buffer size to use from this stage onwards. * @param the type of items to be output from this stage. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder thenFlatMap( - final Function> mapper, final int newBufferSize) { - return addStage(new FlatMapProcessor<>(mapper), newBufferSize); + final String stageName, final Function> mapper, final int newBufferSize) { + return addStage(new FlatMapProcessor<>(mapper), newBufferSize, stageName); } /** * End the pipeline with a {@link Consumer} that is the last stage of the pipeline. * + * @param stageName the name of this stage. Used as the label for the task count metric. * @param completer the {@link Consumer} that accepts the final output of the pipeline. * @return the constructed pipeline ready to execute. */ - public Pipeline andFinishWith(final Consumer completer) { - return new Pipeline(inputPipe, stages, pipes, new CompleterStage<>(pipeEnd, completer)); + public Pipeline andFinishWith(final String stageName, final Consumer completer) { + return new Pipeline( + inputPipe, + stages, + pipes, + new CompleterStage<>(pipeEnd, completer, outputCounter.labels(stageName))); } - private PipelineBuilder addStage(final Processor processor) { - return addStage(processor, bufferSize); + private PipelineBuilder addStage(final Processor processor, final String stageName) { + return addStage(processor, bufferSize, stageName); } private PipelineBuilder addStage( - final Processor processor, final int newBufferSize) { + final Processor processor, final int newBufferSize, final String stageName) { final Pipe outputPipe = new Pipe<>(newBufferSize); - final Runnable processStage = new ProcessingStage<>(pipeEnd, outputPipe, processor); + final Runnable processStage = + new ProcessingStage<>(pipeEnd, outputPipe, processor, outputCounter.labels(stageName)); return addStage(processStage, outputPipe); } private PipelineBuilder addStage(final Runnable stage, final Pipe outputPipe) { final List newStages = concat(stages, stage); return new PipelineBuilder<>( - inputPipe, newStages, concat(pipes, outputPipe), outputPipe, bufferSize); + inputPipe, newStages, concat(pipes, outputPipe), outputPipe, bufferSize, outputCounter); } private List concat(final Collection existing, final X newItem) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java index 614f04859b..0bc199ef16 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java @@ -12,27 +12,34 @@ */ package tech.pegasys.pantheon.services.pipeline; +import tech.pegasys.pantheon.metrics.Counter; + class ProcessingStage implements Runnable { private final InputPipe inputPipe; private final OutputPipe outputPipe; private final Processor processor; + private final Counter outputCounter; public ProcessingStage( final InputPipe inputPipe, final OutputPipe outputPipe, - final Processor processor) { + final Processor processor, + final Counter outputCounter) { this.inputPipe = inputPipe; this.outputPipe = outputPipe; this.processor = processor; + this.outputCounter = outputCounter; } @Override public void run() { while (inputPipe.hasMore()) { - processor.processNextInput(inputPipe, outputPipe); + final int outputCount = processor.processNextInput(inputPipe, outputPipe); + outputCounter.inc(outputCount); } - processor.finalize(outputPipe); + final int outputCount = processor.finalize(outputPipe); + outputCounter.inc(outputCount); outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java index 729f4544e4..56492c5e99 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java @@ -13,7 +13,9 @@ package tech.pegasys.pantheon.services.pipeline; interface Processor { - void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); + int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); - default void finalize(final OutputPipe outputPipe) {} + default int finalize(final OutputPipe outputPipe) { + return 0; + } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java index 460cbef294..5c8d07c2d1 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java @@ -48,6 +48,18 @@ public void shouldCreateBatches() { assertThat(outputPipe.poll()).isNull(); } + @Test + public void shouldCountABatchAsOneNotAsThenNumberOfItemsInTheBatch() { + for (int i = 1; i <= 8; i++) { + inputPipe.put(i); + } + inputPipe.close(); + + assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); + assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); + assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); + } + @Test public void shouldNotOutputItemWhenInputIsClosed() { @SuppressWarnings("unchecked") diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java index f6e1828862..800a87479a 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java @@ -13,6 +13,11 @@ package tech.pegasys.pantheon.services.pipeline; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import tech.pegasys.pantheon.metrics.Counter; import java.util.ArrayList; import java.util.List; @@ -23,7 +28,9 @@ public class CompleterStageTest { private final Pipe pipe = new Pipe<>(10); private final List output = new ArrayList<>(); - private final CompleterStage stage = new CompleterStage<>(pipe, output::add); + private final Counter outputCounter = mock(Counter.class); + private final CompleterStage stage = + new CompleterStage<>(pipe, output::add, outputCounter); @Test public void shouldAddItemsToOutputUntilPipeHasNoMore() { @@ -35,5 +42,6 @@ public void shouldAddItemsToOutputUntilPipeHasNoMore() { stage.run(); assertThat(output).containsExactly("a", "b", "c"); + verify(outputCounter, times(3)).inc(); } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 1a910fb72e..6ac6edf888 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -21,14 +21,18 @@ import static org.assertj.core.api.Assertions.fail; import static org.awaitility.Awaitility.waitAtMost; +import tech.pegasys.pantheon.metrics.Counter; + import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -37,6 +41,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.stream.Stream; @@ -68,7 +73,7 @@ public void afterClass() throws Exception { public void shouldPipeTasksFromSupplierToCompleter() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10).andFinishWith(output::add); + PipelineBuilder.createPipelineFrom(tasks, 10).andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); result.get(10, SECONDS); assertThat(output).containsExactly(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); @@ -79,8 +84,8 @@ public void shouldPassInputThroughIntermediateStage() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) - .thenProcess(Object::toString) - .andFinishWith(output::add); + .thenProcess("toString", Object::toString) + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); result.get(10, SECONDS); @@ -95,7 +100,9 @@ public void shouldCombineIntoBatches() throws Exception { tasks.forEachRemaining(input::put); final BlockingQueue> output = new ArrayBlockingQueue<>(10); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(input).inBatches(6).andFinishWith(output::offer); + PipelineBuilder.createPipelineFrom(input) + .inBatches("batch", 6) + .andFinishWith("end", output::offer); final CompletableFuture result = pipeline.start(executorService); @@ -122,8 +129,8 @@ public void shouldProcessAsync() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) - .thenProcessAsync(value -> completedFuture(Integer.toString(value)), 3) - .andFinishWith(output::add); + .thenProcessAsync("toString", value -> completedFuture(Integer.toString(value)), 3) + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); result.get(10, SECONDS); assertThat(output) @@ -138,13 +145,14 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { final Pipeline pipeline = PipelineBuilder.createPipelineFrom(asList(1, 2, 3, 4, 5, 6, 7).iterator(), 10) .thenProcessAsync( + "createFuture", value -> { final CompletableFuture future = new CompletableFuture<>(); futures.add(future); return future; }, 3) - .andFinishWith(output::add); + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); waitForSize(futures, 3); @@ -173,8 +181,8 @@ public void shouldFlatMapItems() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) - .thenFlatMap(input -> Stream.of(input, input * 2), 20) - .andFinishWith(output::add); + .thenFlatMap("flatMap", input -> Stream.of(input, input * 2), 20) + .andFinishWith("end", output::add); pipeline.start(executorService).get(10, SECONDS); @@ -191,6 +199,7 @@ public void shouldProcessInParallel() throws Exception { final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcessInParallel( + "stageName", value -> { if (value == 2) { try { @@ -202,7 +211,7 @@ public void shouldProcessInParallel() throws Exception { return value.toString(); }, 2) - .andFinishWith(output::add); + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); // One thread will block but the other should process the remaining entries. @@ -227,6 +236,7 @@ public void shouldAbortPipeline() throws Exception { final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcess( + "stageName", value -> { if (value > allowProcessingUpTo) { try { @@ -238,7 +248,7 @@ public void shouldAbortPipeline() throws Exception { } return value; }) - .andFinishWith(output::add); + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); @@ -259,11 +269,12 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { final Pipeline pipeline = PipelineBuilder.createPipelineFrom(tasks, 10) .thenProcess( + "stageName", (Function) value -> { throw expectedError; }) - .andFinishWith(new ArrayList()::add); + .andFinishWith("end", new ArrayList()::add); final CompletableFuture result = pipeline.start(executorService); @@ -274,7 +285,48 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { .isSameAs(expectedError); } + @Test + public void shouldTrackTaskCountMetric() throws Exception { + final Map counters = new ConcurrentHashMap<>(); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom(tasks, 10) + .trackOutputCount( + labels -> counters.computeIfAbsent(labels[0], label -> new SimpleCounter())) + .thenProcess("map", Function.identity()) + .thenProcessInParallel("parallel", Function.identity(), 3) + .thenProcessAsync("async", CompletableFuture::completedFuture, 3) + .inBatches("batch", 4) + .thenFlatMap("flatMap", List::stream, 10) + .andFinishWith("finish", new ArrayList<>()::add); + + pipeline.start(executorService).get(10, SECONDS); + + assertThat(counters).containsOnlyKeys("map", "parallel", "async", "batch", "flatMap", "finish"); + assertThat(counters.get("map").count).hasValue(15); + assertThat(counters.get("parallel").count).hasValue(15); + assertThat(counters.get("async").count).hasValue(15); + assertThat(counters.get("flatMap").count).hasValue(15); + assertThat(counters.get("finish").count).hasValue(15); + // We don't know how many batches will be produced because it's timing dependent but it must + // be at least 4 to fit all the items and shouldn't be more than the items we put in. + assertThat(counters.get("batch").count).hasValueBetween(4, 15); + } + private void waitForSize(final Collection collection, final int targetSize) { waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(collection).hasSize(targetSize)); } + + private static class SimpleCounter implements Counter { + private final AtomicLong count = new AtomicLong(0); + + @Override + public void inc() { + count.incrementAndGet(); + } + + @Override + public void inc(final long amount) { + count.addAndGet(amount); + } + } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java index 4f81a1d751..cd5a7b7d3f 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java @@ -14,10 +14,13 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; +import tech.pegasys.pantheon.metrics.Counter; + import java.util.Locale; import org.junit.Before; @@ -32,15 +35,16 @@ public class ProcessingStageTest { private final Pipe inputPipe = new Pipe<>(10); private final Pipe outputPipe = new Pipe<>(10); @Mock private Processor singleStep; + private final Counter outputCounter = mock(Counter.class); private ProcessingStage stage; @Before public void setUp() { - stage = new ProcessingStage<>(inputPipe, outputPipe, singleStep); + stage = new ProcessingStage<>(inputPipe, outputPipe, singleStep, outputCounter); doAnswer( invocation -> { outputPipe.put(inputPipe.get().toLowerCase(Locale.UK)); - return null; + return 1; }) .when(singleStep) .processNextInput(inputPipe, outputPipe); @@ -61,6 +65,7 @@ public void shouldCallSingleStepStageForEachInput() { assertThat(outputPipe.poll()).isNull(); verify(singleStep, times(3)).processNextInput(inputPipe, outputPipe); + verify(outputCounter, times(3)).inc(1); } @Test From 6b2cfb97093c900528cac6a93e554f03ccfc1a09 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 11:28:21 +1000 Subject: [PATCH 35/39] Put the metrics in the pipe not the stage. --- .../metrics/noop/NoOpMetricsSystem.java | 2 +- .../pipeline/AsyncOperationProcessor.java | 20 +++---- .../services/pipeline/BatchingProcessor.java | 4 +- .../services/pipeline/FlatMapProcessor.java | 5 +- .../services/pipeline/MapProcessor.java | 4 +- .../pantheon/services/pipeline/Pipe.java | 7 ++- .../services/pipeline/PipelineBuilder.java | 55 ++++++++----------- .../services/pipeline/ProcessingStage.java | 13 +---- .../pantheon/services/pipeline/Processor.java | 6 +- .../pipeline/BatchingProcessorTest.java | 17 +----- .../services/pipeline/CompleterStageTest.java | 3 +- .../pipeline/FlatMapProcessorTest.java | 5 +- .../pipeline/IteratorSourceStageTest.java | 3 +- .../services/pipeline/MapProcessorTest.java | 5 +- .../pantheon/services/pipeline/PipeTest.java | 16 +++++- .../pipeline/PipelineBuilderTest.java | 35 +++++++----- .../pipeline/ProcessingStageTest.java | 12 ++-- 17 files changed, 100 insertions(+), 112 deletions(-) diff --git a/metrics/src/main/java/tech/pegasys/pantheon/metrics/noop/NoOpMetricsSystem.java b/metrics/src/main/java/tech/pegasys/pantheon/metrics/noop/NoOpMetricsSystem.java index 52a965e2b1..3c1242fe71 100644 --- a/metrics/src/main/java/tech/pegasys/pantheon/metrics/noop/NoOpMetricsSystem.java +++ b/metrics/src/main/java/tech/pegasys/pantheon/metrics/noop/NoOpMetricsSystem.java @@ -29,7 +29,7 @@ public class NoOpMetricsSystem implements MetricsSystem { - private static final Counter NO_OP_COUNTER = new NoOpCounter(); + public static final Counter NO_OP_COUNTER = new NoOpCounter(); private static final TimingContext NO_OP_TIMING_CONTEXT = () -> 0; private static final OperationTimer NO_OP_TIMER = () -> NO_OP_TIMING_CONTEXT; public static final LabelledMetric NO_OP_LABELLED_TIMER = label -> NO_OP_TIMER; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java index 39f5238db2..039b57be30 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java @@ -38,7 +38,7 @@ public AsyncOperationProcessor( } @Override - public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { if (inProgress.size() < maxConcurrency) { final I value = inputPipe.get(); if (value != null) { @@ -50,25 +50,23 @@ public int processNextInput(final InputPipe inputPipe, final OutputPipe ou inProgress.add(future); } - return outputCompletedTasks(0, outputPipe); + outputCompletedTasks(0, outputPipe); } else { - return outputNextCompletedTask(outputPipe); + outputNextCompletedTask(outputPipe); } } @Override - public int finalize(final OutputPipe outputPipe) { - int outputCount = 0; + public void finalize(final OutputPipe outputPipe) { while (!inProgress.isEmpty()) { - outputCount += outputNextCompletedTask(outputPipe); + outputNextCompletedTask(outputPipe); } - return outputCount; } - private int outputNextCompletedTask(final OutputPipe outputPipe) { + private void outputNextCompletedTask(final OutputPipe outputPipe) { try { waitForAnyFutureToComplete(); - return outputCompletedTasks(1, outputPipe); + outputCompletedTasks(1, outputPipe); } catch (final InterruptedException e) { LOG.trace("Interrupted while waiting for processing to complete", e); } catch (final ExecutionException e) { @@ -76,7 +74,6 @@ private int outputNextCompletedTask(final OutputPipe outputPipe) { } catch (final TimeoutException e) { // Ignore and go back around the loop. } - return 0; } @SuppressWarnings("rawtypes") @@ -85,7 +82,7 @@ private void waitForAnyFutureToComplete() CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])).get(1, TimeUnit.SECONDS); } - private int outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { + private void outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { int outputTasks = 0; for (final Iterator> i = inProgress.iterator(); i.hasNext() && (outputTasks < minTasksToOutput || outputPipe.hasRemainingCapacity()); ) { @@ -97,6 +94,5 @@ private int outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputTasks++; } } - return outputTasks; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java index ba87340fb4..015809919f 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java @@ -23,12 +23,10 @@ public BatchingProcessor(final int maximumBatchSize) { } @Override - public int processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { + public void processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { final List batch = inputPipe.getBatch(maximumBatchSize); if (!batch.isEmpty()) { outputPipe.put(batch); - return 1; } - return 0; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java index e1005602eb..4790517204 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java @@ -25,16 +25,13 @@ public FlatMapProcessor(final Function> mapper) { } @Override - public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { - int outputCount = 0; + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { final I value = inputPipe.get(); if (value != null) { final Iterator outputs = mapper.apply(value).iterator(); while (outputs.hasNext()) { outputPipe.put(outputs.next()); - outputCount++; } } - return outputCount; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java index 9338a540c1..aae818a4f7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java @@ -23,12 +23,10 @@ public MapProcessor(final Function processor) { } @Override - public int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { final I value = inputPipe.get(); if (value != null) { outputPipe.put(processor.apply(value)); - return 1; } - return 0; } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index ad4e9e233e..ccc35dc3c4 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -12,6 +12,8 @@ */ package tech.pegasys.pantheon.services.pipeline; +import tech.pegasys.pantheon.metrics.Counter; + import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -38,12 +40,14 @@ public class Pipe implements InputPipe, OutputPipe { private static final Logger LOG = LogManager.getLogger(); private final BlockingQueue queue; private final int capacity; + private final Counter itemCounter; private final AtomicBoolean closed = new AtomicBoolean(); private final AtomicBoolean aborted = new AtomicBoolean(); - public Pipe(final int capacity) { + public Pipe(final int capacity, final Counter itemCounter) { queue = new ArrayBlockingQueue<>(capacity); this.capacity = capacity; + this.itemCounter = itemCounter; } @Override @@ -120,6 +124,7 @@ public void put(final T value) { while (isOpen()) { try { if (queue.offer(value, 1, TimeUnit.SECONDS)) { + itemCounter.inc(); return; } } catch (final InterruptedException e) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index f059470f43..d2acb2b786 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -15,7 +15,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Collections.emptyList; import static java.util.Collections.singleton; -import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_LABELLED_COUNTER; import tech.pegasys.pantheon.metrics.Counter; import tech.pegasys.pantheon.metrics.LabelledMetric; @@ -69,17 +68,23 @@ public PipelineBuilder( * source returns false from {@link Iterator#hasNext()} and the last item has * been reached the end of the pipeline. * + * @param sourceName the name of this stage. Used as the label for the output count metric. * @param source the source to pull items from for processing. * @param bufferSize the number of items to be buffered between each stage in the pipeline. + * @param outputCounter the counter to increment for each output of a stage. Must have a single + * label which will be filled with the stage name. * @param the type of items input into the pipeline. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public static PipelineBuilder createPipelineFrom( - final Iterator source, final int bufferSize) { - final Pipe pipe = new Pipe<>(bufferSize); + final String sourceName, + final Iterator source, + final int bufferSize, + final LabelledMetric outputCounter) { + final Pipe pipe = new Pipe<>(bufferSize, outputCounter.labels(sourceName)); final IteratorSourceStage sourceStage = new IteratorSourceStage<>(source, pipe); return new PipelineBuilder<>( - pipe, singleton(sourceStage), singleton(pipe), pipe, bufferSize, NO_OP_LABELLED_COUNTER); + pipe, singleton(sourceStage), singleton(pipe), pipe, bufferSize, outputCounter); } /** @@ -87,32 +92,22 @@ public static PipelineBuilder createPipelineFrom( * pipe is closed and the last item has been reached the end of the pipeline. * * @param pipe the pipe feeding the pipeline. - * @param the type of items input into the pipeline. - * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. - */ - public static PipelineBuilder createPipelineFrom(final Pipe pipe) { - return new PipelineBuilder<>( - pipe, emptyList(), singleton(pipe), pipe, pipe.getCapacity(), NO_OP_LABELLED_COUNTER); - } - - /** - * Stages added after this method is called will increment the counter for every task they - * complete processing and add to their output queue. Note that if the output queue is full the - * counter will not increment until space is available and the task has actually been output. - * * @param outputCounter the counter to increment for each output of a stage. Must have a single * label which will be filled with the stage name. + * @param the type of items input into the pipeline. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ - public PipelineBuilder trackOutputCount(final LabelledMetric outputCounter) { - return new PipelineBuilder<>(inputPipe, stages, pipes, pipeEnd, bufferSize, outputCounter); + public static PipelineBuilder createPipelineFrom( + final Pipe pipe, final LabelledMetric outputCounter) { + return new PipelineBuilder<>( + pipe, emptyList(), singleton(pipe), pipe, pipe.getCapacity(), outputCounter); } /** * Adds a 1-to-1 processing stage to the pipeline. A single thread processes each item in the * pipeline with processor outputting its return value to the next stage. * - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param processor the processing to apply to each item. * @param the output type for this processing step. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. @@ -129,7 +124,7 @@ public PipelineBuilder thenProcess( * *

Note: The order of items is not preserved. * - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param processor the processing to apply to each item. * @param numberOfThreads the number of threads to use for processing. * @param the output type for this processing step. @@ -137,13 +132,12 @@ public PipelineBuilder thenProcess( */ public PipelineBuilder thenProcessInParallel( final String stageName, final Function processor, final int numberOfThreads) { - final Pipe newPipeEnd = new Pipe<>(bufferSize); + final Pipe newPipeEnd = new Pipe<>(bufferSize, outputCounter.labels(stageName)); final OutputPipe outputPipe = new SharedOutputPipe<>(newPipeEnd, numberOfThreads); final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { final Runnable processStage = - new ProcessingStage<>( - pipeEnd, outputPipe, new MapProcessor<>(processor), outputCounter.labels(stageName)); + new ProcessingStage<>(pipeEnd, outputPipe, new MapProcessor<>(processor)); newStages.add(processStage); } return new PipelineBuilder<>( @@ -161,7 +155,7 @@ public PipelineBuilder thenProcessInParallel( * *

Note: The order of items is not preserved. * - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param processor the processing to apply to each item. * @param maxConcurrency the maximum number of items being processed concurrently. * @param the output type for this processing step. @@ -182,7 +176,7 @@ public PipelineBuilder thenProcessAsync( * *

The output buffer size is reduced to bufferSize / maximumBatchSize + 1. * - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param maximumBatchSize the maximum number of items to include in a batch. * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ @@ -200,7 +194,7 @@ public PipelineBuilder> inBatches(final String stageName, final int maxi * *

thenFlatMap(List::stream, newBufferSize)
* - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param mapper the function to process each item with. * @param newBufferSize the output buffer size to use from this stage onwards. * @param the type of items to be output from this stage. @@ -214,7 +208,7 @@ public PipelineBuilder thenFlatMap( /** * End the pipeline with a {@link Consumer} that is the last stage of the pipeline. * - * @param stageName the name of this stage. Used as the label for the task count metric. + * @param stageName the name of this stage. Used as the label for the output count metric. * @param completer the {@link Consumer} that accepts the final output of the pipeline. * @return the constructed pipeline ready to execute. */ @@ -232,9 +226,8 @@ private PipelineBuilder addStage(final Processor processor, final S private PipelineBuilder addStage( final Processor processor, final int newBufferSize, final String stageName) { - final Pipe outputPipe = new Pipe<>(newBufferSize); - final Runnable processStage = - new ProcessingStage<>(pipeEnd, outputPipe, processor, outputCounter.labels(stageName)); + final Pipe outputPipe = new Pipe<>(newBufferSize, outputCounter.labels(stageName)); + final Runnable processStage = new ProcessingStage<>(pipeEnd, outputPipe, processor); return addStage(processStage, outputPipe); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java index 0bc199ef16..614f04859b 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java @@ -12,34 +12,27 @@ */ package tech.pegasys.pantheon.services.pipeline; -import tech.pegasys.pantheon.metrics.Counter; - class ProcessingStage implements Runnable { private final InputPipe inputPipe; private final OutputPipe outputPipe; private final Processor processor; - private final Counter outputCounter; public ProcessingStage( final InputPipe inputPipe, final OutputPipe outputPipe, - final Processor processor, - final Counter outputCounter) { + final Processor processor) { this.inputPipe = inputPipe; this.outputPipe = outputPipe; this.processor = processor; - this.outputCounter = outputCounter; } @Override public void run() { while (inputPipe.hasMore()) { - final int outputCount = processor.processNextInput(inputPipe, outputPipe); - outputCounter.inc(outputCount); + processor.processNextInput(inputPipe, outputPipe); } - final int outputCount = processor.finalize(outputPipe); - outputCounter.inc(outputCount); + processor.finalize(outputPipe); outputPipe.close(); } } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java index 56492c5e99..729f4544e4 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java @@ -13,9 +13,7 @@ package tech.pegasys.pantheon.services.pipeline; interface Processor { - int processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); + void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); - default int finalize(final OutputPipe outputPipe) { - return 0; - } + default void finalize(final OutputPipe outputPipe) {} } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java index 5c8d07c2d1..ef1d4d666a 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java @@ -16,6 +16,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyZeroInteractions; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import java.util.List; @@ -23,8 +24,8 @@ public class BatchingProcessorTest { - private final Pipe inputPipe = new Pipe<>(10); - private final Pipe> outputPipe = new Pipe<>(10); + private final Pipe inputPipe = new Pipe<>(10, NO_OP_COUNTER); + private final Pipe> outputPipe = new Pipe<>(10, NO_OP_COUNTER); private final BatchingProcessor stage = new BatchingProcessor<>(3); @Test @@ -48,18 +49,6 @@ public void shouldCreateBatches() { assertThat(outputPipe.poll()).isNull(); } - @Test - public void shouldCountABatchAsOneNotAsThenNumberOfItemsInTheBatch() { - for (int i = 1; i <= 8; i++) { - inputPipe.put(i); - } - inputPipe.close(); - - assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); - assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); - assertThat(stage.processNextInput(inputPipe, outputPipe)).isEqualTo(1); - } - @Test public void shouldNotOutputItemWhenInputIsClosed() { @SuppressWarnings("unchecked") diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java index 800a87479a..ddf9dcbe81 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/CompleterStageTest.java @@ -16,6 +16,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import tech.pegasys.pantheon.metrics.Counter; @@ -26,7 +27,7 @@ public class CompleterStageTest { - private final Pipe pipe = new Pipe<>(10); + private final Pipe pipe = new Pipe<>(10, NO_OP_COUNTER); private final List output = new ArrayList<>(); private final Counter outputCounter = mock(Counter.class); private final CompleterStage stage = diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java index 8f89b748ba..127f2ae30d 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessorTest.java @@ -17,6 +17,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import java.util.function.Function; import java.util.stream.Stream; @@ -25,8 +26,8 @@ public class FlatMapProcessorTest { - private final Pipe input = new Pipe<>(10); - private final Pipe output = new Pipe<>(10); + private final Pipe input = new Pipe<>(10, NO_OP_COUNTER); + private final Pipe output = new Pipe<>(10, NO_OP_COUNTER); @SuppressWarnings("unchecked") private final Function> mapper = mock(Function.class); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java index 6cb5a1b718..202ad5d088 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/IteratorSourceStageTest.java @@ -13,13 +13,14 @@ package tech.pegasys.pantheon.services.pipeline; import static org.assertj.core.api.Assertions.assertThat; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import com.google.common.collect.Iterators; import org.junit.Test; public class IteratorSourceStageTest { - private final Pipe output = new Pipe<>(10); + private final Pipe output = new Pipe<>(10, NO_OP_COUNTER); private final IteratorSourceStage stage = new IteratorSourceStage<>(Iterators.forArray("a", "b", "c", "d"), output); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java index 814219012f..3ccb84aef4 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/MapProcessorTest.java @@ -17,6 +17,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import java.util.function.Function; @@ -24,8 +25,8 @@ public class MapProcessorTest { - private final Pipe input = new Pipe<>(10); - private final Pipe output = new Pipe<>(10); + private final Pipe input = new Pipe<>(10, NO_OP_COUNTER); + private final Pipe output = new Pipe<>(10, NO_OP_COUNTER); @SuppressWarnings("unchecked") private final Function processor = mock(Function.class); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java index 82ae5ec554..bb163dd3fa 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipeTest.java @@ -13,11 +13,17 @@ package tech.pegasys.pantheon.services.pipeline; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import tech.pegasys.pantheon.metrics.Counter; import org.junit.Test; public class PipeTest { - private final Pipe pipe = new Pipe<>(5); + private final Counter itemCounter = mock(Counter.class); + private final Pipe pipe = new Pipe<>(5, itemCounter); @Test public void shouldNotHaveMoreWhenEmptyAndClosed() { @@ -66,4 +72,12 @@ public void shouldNotBeOpenAfterAbort() { pipe.abort(); assertThat(pipe.isOpen()).isFalse(); } + + @Test + public void shouldIncrementCounterWhenItemAddedToPipe() { + pipe.put("A"); + verify(itemCounter).inc(); + pipe.put("B"); + verify(itemCounter, times(2)).inc(); + } } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 6ac6edf888..7c8c1ccb07 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -20,8 +20,11 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; import static org.awaitility.Awaitility.waitAtMost; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_LABELLED_COUNTER; import tech.pegasys.pantheon.metrics.Counter; +import tech.pegasys.pantheon.metrics.LabelledMetric; import java.util.ArrayList; import java.util.Collection; @@ -73,7 +76,8 @@ public void afterClass() throws Exception { public void shouldPipeTasksFromSupplierToCompleter() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10).andFinishWith("end", output::add); + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) + .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); result.get(10, SECONDS); assertThat(output).containsExactly(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); @@ -83,7 +87,7 @@ public void shouldPipeTasksFromSupplierToCompleter() throws Exception { public void shouldPassInputThroughIntermediateStage() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenProcess("toString", Object::toString) .andFinishWith("end", output::add); @@ -96,11 +100,11 @@ public void shouldPassInputThroughIntermediateStage() throws Exception { @Test public void shouldCombineIntoBatches() throws Exception { - final Pipe input = new Pipe<>(20); + final Pipe input = new Pipe<>(20, NO_OP_COUNTER); tasks.forEachRemaining(input::put); final BlockingQueue> output = new ArrayBlockingQueue<>(10); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(input) + PipelineBuilder.createPipelineFrom(input, NO_OP_LABELLED_COUNTER) .inBatches("batch", 6) .andFinishWith("end", output::offer); @@ -128,7 +132,7 @@ public void shouldCombineIntoBatches() throws Exception { public void shouldProcessAsync() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenProcessAsync("toString", value -> completedFuture(Integer.toString(value)), 3) .andFinishWith("end", output::add); final CompletableFuture result = pipeline.start(executorService); @@ -143,7 +147,8 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { final List output = new ArrayList<>(); final List> futures = new CopyOnWriteArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(asList(1, 2, 3, 4, 5, 6, 7).iterator(), 10) + PipelineBuilder.createPipelineFrom( + "input", asList(1, 2, 3, 4, 5, 6, 7).iterator(), 10, NO_OP_LABELLED_COUNTER) .thenProcessAsync( "createFuture", value -> { @@ -180,7 +185,7 @@ public void shouldLimitInFlightProcessesWhenProcessingAsync() throws Exception { public void shouldFlatMapItems() throws Exception { final List output = new ArrayList<>(); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenFlatMap("flatMap", input -> Stream.of(input, input * 2), 20) .andFinishWith("end", output::add); @@ -197,7 +202,7 @@ public void shouldProcessInParallel() throws Exception { final List output = synchronizedList(new ArrayList<>()); final CountDownLatch latch = new CountDownLatch(1); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenProcessInParallel( "stageName", value -> { @@ -234,7 +239,7 @@ public void shouldAbortPipeline() throws Exception { final List output = synchronizedList(new ArrayList<>()); final CountDownLatch startedProcessingValueSix = new CountDownLatch(1); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenProcess( "stageName", value -> { @@ -267,7 +272,7 @@ public void shouldAbortPipeline() throws Exception { public void shouldAbortPipelineWhenProcessorThrowsException() { final RuntimeException expectedError = new RuntimeException("Oops"); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) .thenProcess( "stageName", (Function) @@ -288,10 +293,10 @@ public void shouldAbortPipelineWhenProcessorThrowsException() { @Test public void shouldTrackTaskCountMetric() throws Exception { final Map counters = new ConcurrentHashMap<>(); + final LabelledMetric labelledCounter = + labels -> counters.computeIfAbsent(labels[0], label -> new SimpleCounter()); final Pipeline pipeline = - PipelineBuilder.createPipelineFrom(tasks, 10) - .trackOutputCount( - labels -> counters.computeIfAbsent(labels[0], label -> new SimpleCounter())) + PipelineBuilder.createPipelineFrom("input", tasks, 10, labelledCounter) .thenProcess("map", Function.identity()) .thenProcessInParallel("parallel", Function.identity(), 3) .thenProcessAsync("async", CompletableFuture::completedFuture, 3) @@ -301,7 +306,9 @@ public void shouldTrackTaskCountMetric() throws Exception { pipeline.start(executorService).get(10, SECONDS); - assertThat(counters).containsOnlyKeys("map", "parallel", "async", "batch", "flatMap", "finish"); + assertThat(counters) + .containsOnlyKeys("input", "map", "parallel", "async", "batch", "flatMap", "finish"); + assertThat(counters.get("input").count).hasValue(15); assertThat(counters.get("map").count).hasValue(15); assertThat(counters.get("parallel").count).hasValue(15); assertThat(counters.get("async").count).hasValue(15); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java index cd5a7b7d3f..01e1d838e6 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/ProcessingStageTest.java @@ -14,12 +14,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; - -import tech.pegasys.pantheon.metrics.Counter; +import static tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem.NO_OP_COUNTER; import java.util.Locale; @@ -32,15 +30,14 @@ @RunWith(MockitoJUnitRunner.class) public class ProcessingStageTest { - private final Pipe inputPipe = new Pipe<>(10); - private final Pipe outputPipe = new Pipe<>(10); + private final Pipe inputPipe = new Pipe<>(10, NO_OP_COUNTER); + private final Pipe outputPipe = new Pipe<>(10, NO_OP_COUNTER); @Mock private Processor singleStep; - private final Counter outputCounter = mock(Counter.class); private ProcessingStage stage; @Before public void setUp() { - stage = new ProcessingStage<>(inputPipe, outputPipe, singleStep, outputCounter); + stage = new ProcessingStage<>(inputPipe, outputPipe, singleStep); doAnswer( invocation -> { outputPipe.put(inputPipe.get().toLowerCase(Locale.UK)); @@ -65,7 +62,6 @@ public void shouldCallSingleStepStageForEachInput() { assertThat(outputPipe.poll()).isNull(); verify(singleStep, times(3)).processNextInput(inputPipe, outputPipe); - verify(outputCounter, times(3)).inc(1); } @Test From 4cf7cca79b2195ca719a4f7b97ea275835acee0f Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 12 Mar 2019 12:45:40 +1000 Subject: [PATCH 36/39] Abort the pipeline when the returned future is cancelled. --- .../pantheon/services/pipeline/Pipeline.java | 13 +++++-- .../pipeline/PipelineBuilderTest.java | 36 +++++++++++++++++++ 2 files changed, 47 insertions(+), 2 deletions(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index 61c508ba40..e84a925045 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -14,6 +14,8 @@ import static java.util.stream.Collectors.toList; +import tech.pegasys.pantheon.util.ExceptionUtils; + import java.util.Collection; import java.util.List; import java.util.concurrent.CancellationException; @@ -42,7 +44,7 @@ public class Pipeline { */ private final AtomicBoolean completing = new AtomicBoolean(false); - private final CompletableFuture overallFuture = new CompletableFuture<>(); + private final CompletableFuture overallFuture = new CompletableFuture<>(); private volatile List> futures; public Pipeline( @@ -64,7 +66,7 @@ public Pipeline( * @return a future that will be completed when the pipeline completes. If the pipeline fails or * is aborted the returned future will be completed exceptionally. */ - public synchronized CompletableFuture start(final ExecutorService executorService) { + public synchronized CompletableFuture start(final ExecutorService executorService) { if (!started.compareAndSet(false, true)) { return overallFuture; } @@ -84,6 +86,13 @@ public synchronized CompletableFuture start(final ExecutorService executorSer } } }); + overallFuture.exceptionally( + error -> { + if (ExceptionUtils.rootCause(error) instanceof CancellationException) { + abort(); + } + return null; + }); return overallFuture; } diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java index 7c8c1ccb07..c01ab867dd 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilderTest.java @@ -268,6 +268,42 @@ public void shouldAbortPipeline() throws Exception { waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(processorInterrupted).isTrue()); } + @Test + public void shouldAbortPipelineWhenFutureIsCancelled() throws Exception { + final int allowProcessingUpTo = 5; + final AtomicBoolean processorInterrupted = new AtomicBoolean(false); + final List output = synchronizedList(new ArrayList<>()); + final CountDownLatch startedProcessingValueSix = new CountDownLatch(1); + final Pipeline pipeline = + PipelineBuilder.createPipelineFrom("input", tasks, 10, NO_OP_LABELLED_COUNTER) + .thenProcess( + "stageName", + value -> { + if (value > allowProcessingUpTo) { + try { + startedProcessingValueSix.countDown(); + Thread.sleep(TimeUnit.MINUTES.toNanos(2)); + } catch (final InterruptedException e) { + processorInterrupted.set(true); + } + } + return value; + }) + .andFinishWith("end", output::add); + + final CompletableFuture result = pipeline.start(executorService); + + startedProcessingValueSix.await(10, SECONDS); + waitForSize(output, allowProcessingUpTo); + + result.cancel(false); + + assertThatThrownBy(() -> result.get(10, SECONDS)).isInstanceOf(CancellationException.class); + assertThat(output).containsExactly(1, 2, 3, 4, 5); + + waitAtMost(10, SECONDS).untilAsserted(() -> assertThat(processorInterrupted).isTrue()); + } + @Test public void shouldAbortPipelineWhenProcessorThrowsException() { final RuntimeException expectedError = new RuntimeException("Oops"); From f8df301e3982f500e18078761c67c9d18a86788a Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 13 Mar 2019 06:22:29 +1000 Subject: [PATCH 37/39] Rename InputPipe to ReadPipe and OutputPipe to WritePipe to avoid the confusion of which way to look at input and output. --- .../pipeline/AsyncOperationProcessor.java | 8 ++++---- .../services/pipeline/BatchingProcessor.java | 2 +- .../services/pipeline/CompleterStage.java | 4 ++-- .../services/pipeline/FlatMapProcessor.java | 2 +- .../pantheon/services/pipeline/MapProcessor.java | 2 +- .../pegasys/pantheon/services/pipeline/Pipe.java | 4 ++-- .../services/pipeline/PipelineBuilder.java | 6 +++--- .../services/pipeline/ProcessingStage.java | 8 +++----- .../pantheon/services/pipeline/Processor.java | 4 ++-- .../pipeline/{InputPipe.java => ReadPipe.java} | 16 ++++++++-------- ...haredOutputPipe.java => SharedWritePipe.java} | 14 +++++++------- .../pipeline/{OutputPipe.java => WritePipe.java} | 8 ++++---- .../services/pipeline/BatchingProcessorTest.java | 2 +- ...putPipeTest.java => SharedWritePipeTest.java} | 6 +++--- 14 files changed, 42 insertions(+), 44 deletions(-) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{InputPipe.java => ReadPipe.java} (74%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{SharedOutputPipe.java => SharedWritePipe.java} (76%) rename services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/{OutputPipe.java => WritePipe.java} (83%) rename services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/{SharedOutputPipeTest.java => SharedWritePipeTest.java} (84%) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java index 039b57be30..f33341daf2 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/AsyncOperationProcessor.java @@ -38,7 +38,7 @@ public AsyncOperationProcessor( } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public void processNextInput(final ReadPipe inputPipe, final WritePipe outputPipe) { if (inProgress.size() < maxConcurrency) { final I value = inputPipe.get(); if (value != null) { @@ -57,13 +57,13 @@ public void processNextInput(final InputPipe inputPipe, final OutputPipe o } @Override - public void finalize(final OutputPipe outputPipe) { + public void finalize(final WritePipe outputPipe) { while (!inProgress.isEmpty()) { outputNextCompletedTask(outputPipe); } } - private void outputNextCompletedTask(final OutputPipe outputPipe) { + private void outputNextCompletedTask(final WritePipe outputPipe) { try { waitForAnyFutureToComplete(); outputCompletedTasks(1, outputPipe); @@ -82,7 +82,7 @@ private void waitForAnyFutureToComplete() CompletableFuture.anyOf(inProgress.toArray(new CompletableFuture[0])).get(1, TimeUnit.SECONDS); } - private void outputCompletedTasks(final int minTasksToOutput, final OutputPipe outputPipe) { + private void outputCompletedTasks(final int minTasksToOutput, final WritePipe outputPipe) { int outputTasks = 0; for (final Iterator> i = inProgress.iterator(); i.hasNext() && (outputTasks < minTasksToOutput || outputPipe.hasRemainingCapacity()); ) { diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java index 015809919f..2d91cf4531 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessor.java @@ -23,7 +23,7 @@ public BatchingProcessor(final int maximumBatchSize) { } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe> outputPipe) { + public void processNextInput(final ReadPipe inputPipe, final WritePipe> outputPipe) { final List batch = inputPipe.getBatch(maximumBatchSize); if (!batch.isEmpty()) { outputPipe.put(batch); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java index 1fc61af67e..4d83fcc058 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/CompleterStage.java @@ -18,13 +18,13 @@ import java.util.function.Consumer; class CompleterStage implements Runnable { - private final InputPipe input; + private final ReadPipe input; private final Consumer completer; private final Counter outputCounter; private final CompletableFuture future = new CompletableFuture<>(); CompleterStage( - final InputPipe input, final Consumer completer, final Counter outputCounter) { + final ReadPipe input, final Consumer completer, final Counter outputCounter) { this.input = input; this.completer = completer; this.outputCounter = outputCounter; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java index 4790517204..b791e7ecdd 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/FlatMapProcessor.java @@ -25,7 +25,7 @@ public FlatMapProcessor(final Function> mapper) { } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public void processNextInput(final ReadPipe inputPipe, final WritePipe outputPipe) { final I value = inputPipe.get(); if (value != null) { final Iterator outputs = mapper.apply(value).iterator(); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java index aae818a4f7..720d146473 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/MapProcessor.java @@ -23,7 +23,7 @@ public MapProcessor(final Function processor) { } @Override - public void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe) { + public void processNextInput(final ReadPipe inputPipe, final WritePipe outputPipe) { final I value = inputPipe.get(); if (value != null) { outputPipe.put(processor.apply(value)); diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index ccc35dc3c4..705b968141 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -31,12 +31,12 @@ * the pipeline aborted. * *

In most cases a Pipe is used through one of two narrower interfaces it supports {@link - * InputPipe} and {@link OutputPipe}. These are designed to expose only the operations relevant to + * ReadPipe} and {@link WritePipe}. These are designed to expose only the operations relevant to * objects either reading from or publishing to the pipe respectively. * * @param the type of item that flows through the pipe. */ -public class Pipe implements InputPipe, OutputPipe { +public class Pipe implements ReadPipe, WritePipe { private static final Logger LOG = LogManager.getLogger(); private final BlockingQueue queue; private final int capacity; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index d2acb2b786..2236716112 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -43,7 +43,7 @@ public class PipelineBuilder { private final Pipe inputPipe; private final Collection stages; private final Collection> pipes; - private final InputPipe pipeEnd; + private final ReadPipe pipeEnd; private final int bufferSize; private final LabelledMetric outputCounter; @@ -51,7 +51,7 @@ public PipelineBuilder( final Pipe inputPipe, final Collection stages, final Collection> pipes, - final InputPipe pipeEnd, + final ReadPipe pipeEnd, final int bufferSize, final LabelledMetric outputCounter) { this.outputCounter = outputCounter; @@ -133,7 +133,7 @@ public PipelineBuilder thenProcess( public PipelineBuilder thenProcessInParallel( final String stageName, final Function processor, final int numberOfThreads) { final Pipe newPipeEnd = new Pipe<>(bufferSize, outputCounter.labels(stageName)); - final OutputPipe outputPipe = new SharedOutputPipe<>(newPipeEnd, numberOfThreads); + final WritePipe outputPipe = new SharedWritePipe<>(newPipeEnd, numberOfThreads); final ArrayList newStages = new ArrayList<>(stages); for (int i = 0; i < numberOfThreads; i++) { final Runnable processStage = diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java index 614f04859b..1f73c7021f 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ProcessingStage.java @@ -14,14 +14,12 @@ class ProcessingStage implements Runnable { - private final InputPipe inputPipe; - private final OutputPipe outputPipe; + private final ReadPipe inputPipe; + private final WritePipe outputPipe; private final Processor processor; public ProcessingStage( - final InputPipe inputPipe, - final OutputPipe outputPipe, - final Processor processor) { + final ReadPipe inputPipe, final WritePipe outputPipe, final Processor processor) { this.inputPipe = inputPipe; this.outputPipe = outputPipe; this.processor = processor; diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java index 729f4544e4..ab515e3fc7 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Processor.java @@ -13,7 +13,7 @@ package tech.pegasys.pantheon.services.pipeline; interface Processor { - void processNextInput(final InputPipe inputPipe, final OutputPipe outputPipe); + void processNextInput(final ReadPipe inputPipe, final WritePipe outputPipe); - default void finalize(final OutputPipe outputPipe) {} + default void finalize(final WritePipe outputPipe) {} } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ReadPipe.java similarity index 74% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ReadPipe.java index 87242cc45d..026d95f64f 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/InputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/ReadPipe.java @@ -15,32 +15,32 @@ import java.util.List; /** - * The end of the pipe that stages read input from. + * The interface used to read items from a pipe. * * @param the type of input. */ -public interface InputPipe { +public interface ReadPipe { /** * Determines if this pipe has more items to be read. The pipe is considered to have no more items - * when it has either been aborted with {@link OutputPipe#abort()} or if all queued items have - * been read and the pipe has been closed with {@link OutputPipe#close()}. + * when it has either been aborted with {@link WritePipe#abort()} or if all queued items have been + * read and the pipe has been closed with {@link WritePipe#close()}. * * @return true if there are more items to process, otherwise false. */ boolean hasMore(); /** - * Get the next item from this pipe. This method will block until the next item is available but - * may still return null if the pipe is closed or the thread interrupted while - * waiting. + * Get and remove the next item from this pipe. This method will block until the next item is + * available but may still return null if the pipe is closed or the thread + * interrupted while waiting. * * @return the next item or null if the pipe is closed or the thread interrupted. */ T get(); /** - * Get the next item from this pipe without blocking if it is available. + * Get and remove the next item from this pipe without blocking if it is available. * * @return the next item or null if the pipe is empty. */ diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipe.java similarity index 76% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipe.java index 0553b6be7b..bdc3b2436f 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipe.java @@ -15,24 +15,24 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * A wrapper around an {@link OutputPipe} which allows multiple stages to share the same output - * pipe. Most operations simply pass through to the underlying pipe but the underlying pipe is only - * closed when all stages have signalled this pipe should close. + * A wrapper around an {@link WritePipe} which allows multiple stages to share the same write pipe. + * Most operations simply pass through to the underlying pipe but the underlying pipe is only closed + * when all stages have signalled this pipe should close. * * @param the type of item in the pipe. */ -class SharedOutputPipe implements OutputPipe { - private final OutputPipe delegate; +class SharedWritePipe implements WritePipe { + private final WritePipe delegate; private final AtomicInteger remainingClosesRequired; /** - * Creates a new SharedOutputPipe. + * Creates a new SharedWritePipe. * * @param delegate the pipe to wrap. * @param closesRequired the number of stages this output pipe will be shared with. The underlying * pipe will only be closed when {@link #close()} is called this many times. */ - public SharedOutputPipe(final OutputPipe delegate, final int closesRequired) { + public SharedWritePipe(final WritePipe delegate, final int closesRequired) { this.delegate = delegate; this.remainingClosesRequired = new AtomicInteger(closesRequired); } diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/WritePipe.java similarity index 83% rename from services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java rename to services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/WritePipe.java index 8628494780..ea07a3c609 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/OutputPipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/WritePipe.java @@ -13,11 +13,11 @@ package tech.pegasys.pantheon.services.pipeline; /** - * The end of the pipe that stages write their output to. + * The interface used to add items to a pipe. * * @param the type of output. */ -public interface OutputPipe { +public interface WritePipe { /** * Determine if this pipe is still open and accepting output. @@ -42,8 +42,8 @@ public interface OutputPipe { boolean hasRemainingCapacity(); /** - * Close this output pipe indicating that no further data will be published to it. When reading - * from the other end of this pipe {@link InputPipe#hasMore()} will continue to return true until + * Close this write pipe indicating that no further data will be published to it. When reading + * from the other end of this pipe {@link ReadPipe#hasMore()} will continue to return true until * all the already queued data has been drained. */ void close(); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java index ef1d4d666a..21fb184c8a 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/BatchingProcessorTest.java @@ -52,7 +52,7 @@ public void shouldCreateBatches() { @Test public void shouldNotOutputItemWhenInputIsClosed() { @SuppressWarnings("unchecked") - final OutputPipe> outputPipe = mock(OutputPipe.class); + final WritePipe> outputPipe = mock(WritePipe.class); inputPipe.close(); stage.processNextInput(inputPipe, outputPipe); verifyZeroInteractions(outputPipe); diff --git a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipeTest.java similarity index 84% rename from services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java rename to services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipeTest.java index cf6bdd3a8c..eebd1d4756 100644 --- a/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedOutputPipeTest.java +++ b/services/pipeline/src/test/java/tech/pegasys/pantheon/services/pipeline/SharedWritePipeTest.java @@ -18,14 +18,14 @@ import org.junit.Test; -public class SharedOutputPipeTest { +public class SharedWritePipeTest { private static final int CLOSES_REQUIRED = 3; @SuppressWarnings("unchecked") - private final OutputPipe delegate = mock(OutputPipe.class); + private final WritePipe delegate = mock(WritePipe.class); - private final SharedOutputPipe pipe = new SharedOutputPipe<>(delegate, CLOSES_REQUIRED); + private final SharedWritePipe pipe = new SharedWritePipe<>(delegate, CLOSES_REQUIRED); @Test public void shouldOnlyCloseDelegatePipeWhenCloseCalledSpecifiedNumberOfTimes() { From 9d0be662d5de3553148027165cae57b58b89ba1d Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 13 Mar 2019 06:24:41 +1000 Subject: [PATCH 38/39] Make Pipeline constructor package private. --- .../java/tech/pegasys/pantheon/services/pipeline/Pipeline.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java index e84a925045..c4cc851474 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipeline.java @@ -47,7 +47,7 @@ public class Pipeline { private final CompletableFuture overallFuture = new CompletableFuture<>(); private volatile List> futures; - public Pipeline( + Pipeline( final Pipe inputPipe, final Collection stages, final Collection> pipes, From 5b336aadacba1fa4bc4e56b3d6ea8c4afb03bd5e Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 13 Mar 2019 06:28:05 +1000 Subject: [PATCH 39/39] Check batch size is greater than 0. Pipe's don't have remaining capacity when they're closed. --- .../main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java | 2 +- .../pegasys/pantheon/services/pipeline/PipelineBuilder.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java index 705b968141..44d94d9769 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/Pipe.java @@ -66,7 +66,7 @@ public int getCapacity() { @Override public boolean hasRemainingCapacity() { - return queue.remainingCapacity() > 0; + return queue.remainingCapacity() > 0 && isOpen(); } @Override diff --git a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java index 2236716112..666378b103 100644 --- a/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java +++ b/services/pipeline/src/main/java/tech/pegasys/pantheon/services/pipeline/PipelineBuilder.java @@ -181,6 +181,7 @@ public PipelineBuilder thenProcessAsync( * @return a {@link PipelineBuilder} ready to extend the pipeline with additional stages. */ public PipelineBuilder> inBatches(final String stageName, final int maximumBatchSize) { + checkArgument(maximumBatchSize > 0, "Maximum batch size must be greater than 0"); return addStage( new BatchingProcessor<>(maximumBatchSize), bufferSize / maximumBatchSize + 1, stageName); }