[ 
https://issues.apache.org/jira/browse/BEAM-6252?focusedWorklogId=176656&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-176656
 ]

ASF GitHub Bot logged work on BEAM-6252:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Dec/18 18:26
            Start Date: 18/Dec/18 18:26
    Worklog Time Spent: 10m 
      Work Description: xinyuiscool closed pull request #7301: [BEAM-6252] 
SamzaRunner: Add a registrar to allow customized DoFnInvoker
URL: https://github.com/apache/beam/pull/7301
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
index 7918385cf556..ed6091eb70b5 100644
--- 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
+++ 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
@@ -17,11 +17,14 @@
  */
 package org.apache.beam.runners.samza.runtime;
 
+import com.google.common.collect.Iterators;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.ServiceLoader;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
@@ -168,7 +171,14 @@ public void open(
 
     this.pushbackValues = new ArrayList<>();
 
-    doFnInvoker = DoFnInvokers.invokerFor(doFn);
+    final Iterator<SamzaDoFnInvokerRegistrar> invokerReg =
+        ServiceLoader.load(SamzaDoFnInvokerRegistrar.class).iterator();
+    if (!invokerReg.hasNext()) {
+      // use the default invoker here
+      doFnInvoker = DoFnInvokers.invokerFor(doFn);
+    } else {
+      doFnInvoker = Iterators.getOnlyElement(invokerReg).invokerFor(doFn, 
context);
+    }
 
     doFnInvoker.invokeSetup();
   }
diff --git 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnInvokerRegistrar.java
 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnInvokerRegistrar.java
new file mode 100644
index 000000000000..5ac9c118f6df
--- /dev/null
+++ 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnInvokerRegistrar.java
@@ -0,0 +1,34 @@
+/*
+ * 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.samza.runtime;
+
+import java.util.Map;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.samza.task.TaskContext;
+
+/** A registrar for Samza DoFnInvoker. */
+public interface SamzaDoFnInvokerRegistrar {
+
+  /** Returns the invoker for a {@link DoFn}. */
+  <InputT, OutputT> DoFnInvoker<InputT, OutputT> invokerFor(
+      DoFn<InputT, OutputT> fn, TaskContext context);
+
+  /** Returns the configs for a {@link DoFn}. */
+  <InputT, OutputT> Map<String, String> configFor(DoFn<InputT, OutputT> fn);
+}
diff --git 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
index 64e55d309e70..f1cb6a25625c 100644
--- 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
+++ 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
@@ -17,18 +17,22 @@
  */
 package org.apache.beam.runners.samza.translation;
 
+import com.google.common.collect.Iterators;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.ServiceLoader;
 import java.util.stream.Collectors;
 import org.apache.beam.runners.samza.runtime.DoFnOp;
 import org.apache.beam.runners.samza.runtime.Op;
 import org.apache.beam.runners.samza.runtime.OpAdapter;
 import org.apache.beam.runners.samza.runtime.OpEmitter;
 import org.apache.beam.runners.samza.runtime.OpMessage;
+import org.apache.beam.runners.samza.runtime.SamzaDoFnInvokerRegistrar;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.runners.TransformHierarchy;
@@ -52,6 +56,14 @@
     implements TransformTranslator<ParDo.MultiOutput<InT, OutT>>,
         TransformConfigGenerator<ParDo.MultiOutput<InT, OutT>> {
 
+  private final SamzaDoFnInvokerRegistrar doFnInvokerRegistrar;
+
+  ParDoBoundMultiTranslator() {
+    final Iterator<SamzaDoFnInvokerRegistrar> invokerReg =
+        ServiceLoader.load(SamzaDoFnInvokerRegistrar.class).iterator();
+    doFnInvokerRegistrar = invokerReg.hasNext() ? 
Iterators.getOnlyElement(invokerReg) : null;
+  }
+
   @Override
   public void translate(
       ParDo.MultiOutput<InT, OutT> transform,
@@ -156,6 +168,11 @@ public void translate(
         config.put("stores." + storeId + ".msg.serde", "byteSerde");
       }
     }
+
+    if (doFnInvokerRegistrar != null) {
+      config.putAll(doFnInvokerRegistrar.configFor(transform.getFn()));
+    }
+
     return config;
   }
 
diff --git 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
index 30bb07aaa79c..5f978022c2c4 100644
--- 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
+++ 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
@@ -29,8 +29,14 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
 import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.table.Table;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Helper that keeps the mapping from BEAM {@link PValue}/{@link 
PCollectionView} to Samza {@link
@@ -38,10 +44,13 @@
  * PTransform}.
  */
 public class TranslationContext {
+  public static final Logger LOG = 
LoggerFactory.getLogger(TranslationContext.class);
   private final StreamGraph streamGraph;
   private final Map<PValue, MessageStream<?>> messsageStreams = new 
HashMap<>();
   private final Map<PCollectionView<?>, MessageStream<?>> viewStreams = new 
HashMap<>();
   private final Map<PValue, String> idMap;
+  private final Map<String, MessageStream> registeredInputStreams = new 
HashMap<>();
+  private final Map<String, Table> registeredTables = new HashMap<>();
   private final PValue dummySource;
   private final SamzaPipelineOptions options;
 
@@ -62,7 +71,14 @@ public TranslationContext(
   public <OutT> void registerInputMessageStream(PValue pvalue) {
     // We only register dummySource if it is actually used (determined by a 
call to getDummyStream).
     if (!pvalue.equals(dummySource)) {
-      doRegisterInputMessageStream(pvalue);
+      doRegisterInputMessageStream(pvalue, getIdForPValue(pvalue));
+    }
+  }
+
+  public <OutT> void registerInputMessageStreamById(PValue pvalue, String 
streamId) {
+    // We only register dummySource if it is actually used (determined by a 
call to getDummyStream).
+    if (!pvalue.equals(dummySource)) {
+      doRegisterInputMessageStream(pvalue, streamId);
     }
   }
 
@@ -76,7 +92,7 @@ public TranslationContext(
 
   public MessageStream<OpMessage<String>> getDummyStream() {
     if (!messsageStreams.containsKey(dummySource)) {
-      doRegisterInputMessageStream(dummySource);
+      doRegisterInputMessageStream(dummySource, getIdForPValue(dummySource));
     }
 
     return getMessageStream(dummySource);
@@ -161,15 +177,36 @@ public SamzaPipelineOptions getPipelineOptions() {
     return this.options;
   }
 
-  private <OutT> void doRegisterInputMessageStream(PValue pvalue) {
+  public <OutT> OutputStream<OutT> getOutputStreamById(String outputStreamId) {
+    return streamGraph.getOutputStream(outputStreamId);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDesc) {
+    return registeredTables.computeIfAbsent(
+        tableDesc.getTableId(), id -> streamGraph.getTable(tableDesc));
+  }
+
+  private <OutT> void doRegisterInputMessageStream(PValue pvalue, String 
streamId) {
+    // we want to register it with the Samza graph only once per i/o stream
+    if (registeredInputStreams.containsKey(streamId)) {
+      MessageStream<OpMessage<OutT>> messageStream = 
registeredInputStreams.get(streamId);
+      LOG.info(
+          String.format(
+              "Stream id %s has already been mapped to %s stream. Mapping %s 
to the same message stream.",
+              streamId, messageStream, pvalue));
+      registerMessageStream(pvalue, messageStream);
+
+      return;
+    }
     @SuppressWarnings("unchecked")
     final MessageStream<OpMessage<OutT>> typedStream =
         streamGraph
-            .<org.apache.samza.operators.KV<?, OpMessage<OutT>>>getInputStream(
-                getIdForPValue(pvalue))
+            .<org.apache.samza.operators.KV<?, 
OpMessage<OutT>>>getInputStream(streamId)
             .map(org.apache.samza.operators.KV::getValue);
 
     registerMessageStream(pvalue, typedStream);
+    registeredInputStreams.put(streamId, typedStream);
   }
 
   private String getIdForPValue(PValue pvalue) {
diff --git 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineDotRenderer.java
 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineDotRenderer.java
index 09f4a978420a..25cd44ace82c 100644
--- 
a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineDotRenderer.java
+++ 
b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineDotRenderer.java
@@ -65,12 +65,13 @@ public void leaveCompositeTransform(TransformHierarchy.Node 
node) {
   @Override
   public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     final int nodeId = nextNodeId++;
+    writeLine("%d [label=\"%s\"]", nodeId, 
escapeString(node.getTransform().getName()));
+
     node.getOutputs()
         .values()
         .forEach(
             x -> {
               valueToProducerNodeId.put(x, nodeId);
-              writeLine("%d [label=\"%s\"]", nodeId, 
escapeString(node.getTransform().getName()));
             });
 
     node.getInputs()


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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: 176656)
    Time Spent: 1h 10m  (was: 1h)

> SamzaRunner: Add a registrar to allow customized DoFnInvoker
> ------------------------------------------------------------
>
>                 Key: BEAM-6252
>                 URL: https://issues.apache.org/jira/browse/BEAM-6252
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-samza
>            Reporter: Xinyu Liu
>            Assignee: Xinyu Liu
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Add an registrar to allow us to extend the open source DoFnInvoker for our 
> use cases.
> Also fixed a bug in the dot renderer.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to