Skip to content
This repository was archived by the owner on Nov 11, 2022. It is now read-only.
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Copyright (C) 2016 Google Inc.
*
* 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 com.google.cloud.dataflow.sdk.runners.inprocess;

import com.google.auto.value.AutoValue;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;

/**
* A {@link InProcessTransformResult} that has been committed.
*/
@AutoValue
abstract class CommittedResult {
/**
* Returns the {@link AppliedPTransform} that produced this result.
*/
public abstract AppliedPTransform<?, ?, ?> getTransform();

/**
* Returns the outputs produced by the transform.
*/
public abstract Iterable<? extends CommittedBundle<?>> getOutputs();

public static CommittedResult create(
InProcessTransformResult original, Iterable<? extends CommittedBundle<?>> outputs) {
return new AutoValue_CommittedResult(original.getTransform(),
outputs);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ interface CompletionCallback {
/**
* Handle a successful result, returning the committed outputs of the result.
*/
Iterable<? extends CommittedBundle<?>> handleResult(
CommittedResult handleResult(
CommittedBundle<?> inputBundle, InProcessTransformResult result);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,14 +213,14 @@ public void awaitCompletion() throws Throwable {
*/
private class DefaultCompletionCallback implements CompletionCallback {
@Override
public Iterable<? extends CommittedBundle<?>> handleResult(
public CommittedResult handleResult(
CommittedBundle<?> inputBundle, InProcessTransformResult result) {
Iterable<? extends CommittedBundle<?>> resultBundles =
CommittedResult committedResult =
evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
for (CommittedBundle<?> outputBundle : resultBundles) {
for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
}
return resultBundles;
return committedResult;
}

@Override
Expand All @@ -243,14 +243,14 @@ private TimerCompletionCallback(Iterable<TimerData> timers) {
}

@Override
public Iterable<? extends CommittedBundle<?>> handleResult(
public CommittedResult handleResult(
CommittedBundle<?> inputBundle, InProcessTransformResult result) {
Iterable<? extends CommittedBundle<?>> resultBundles =
CommittedResult committedResult =
evaluationContext.handleResult(inputBundle, timers, result);
for (CommittedBundle<?> outputBundle : resultBundles) {
for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
}
return resultBundles;
return committedResult;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -797,18 +797,19 @@ public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
* </pre>.
*
* @param completed the input that has completed
* @param transform the transform that has completed processing the input
* @param outputs the bundles the transform has output
* @param timerUpdate the timers that were added, removed, and completed as part of producing
* this update
* @param result the result that was produced by processing the input
* @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
* is no hold
*/
public void updateWatermarks(
@Nullable CommittedBundle<?> completed,
AppliedPTransform<?, ?, ?> transform,
TimerUpdate timerUpdate,
Iterable<? extends CommittedBundle<?>> outputs,
CommittedResult result,
@Nullable Instant earliestHold) {
updatePending(completed, transform, timerUpdate, outputs);
AppliedPTransform<?, ?, ?> transform = result.getTransform();
updatePending(completed, timerUpdate, result);
TransformWatermarks transformWms = transformToWatermarks.get(transform);
transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
refreshWatermarks(transform);
Expand Down Expand Up @@ -843,15 +844,14 @@ private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
*/
private void updatePending(
CommittedBundle<?> input,
AppliedPTransform<?, ?, ?> transform,
TimerUpdate timerUpdate,
Iterable<? extends CommittedBundle<?>> outputs) {
TransformWatermarks completedTransform = transformToWatermarks.get(transform);
CommittedResult result) {
TransformWatermarks completedTransform = transformToWatermarks.get(result.getTransform());

// Newly pending elements must be added before completed elements are removed, as the two
// do not share a Mutex within this call and thus can be interleaved with external calls to
// refresh.
for (CommittedBundle<?> bundle : outputs) {
for (CommittedBundle<?> bundle : result.getOutputs()) {
for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
TransformWatermarks watermarks = transformToWatermarks.get(consumer);
watermarks.addPending(bundle);
Expand All @@ -862,7 +862,6 @@ private void updatePending(
if (input != null) {
completedTransform.removePending(input);
}

}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,18 +142,18 @@ private InProcessEvaluationContext(
* @param result the result of evaluating the input bundle
* @return the committed bundles contained within the handled {@code result}
*/
public synchronized Iterable<? extends CommittedBundle<?>> handleResult(
public synchronized CommittedResult handleResult(
@Nullable CommittedBundle<?> completedBundle,
Iterable<TimerData> completedTimers,
InProcessTransformResult result) {
Iterable<? extends CommittedBundle<?>> committedBundles =
commitBundles(result.getOutputBundles());
// Update watermarks and timers
CommittedResult committedResult = CommittedResult.create(result, committedBundles);
watermarkManager.updateWatermarks(
completedBundle,
result.getTransform(),
result.getTimerUpdate().withCompletedTimers(completedTimers),
committedBundles,
committedResult,
result.getWatermarkHold());
fireAllAvailableCallbacks();
// Update counters
Expand All @@ -173,7 +173,7 @@ public synchronized Iterable<? extends CommittedBundle<?>> handleResult(
applicationStateInternals.remove(stepAndKey);
}
}
return committedBundles;
return committedResult;
}

private Iterable<? extends CommittedBundle<?>> commitBundles(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,9 +157,9 @@ private InProcessTransformResult finishBundle(
TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
throws Exception {
InProcessTransformResult result = evaluator.finishBundle();
Iterable<? extends CommittedBundle<?>> outputs = onComplete.handleResult(inputBundle, result);
CommittedResult outputs = onComplete.handleResult(inputBundle, result);
for (ModelEnforcement<T> enforcement : enforcements) {
enforcement.afterFinish(inputBundle, result, outputs);
enforcement.afterFinish(inputBundle, result, outputs.getOutputs());
}
return result;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* Copyright (C) 2016 Google Inc.
*
* 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 com.google.cloud.dataflow.sdk.runners.inprocess;

import static org.junit.Assert.assertThat;

import com.google.cloud.dataflow.sdk.testing.TestPipeline;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
import com.google.cloud.dataflow.sdk.values.PBegin;
import com.google.cloud.dataflow.sdk.values.PCollection;
import com.google.cloud.dataflow.sdk.values.PDone;
import com.google.common.collect.ImmutableList;

import org.hamcrest.Matchers;
import org.joda.time.Instant;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

import java.io.Serializable;
import java.util.Collections;
import java.util.List;

/**
* Tests for {@link CommittedResult}.
*/
@RunWith(JUnit4.class)
public class CommittedResultTest implements Serializable {
private transient TestPipeline p = TestPipeline.create();
private transient AppliedPTransform<?, ?, ?> transform =
AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
});
private transient BundleFactory bundleFactory = InProcessBundleFactory.create();

@Test
public void getTransformExtractsFromResult() {
CommittedResult result =
CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());

assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
}

@Test
public void getOutputsEqualInput() {
List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
WindowingStrategy.globalDefault(),
PCollection.IsBounded.BOUNDED)).commit(Instant.now()),
bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
WindowingStrategy.globalDefault(),
PCollection.IsBounded.UNBOUNDED)).commit(Instant.now()));
CommittedResult result =
CommittedResult.create(StepTransformResult.withoutHold(transform).build(), outputs);

assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
}
}
Loading