[
https://issues.apache.org/jira/browse/BEAM-6747?focusedWorklogId=211920&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-211920
]
ASF GitHub Bot logged work on BEAM-6747:
----------------------------------------
Author: ASF GitHub Bot
Created on: 12/Mar/19 18:13
Start Date: 12/Mar/19 18:13
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #7954: [BEAM-6747] Adding
ExternalTransform in JavaSDK
URL: https://github.com/apache/beam/pull/7954#discussion_r264807727
##########
File path:
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.annotation.Nullable;
+import org.apache.beam.model.expansion.v1.ExpansionApi;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.ManagedChannelBuilder;
+import
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
+
+/** Cross-language external transform. */
+public class External {
+ private static final String EXPANDED_TRANSFORM_BASE_NAME = "external";
+ private static final String IMPULSE_PREFIX = "IMPULSE";
+ private static AtomicInteger namespaceCounter = new AtomicInteger(0);
+
+ private static final ExpansionServiceClientFactory DEFAULT =
+ new DefaultExpansionServiceClientFactory(
+ endPoint ->
ManagedChannelBuilder.forTarget(endPoint.getUrl()).usePlaintext().build());
+
+ private static int getFreshNamespaceIndex() {
+ return namespaceCounter.getAndIncrement();
+ }
+
+ public static <OutputT> SingleOutputExpandableTransform<OutputT> of(
+ String urn, byte[] payload, String endpoint) {
+ Endpoints.ApiServiceDescriptor apiDesc =
+ Endpoints.ApiServiceDescriptor.newBuilder().setUrl(endpoint).build();
+ return new SingleOutputExpandableTransform<>(urn, payload, apiDesc,
getFreshNamespaceIndex());
+ }
+
+ /** Expandable transform for output type of PCollection. */
+ public static class SingleOutputExpandableTransform<OutputT>
+ extends ExpandableTransform<PCollection<OutputT>> {
+ SingleOutputExpandableTransform(
+ String urn,
+ byte[] payload,
+ Endpoints.ApiServiceDescriptor endpoint,
+ Integer namespaceIndex) {
+ super(urn, payload, endpoint, namespaceIndex);
+ }
+
+ @Override
+ PCollection<OutputT> toOutputCollection(Map<TupleTag<?>, PCollection>
output) {
+ checkArgument(output.size() > 0, "output shouldn't be empty.");
+ return Iterables.getOnlyElement(output.values());
+ }
+
+ public MultiOutputExpandableTransform withMultiOutputs() {
+ return new MultiOutputExpandableTransform(
+ getUrn(), getPayload(), getEndpoint(), getNamespaceIndex());
+ }
+ }
+
+ /** Expandable transform for output type of PCollectionTuple. */
+ public static class MultiOutputExpandableTransform extends
ExpandableTransform<PCollectionTuple> {
+ MultiOutputExpandableTransform(
+ String urn,
+ byte[] payload,
+ Endpoints.ApiServiceDescriptor endpoint,
+ Integer namespaceIndex) {
+ super(urn, payload, endpoint, namespaceIndex);
+ }
+
+ @Override
+ PCollectionTuple toOutputCollection(Map<TupleTag<?>, PCollection> output) {
+ checkArgument(output.size() > 0, "output shouldn't be empty.");
+ PCollection firstElem = Iterables.getFirst(output.values(), null);
+ PCollectionTuple pCollectionTuple =
PCollectionTuple.empty(firstElem.getPipeline());
+ for (Map.Entry<TupleTag<?>, PCollection> entry : output.entrySet()) {
+ pCollectionTuple = pCollectionTuple.and(entry.getKey(),
entry.getValue());
+ }
+ return pCollectionTuple;
+ }
+ }
+
+ /** Base Expandable Transform which calls ExpansionService to expand itself.
*/
+ public abstract static class ExpandableTransform<OutputT extends POutput>
+ extends PTransform<PInput, OutputT> {
+ private final String urn;
+ private final byte[] payload;
+ private final Endpoints.ApiServiceDescriptor endpoint;
+ private final Integer namespaceIndex;
+
+ @Nullable private transient RunnerApi.Components expandedComponents;
+ @Nullable private transient RunnerApi.PTransform expandedTransform;
+ @Nullable private transient Map<PCollection, String>
externalPCollectionIdMap;
+
+ ExpandableTransform(
+ String urn,
+ byte[] payload,
+ Endpoints.ApiServiceDescriptor endpoint,
+ Integer namespaceIndex) {
+ this.urn = urn;
+ this.payload = payload;
+ this.endpoint = endpoint;
+ this.namespaceIndex = namespaceIndex;
+ }
+
+ @Override
+ public OutputT expand(PInput input) {
+ Pipeline p = input.getPipeline();
+ SdkComponents components = SdkComponents.create(p.getOptions());
+ RunnerApi.PTransform.Builder ptransformBuilder =
+ RunnerApi.PTransform.newBuilder()
+ .setUniqueName(EXPANDED_TRANSFORM_BASE_NAME + namespaceIndex)
+ .setSpec(
+ RunnerApi.FunctionSpec.newBuilder()
+ .setUrn(urn)
+ .setPayload(ByteString.copyFrom(payload))
+ .build());
+ ImmutableMap.Builder<PCollection, String>
externalPCollectionIdMapBuilder =
+ ImmutableMap.builder();
+ for (Map.Entry<TupleTag<?>, PValue> entry : input.expand().entrySet()) {
+ if (entry.getValue() instanceof PCollection<?>) {
+ try {
+ String id = components.registerPCollection((PCollection)
entry.getValue());
+ externalPCollectionIdMapBuilder.put((PCollection)
entry.getValue(), id);
+ ptransformBuilder.putInputs(entry.getKey().getId(), id);
+ AppliedPTransform<?, ?, ?> fakeImpulse =
+ AppliedPTransform.of(
+ String.format("%s_%s", IMPULSE_PREFIX,
entry.getKey().getId()),
+ PBegin.in(p).expand(),
+ ImmutableMap.of(entry.getKey(), entry.getValue()),
+ Impulse.create(),
+ p);
+ components.registerPTransform(fakeImpulse,
Collections.emptyList());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
Review comment:
Please add a error message.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 211920)
Time Spent: 1h 40m (was: 1.5h)
> Adding ExternalTransform in JavaSDK
> -----------------------------------
>
> Key: BEAM-6747
> URL: https://issues.apache.org/jira/browse/BEAM-6747
> Project: Beam
> Issue Type: Improvement
> Components: runner-core
> Reporter: Heejong Lee
> Assignee: Heejong Lee
> Priority: Major
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
> Adding Java counterpart of Python ExternalTransform for testing Python
> transforms from pipelines in Java SDK.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)