Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-22] Support Unbounded PCollections in same-process execution #3

Closed
wants to merge 3 commits into from
Closed
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
Expand Up @@ -18,7 +18,6 @@
import com.google.cloud.dataflow.sdk.io.Read.Bounded;
import com.google.cloud.dataflow.sdk.io.Source.Reader;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.cloud.dataflow.sdk.options.DefaultValueFactory;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;

import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;

/**
* A {@link DefaultValueFactory} that produces cached thread pools via
* {@link Executors#newCachedThreadPool()}.
*/
class CachedThreadPoolExecutorServiceFactory
implements DefaultValueFactory<ExecutorService> {
@Override
public ExecutorService create(PipelineOptions options) {
return Executors.newCachedThreadPool();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;

/**
* A callback for completing a bundle of input.
*/
interface CompletionCallback {
/**
* Handle a successful result.
*/
void handleResult(CommittedBundle<?> inputBundle, InProcessTransformResult result);

/**
* Handle a result that terminated abnormally due to the provided {@link Throwable}.
*/
void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* 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.cloud.dataflow.sdk.Pipeline;
import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
import com.google.cloud.dataflow.sdk.values.PCollectionView;
import com.google.cloud.dataflow.sdk.values.PValue;

import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;

/**
* Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
* {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
* input after the upstream transform has produced and committed output.
*/
public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
private Collection<PCollectionView<?>> views = new ArrayList<>();
private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
private Set<PValue> toFinalize = new HashSet<>();
private int numTransforms = 0;

@Override
public void enterCompositeTransform(TransformTreeNode node) {}

@Override
public void leaveCompositeTransform(TransformTreeNode node) {}

@Override
public void visitTransform(TransformTreeNode node) {
toFinalize.removeAll(node.getInputs().keySet());
AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
if (node.getInput().expand().isEmpty()) {
rootTransforms.add(appliedTransform);
} else {
for (PValue value : node.getInputs().keySet()) {
valueToConsumers.get(value).add(appliedTransform);
stepNames.put(appliedTransform, genStepName());
}
}
}

private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
@SuppressWarnings({"rawtypes", "unchecked"})
AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
return application;
}

@Override
public void visitValue(PValue value, TransformTreeNode producer) {
toFinalize.add(value);
for (PValue expandedValue : value.expand()) {
valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
if (expandedValue instanceof PCollectionView) {
views.add((PCollectionView<?>) expandedValue);
}
expandedValue.recordAsOutput(getAppliedTransform(producer));
}
value.recordAsOutput(getAppliedTransform(producer));
}

private String genStepName() {
return String.format("s%s", numTransforms++);
}

public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
return valueToConsumers;
}

public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
return stepNames;
}

public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
return rootTransforms;
}

public Collection<PCollectionView<?>> getViews() {
return views;
}

public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToCustomers() {
return valueToConsumers;
}

/**
* @return
*/
public Set<PValue> getUnfinalizedPValues() {
return toFinalize;
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
*/
package com.google.cloud.dataflow.sdk.runners.inprocess;

import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;

import java.util.Objects;
Expand Down
Loading