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

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

                Author: ASF GitHub Bot
            Created on: 30/Jan/19 14:22
            Start Date: 30/Jan/19 14:22
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on pull request #7619: [BEAM-6504] 
Create Portable sideInput handler for Dataflow (Part One)
URL: https://github.com/apache/beam/pull/7619#discussion_r252269250
 
 

 ##########
 File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
 ##########
 @@ -0,0 +1,149 @@
+/*
+ * 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.dataflow.worker.fn.control;
+
+import static com.google.common.base.Preconditions.checkState;
+import static 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Materializations;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataflowSideInputHandlerFactory
+    implements StateRequestHandlers.SideInputHandlerFactory {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataflowSideInputHandlerFactory.class);
+
+  private final Map<String, SideInputReader> ptransformIdToSideInputReader;
+  private final Map<RunnerApi.ExecutableStagePayload.SideInputId, 
PCollectionView<?>>
+      sideInputIdToPCollectionViewMap;
+
+  static DataflowSideInputHandlerFactory of(
+      Map<String, SideInputReader> ptransformIdToSideInputReader,
+      Map<RunnerApi.ExecutableStagePayload.SideInputId, PCollectionView<?>>
+          sideInputIdToPCollectionViewMap) {
+    return new DataflowSideInputHandlerFactory(
+        ptransformIdToSideInputReader, sideInputIdToPCollectionViewMap);
+  }
+
+  private DataflowSideInputHandlerFactory(
+      Map<String, SideInputReader> ptransformIdToSideInputReader,
+      Map<RunnerApi.ExecutableStagePayload.SideInputId, PCollectionView<?>>
+          sideInputIdToPCollectionViewMap) {
+    this.ptransformIdToSideInputReader = ptransformIdToSideInputReader;
+    this.sideInputIdToPCollectionViewMap = sideInputIdToPCollectionViewMap;
+  }
+
+  @Override
+  public <T, V, W extends BoundedWindow> 
StateRequestHandlers.SideInputHandler<V, W> forSideInput(
+      String pTransformId,
+      String sideInputId,
+      RunnerApi.FunctionSpec accessPattern,
+      Coder<T> elementCoder,
+      Coder<W> windowCoder) {
+    checkArgument(pTransformId != null, String.format("Expect a valid 
PTransform ID."));
 
 Review comment:
   Lets check for both null and empty string 
 
----------------------------------------------------------------
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: 192304)

> Integration of Portabability sideInput into Dataflow
> ----------------------------------------------------
>
>                 Key: BEAM-6504
>                 URL: https://issues.apache.org/jira/browse/BEAM-6504
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-dataflow
>            Reporter: Ruoyun Huang
>            Assignee: Ruoyun Huang
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Underlying fn api support is done in BEAM-2929, this Jira integrates 
> everything into dataflow. 
>  
> 1) introduce a sideInputHandler for dataflow. 
> 2) wire the handler to dataflow runner (i.e.  ProcessRemoteBundleOperation)



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

Reply via email to