Skip to content

Commit

Permalink
This closes #2587
Browse files Browse the repository at this point in the history
  • Loading branch information
tgroh committed Apr 19, 2017
2 parents 714fdd2 + 418c304 commit 29e054a
Show file tree
Hide file tree
Showing 4 changed files with 205 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.runners.core.construction;

import java.util.HashSet;
import java.util.Set;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.Pipeline.PipelineVisitor;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.runners.TransformHierarchy.Node;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PValue;

/**
* Utilities for ensuring that all {@link Read} {@link PTransform PTransforms} are consumed by some
* {@link PTransform}.
*/
public class UnconsumedReads {
public static void ensureAllReadsConsumed(Pipeline pipeline) {
final Set<PCollection<?>> unconsumed = new HashSet<>();
pipeline.traverseTopologically(
new PipelineVisitor.Defaults() {
@Override
public void visitPrimitiveTransform(Node node) {
unconsumed.removeAll(node.getInputs().values());
}

@Override
public void visitValue(PValue value, Node producer) {
if (producer.getTransform() instanceof Read.Bounded
|| producer.getTransform() instanceof Read.Unbounded) {
unconsumed.add((PCollection<?>) value);
}
}
});
int i = 0;
for (PCollection<?> unconsumedPCollection : unconsumed) {
consume(unconsumedPCollection, i);
i++;
}
}

private static <T> void consume(PCollection<T> unconsumedPCollection, int uniq) {
// Multiple applications should never break due to stable unique names.
String uniqueName = "DropInputs" + (uniq == 0 ? "" : uniq);
unconsumedPCollection.apply(uniqueName, ParDo.of(new NoOpDoFn<T>()));
}

private static class NoOpDoFn<T> extends DoFn<T, T> {
@ProcessElement
public void doNothing(ProcessContext context) {}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.runners.core.construction;

import static org.junit.Assert.assertThat;

import java.util.HashSet;
import java.util.Set;
import org.apache.beam.sdk.Pipeline.PipelineVisitor;
import org.apache.beam.sdk.io.CountingSource;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.io.Read.Bounded;
import org.apache.beam.sdk.io.Read.Unbounded;
import org.apache.beam.sdk.runners.TransformHierarchy.Node;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
import org.apache.beam.sdk.values.PValue;
import org.hamcrest.Matchers;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/**
* Tests for {@link UnconsumedReads}.
*/
@RunWith(JUnit4.class)
public class UnconsumedReadsTest {
@Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);

@Test
public void matcherProducesUnconsumedValueBoundedRead() {
Bounded<Long> transform = Read.from(CountingSource.upTo(20L));
PCollection<Long> output = pipeline.apply(transform);
UnconsumedReads.ensureAllReadsConsumed(pipeline);
validateConsumed();
}

@Test
public void matcherProducesUnconsumedValueUnboundedRead() {
Unbounded<Long> transform = Read.from(CountingSource.unbounded());
PCollection<Long> output = pipeline.apply(transform);
UnconsumedReads.ensureAllReadsConsumed(pipeline);
validateConsumed();
}

@Test
public void doesNotConsumeAlreadyConsumedRead() {
Unbounded<Long> transform = Read.from(CountingSource.unbounded());
final PCollection<Long> output = pipeline.apply(transform);
final Flatten.PCollections<Long> consumer = Flatten.<Long>pCollections();
PCollectionList.of(output).apply(consumer);
UnconsumedReads.ensureAllReadsConsumed(pipeline);
pipeline.traverseTopologically(
new PipelineVisitor.Defaults() {
@Override
public void visitPrimitiveTransform(Node node) {
// The output should only be consumed by a single consumer
if (node.getInputs().values().contains(output)) {
assertThat(node.getTransform(), Matchers.<PTransform<?, ?>>is(consumer));
}
}
});
}

private void validateConsumed() {
final Set<PValue> consumedOutputs = new HashSet<PValue>();
final Set<PValue> allReadOutputs = new HashSet<PValue>();
pipeline.traverseTopologically(
new PipelineVisitor.Defaults() {
@Override
public void visitPrimitiveTransform(Node node) {
consumedOutputs.addAll(node.getInputs().values());
}

@Override
public void visitValue(PValue value, Node producer) {
if (producer.getTransform() instanceof Read.Bounded
|| producer.getTransform() instanceof Read.Unbounded) {
allReadOutputs.add(value);
}
}
});
assertThat(consumedOutputs, Matchers.hasItems(allReadOutputs.toArray(new PValue[0])));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.apache.beam.runners.core.construction.ReplacementOutputs;
import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory;
import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource;
import org.apache.beam.runners.core.construction.UnconsumedReads;
import org.apache.beam.runners.dataflow.BatchViewOverrides.BatchCombineGloballyAsSingletonViewFactory;
import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
import org.apache.beam.runners.dataflow.StreamingViewOverrides.StreamingCreatePCollectionViewFactory;
Expand Down Expand Up @@ -690,6 +691,9 @@ private static Map<String, Object> getEnvironmentVersion(DataflowPipelineOptions
@VisibleForTesting
void replaceTransforms(Pipeline pipeline) {
boolean streaming = options.isStreaming() || containsUnboundedPCollection(pipeline);
// Ensure all outputs of all reads are consumed before potentially replacing any
// Read PTransforms
UnconsumedReads.ensureAllReadsConsumed(pipeline);
pipeline.replaceAll(getOverrides(streaming));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.startsWith;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
Expand Down Expand Up @@ -57,6 +58,7 @@
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
Expand All @@ -65,11 +67,13 @@
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.io.TextIO.Read;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.runners.TransformHierarchy.Node;
import org.apache.beam.sdk.testing.ExpectedLogs;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
Expand Down Expand Up @@ -331,6 +335,26 @@ public void testTextIOWithRuntimeParameters() throws IOException {
.apply(TextIO.Write.to(options.getOutput()).withoutValidation());
}

/**
* Tests that all reads are consumed by at least one {@link PTransform}.
*/
@Test
public void testUnconsumedReads() throws IOException {
DataflowPipelineOptions dataflowOptions = buildPipelineOptions();
RuntimeTestOptions options = dataflowOptions.as(RuntimeTestOptions.class);
Pipeline p = buildDataflowPipeline(dataflowOptions);
PCollection<String> unconsumed = p.apply(Read.from(options.getInput()).withoutValidation());
DataflowRunner.fromOptions(dataflowOptions).replaceTransforms(p);
final AtomicBoolean unconsumedSeenAsInput = new AtomicBoolean();
p.traverseTopologically(new PipelineVisitor.Defaults() {
@Override
public void visitPrimitiveTransform(Node node) {
unconsumedSeenAsInput.set(true);
}
});
assertThat(unconsumedSeenAsInput.get(), is(true));
}

@Test
public void testRunReturnDifferentRequestId() throws IOException {
DataflowPipelineOptions options = buildPipelineOptions();
Expand Down

0 comments on commit 29e054a

Please sign in to comment.