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

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

                Author: ASF GitHub Bot
            Created on: 18/Jun/18 15:25
            Start Date: 18/Jun/18 15:25
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#5517: [BEAM-3079] Update samza-runner with more features and improvements
URL: https://github.com/apache/beam/pull/5517#discussion_r196119260
 
 

 ##########
 File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java
 ##########
 @@ -0,0 +1,216 @@
+/*
+ * 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 static com.google.common.base.Preconditions.checkState;
+
+import java.util.List;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateInternalsFactory;
+import org.apache.beam.runners.core.StatefulDoFnRunner;
+import org.apache.beam.runners.core.StepContext;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.core.TimerInternalsFactory;
+import org.apache.beam.runners.samza.metrics.DoFnRunnerWithMetrics;
+import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Instant;
+
+/**
+ * This class wraps a DoFnRunner with keyed StateInternals and TimerInternals 
access.
+ */
+public class DoFnRunnerWithKeyedInternals<InputT, OutputT> implements 
DoFnRunner<InputT, OutputT> {
+  private final DoFnRunner<InputT, OutputT> underlying;
+  private final KeyedInternals keyedInternals;
+
+  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> of(
+      PipelineOptions options,
+      DoFn<InputT, OutputT> doFn,
+      SideInputReader sideInputReader,
+      DoFnRunners.OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> additionalOutputTags,
+      StateInternalsFactory<?> stateInternalsFactory,
+      TimerInternalsFactory<?> timerInternalsFactory,
+      WindowingStrategy<?, ?> windowingStrategy,
+      SamzaMetricsContainer metricsContainer,
+      String stepName) {
+
+    final DoFnSignature signature = 
DoFnSignatures.getSignature(doFn.getClass());
+    final KeyedInternals keyedInternals;
+    final TimerInternals timerInternals;
+    final StateInternals stateInternals;
+
+    if (signature.usesState()) {
+      keyedInternals = new KeyedInternals(stateInternalsFactory, 
timerInternalsFactory);
+      stateInternals = keyedInternals.stateInternals();
+      timerInternals = keyedInternals.timerInternals();
+    } else {
+      keyedInternals = null;
+      stateInternals = stateInternalsFactory.stateInternalsForKey(null);
+      timerInternals = timerInternalsFactory.timerInternalsForKey(null);
+    }
+
+    final DoFnRunner<InputT, OutputT> doFnRunner = DoFnRunners.simpleRunner(
+        options,
+        doFn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        additionalOutputTags,
+        createStepContext(stateInternals, timerInternals),
+        windowingStrategy);
+
+    final DoFnRunner<InputT, OutputT> doFnRunnerWithMetrics = 
DoFnRunnerWithMetrics
+        .wrap(doFnRunner, metricsContainer, stepName);
+
+    if (keyedInternals != null) {
 
 Review comment:
   This conditional goes right along with the one above to make just two code 
paths: the one where this method actually does something, and the one that is 
basically pass through.

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 112779)
    Time Spent: 2h 40m  (was: 2.5h)

> Samza runner
> ------------
>
>                 Key: BEAM-3079
>                 URL: https://issues.apache.org/jira/browse/BEAM-3079
>             Project: Beam
>          Issue Type: Wish
>          Components: runner-ideas
>            Reporter: Xinyu Liu
>            Assignee: Kenneth Knowles
>            Priority: Major
>             Fix For: Not applicable
>
>          Time Spent: 2h 40m
>  Remaining Estimate: 0h
>
> Apache Samza is a distributed data-processing platform which supports both 
> stream and batch processing. It'll be awesome if we can run BEAM's advanced 
> data transform and multi-language sdks on top of Samza.



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

Reply via email to