Skip to content

Commit

Permalink
[BEAM-2915] Add support for handling bag user state to the java-fn-ex…
Browse files Browse the repository at this point in the history
…ecution library to support runner integration.

Note that we specifically use an optimization which significantly simplifies the logical stream handling during get/append since we don't need
to encode/decode elements to actually find the element boundaries. We just store and replay the chunks the user gave us significantly
reducing the amount of pipeline manipulation the needs to happen.
  • Loading branch information
lukecwik committed Jul 10, 2018
1 parent 109f03c commit 4d62b35
Show file tree
Hide file tree
Showing 22 changed files with 860 additions and 22 deletions.
2 changes: 1 addition & 1 deletion .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@
**/.gogradle/**/*
**/gogradle.lock
**/build/**/*
**/vendor/**/*
sdks/go/**/vendor/**/*
**/.gradletasknamecache

# Ignore files generated by the Maven build process.
Expand Down
16 changes: 15 additions & 1 deletion model/pipeline/src/main/proto/beam_runner_api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ message ExecutableStagePayload {
// PTransform the ExecutableStagePayload is the payload of.
string input = 2;

// The side inputs required for this executable stage. Each Side Input of each PTransform within
// The side inputs required for this executable stage. Each side input of each PTransform within
// this ExecutableStagePayload must be represented within this field.
repeated SideInputId side_inputs = 3;

Expand All @@ -355,6 +355,10 @@ message ExecutableStagePayload {
// in transforms, and the closure of all of the components they recognize.
Components components = 6;

// The user states required for this executable stage. Each user state of each PTransform within
// this ExecutableStagePayload must be represented within this field.
repeated UserStateId user_states = 7;

// A reference to a side input. Side inputs are uniquely identified by PTransform id and
// local name.
message SideInputId {
Expand All @@ -364,6 +368,16 @@ message ExecutableStagePayload {
// (Required) The local name of this side input from the PTransform that references it.
string local_name = 2;
}

// A reference to user state. User states are uniquely identified by PTransform id and
// local name.
message UserStateId {
// (Required) The id of the PTransform that references this user state.
string transform_id = 1;

// (Required) The local name of this user state for the PTransform that references it.
string local_name = 2;
}
}

// The payload for the primitive ParDo transform.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.UserStateId;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
Expand Down Expand Up @@ -80,6 +81,9 @@ public interface ExecutableStage {
*/
Collection<SideInputReference> getSideInputs();

/** Returns the set of {@link PTransformNode PTransforms} that contain user state. */
Collection<UserStateReference> getUserStates();

/**
* Returns the leaf {@link PCollectionNode PCollections} of this {@link ExecutableStage}.
*
Expand Down Expand Up @@ -132,8 +136,14 @@ default PTransform toPTransform(String uniqueName) {
payload.addSideInputs(
SideInputId.newBuilder()
.setTransformId(sideInput.transform().getId())
.setLocalName(sideInput.localName())
.build());
.setLocalName(sideInput.localName()));
}

for (UserStateReference userState : getUserStates()) {
payload.addUserStates(
UserStateId.newBuilder()
.setTransformId(userState.transform().getId())
.setLocalName(userState.localName()));
}

int outputIndex = 0;
Expand Down Expand Up @@ -179,6 +189,7 @@ default PTransform toPTransform(String uniqueName) {
static ExecutableStage fromPayload(ExecutableStagePayload payload) {
Components components = payload.getComponents();
Environment environment = payload.getEnvironment();

PCollectionNode input =
PipelineNode.pCollection(
payload.getInput(), components.getPcollectionsOrThrow(payload.getInput()));
Expand All @@ -188,6 +199,12 @@ static ExecutableStage fromPayload(ExecutableStagePayload payload) {
.stream()
.map(sideInputId -> SideInputReference.fromSideInputId(sideInputId, components))
.collect(Collectors.toList());
List<UserStateReference> userStates =
payload
.getUserStatesList()
.stream()
.map(userStateId -> UserStateReference.fromUserStateId(userStateId, components))
.collect(Collectors.toList());
List<PTransformNode> transforms =
payload
.getTransformsList()
Expand All @@ -201,6 +218,6 @@ static ExecutableStage fromPayload(ExecutableStagePayload payload) {
.map(id -> PipelineNode.pCollection(id, components.getPcollectionsOrThrow(id)))
.collect(Collectors.toList());
return ImmutableExecutableStage.of(
components, environment, input, sideInputs, transforms, outputs);
components, environment, input, sideInputs, userStates, transforms, outputs);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,10 @@ private static boolean parDoCompatibility(
// side inputs can be fused with other transforms in the same environment which are not
// upstream of any of the side inputs.
return pipeline.getSideInputs(parDo).isEmpty()
// Since we lack the ability to mark upstream transforms as key preserving, we
// purposefully break fusion here to provide runners the opportunity to insert a
// grouping operation
&& pipeline.getUserStates(parDo).isEmpty()
&& compatibleEnvironments(parDo, other, pipeline);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,9 @@
*
* <p>A {@link PCollectionNode} is fused into a stage if all of its consumers can be fused into the
* stage. A consumer can be fused into a stage if it is executed within the environment of that
* {@link ExecutableStage}, and receives only per-element inputs. PTransforms which consume side
* inputs are always at the root of a stage.
* {@link ExecutableStage}, and receives only per-element inputs. To simplify integration for
* runners, this fuser specifically does not fuse PTransforms which consume side inputs or have user
* state, always making them the root of {@link ExecutableStage}.
*
* <p>A {@link PCollectionNode} with consumers that execute in an environment other than a stage is
* materialized, and its consumers execute in independent stages.
Expand Down Expand Up @@ -80,13 +81,15 @@ public static ExecutableStage forGrpcPortRead(
fusedTransforms.addAll(initialNodes);

Set<SideInputReference> sideInputs = new LinkedHashSet<>();
Set<UserStateReference> userStates = new LinkedHashSet<>();
Set<PCollectionNode> fusedCollections = new LinkedHashSet<>();
Set<PCollectionNode> materializedPCollections = new LinkedHashSet<>();

Queue<PCollectionNode> fusionCandidates = new ArrayDeque<>();
for (PTransformNode initialConsumer : initialNodes) {
fusionCandidates.addAll(pipeline.getOutputPCollections(initialConsumer));
sideInputs.addAll(pipeline.getSideInputs(initialConsumer));
userStates.addAll(pipeline.getUserStates(initialConsumer));
}
while (!fusionCandidates.isEmpty()) {
PCollectionNode candidate = fusionCandidates.poll();
Expand Down Expand Up @@ -130,6 +133,7 @@ public static ExecutableStage forGrpcPortRead(
environment,
inputPCollection,
sideInputs,
userStates,
fusedTransforms.build(),
materializedPCollections);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ public static ImmutableExecutableStage ofFullComponents(
Environment environment,
PCollectionNode input,
Collection<SideInputReference> sideInputs,
Collection<UserStateReference> userStates,
Collection<PTransformNode> transforms,
Collection<PCollectionNode> outputs) {
Components prunedComponents =
Expand All @@ -46,21 +47,23 @@ public static ImmutableExecutableStage ofFullComponents(
.stream()
.collect(Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform)))
.build();
return of(prunedComponents, environment, input, sideInputs, transforms, outputs);
return of(prunedComponents, environment, input, sideInputs, userStates, transforms, outputs);
}

public static ImmutableExecutableStage of(
Components components,
Environment environment,
PCollectionNode input,
Collection<SideInputReference> sideInputs,
Collection<UserStateReference> userStates,
Collection<PTransformNode> transforms,
Collection<PCollectionNode> outputs) {
return new AutoValue_ImmutableExecutableStage(
components,
environment,
input,
ImmutableSet.copyOf(sideInputs),
ImmutableSet.copyOf(userStates),
ImmutableSet.copyOf(transforms),
ImmutableSet.copyOf(outputs));
}
Expand All @@ -78,6 +81,9 @@ public static ImmutableExecutableStage of(
@Override
public abstract Collection<SideInputReference> getSideInputs();

@Override
public abstract Collection<UserStateReference> getUserStates();

@Override
public abstract Collection<PTransformNode> getTransforms();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -309,6 +309,7 @@ private static ExecutableStage deduplicateStageOutput(
stage.getEnvironment(),
stage.getInputPCollection(),
stage.getSideInputs(),
stage.getUserStates(),
updatedTransforms,
updatedOutputs);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.graph.MutableNetwork;
import com.google.common.graph.Network;
import com.google.common.graph.NetworkBuilder;
Expand Down Expand Up @@ -298,6 +299,31 @@ public Collection<SideInputReference> getSideInputs(PTransformNode transform) {
.collect(Collectors.toSet());
}

public Collection<UserStateReference> getUserStates(PTransformNode transform) {
return getLocalUserStateNames(transform.getTransform())
.stream()
.map(
localName -> {
String transformId = transform.getId();
PTransform transformProto = components.getTransformsOrThrow(transformId);
// Get the main input PCollection id.
String collectionId =
transform
.getTransform()
.getInputsOrThrow(
Iterables.getOnlyElement(
Sets.difference(
transform.getTransform().getInputsMap().keySet(),
getLocalSideInputNames(transformProto))));
PCollection collection = components.getPcollectionsOrThrow(collectionId);
return UserStateReference.of(
PipelineNode.pTransform(transformId, transformProto),
localName,
PipelineNode.pCollection(collectionId, collection));
})
.collect(Collectors.toSet());
}

private Set<String> getLocalSideInputNames(PTransform transform) {
if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
try {
Expand All @@ -310,6 +336,18 @@ private Set<String> getLocalSideInputNames(PTransform transform) {
}
}

private Set<String> getLocalUserStateNames(PTransform transform) {
if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
try {
return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getStateSpecsMap().keySet();
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException(e);
}
} else {
return Collections.emptySet();
}
}

public Optional<Environment> getEnvironment(PTransformNode parDo) {
return Environments.getEnvironment(parDo.getId(), components);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.graph;

import com.google.auto.value.AutoValue;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import java.util.Collections;
import java.util.Set;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.UserStateId;
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
import org.apache.beam.vendor.protobuf.v3.com.google.protobuf.InvalidProtocolBufferException;

/**
* A reference to user state. This includes the PTransform that references the user state as well as
* the local name. Both are necessary in order to fully resolve user state.
*/
@AutoValue
public abstract class UserStateReference {

/** Create a user state reference. */
public static UserStateReference of(
PTransformNode transform, String localName, PCollectionNode collection) {
return new AutoValue_UserStateReference(transform, localName, collection);
}

/** Create a user state reference from a UserStateId proto and components. */
public static UserStateReference fromUserStateId(
UserStateId userStateId, RunnerApi.Components components) {
String transformId = userStateId.getTransformId();
String localName = userStateId.getLocalName();

PTransform transform = components.getTransformsOrThrow(transformId);

Set<String> sideInputNames = Collections.emptySet();
if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
try {
sideInputNames =
ParDoPayload.parseFrom(transform.getSpec().getPayload()).getSideInputsMap().keySet();
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException(e);
}
}

// Get the main input PCollection id.
String collectionId =
transform.getInputsOrThrow(
Iterables.getOnlyElement(
Sets.difference(transform.getInputsMap().keySet(), sideInputNames)));
PCollection collection = components.getPcollectionsOrThrow(collectionId);
return UserStateReference.of(
PipelineNode.pTransform(transformId, transform),
localName,
PipelineNode.pCollection(collectionId, collection));
}

/** The id of the PTransform that uses this user state. */
public abstract PTransformNode transform();
/** The local name the referencing PTransform uses to refer to this user state. */
public abstract String localName();
/** The PCollection that represents the input to the PTransform. */
public abstract PCollectionNode collection();
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
Expand All @@ -62,6 +63,7 @@ public void testRoundTripToFromTransform() throws Exception {
ParDoPayload.newBuilder()
.setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("foo"))
.putSideInputs("side_input", SideInput.getDefaultInstance())
.putStateSpecs("user_state", StateSpec.getDefaultInstance())
.build()
.toByteString()))
.build();
Expand All @@ -82,12 +84,16 @@ public void testRoundTripToFromTransform() throws Exception {
SideInputReference sideInputRef =
SideInputReference.of(
transformNode, "side_input", PipelineNode.pCollection("sideInput.in", sideInput));
UserStateReference userStateRef =
UserStateReference.of(
transformNode, "user_state", PipelineNode.pCollection("input.out", input));
ImmutableExecutableStage stage =
ImmutableExecutableStage.of(
components,
env,
PipelineNode.pCollection("input.out", input),
Collections.singleton(sideInputRef),
Collections.singleton(userStateRef),
Collections.singleton(PipelineNode.pTransform("pt", pt)),
Collections.singleton(PipelineNode.pCollection("output.out", output)));

Expand Down
Loading

0 comments on commit 4d62b35

Please sign in to comment.