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

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

                Author: ASF GitHub Bot
            Created on: 10/May/18 00:09
            Start Date: 10/May/18 00:09
    Worklog Time Spent: 10m 
      Work Description: bsidhom commented on a change in pull request #5285: 
[BEAM-2597] Flink batch ExecutableStage operator
URL: https://github.com/apache/beam/pull/5285#discussion_r187207725
 
 

 ##########
 File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
 ##########
 @@ -0,0 +1,245 @@
+/*
+ * 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.flink.translation.functions;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.Struct;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
+import org.apache.beam.runners.flink.ArtifactSourcePool;
+import org.apache.beam.runners.flink.FlinkBundleFactory;
+import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
+import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
+import org.apache.beam.runners.fnexecution.control.RemoteBundle;
+import org.apache.beam.runners.fnexecution.control.StageBundleFactory;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
+import org.apache.beam.sdk.fn.data.FnDataReceiver;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.api.common.cache.DistributedCache;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.InOrder;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.stubbing.Answer;
+
+/** Tests for {@link FlinkExecutableStageFunction}. */
+@RunWith(JUnit4.class)
+public class FlinkExecutableStageFunctionTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Mock private RuntimeContext runtimeContext;
+  @Mock private DistributedCache distributedCache;
+  @Mock private Collector<RawUnionValue> collector;
+  @Mock private StageBundleFactory stageBundleFactory;
+  @Mock private ArtifactSourcePool cachePool;
+  @Mock private StateRequestHandler stateRequestHandler;
+
+  // NOTE: ExecutableStage.fromPayload expects exactly one input, so we 
provide one here. These unit
+  // tests in general ignore the executable stage itself and mock around it.
+  private final ExecutableStagePayload stagePayload =
+      ExecutableStagePayload.newBuilder()
+          .setInput("input")
+          .setComponents(
+              Components.newBuilder()
+                  .putPcollections("input", PCollection.getDefaultInstance())
+                  .build())
+          .build();
+  private final JobInfo jobInfo = JobInfo.create("job-id", "job-name", 
Struct.getDefaultInstance());
+
+  @Before
+  public void setUpMocks() {
+    MockitoAnnotations.initMocks(this);
+    when(runtimeContext.getDistributedCache()).thenReturn(distributedCache);
+  }
+
+  @Test
+  public void sdkErrorsSurfaceOnClose() throws Exception {
+    FlinkExecutableStageFunction<Integer> function = 
getFunction(Collections.emptyMap());
+    function.open(new Configuration());
+
+    @SuppressWarnings("unchecked")
+    RemoteBundle<Integer> bundle = Mockito.mock(RemoteBundle.class);
+    when(stageBundleFactory.<Integer>getBundle(any(), 
any())).thenReturn(bundle);
+
+    @SuppressWarnings("unchecked")
+    FnDataReceiver<WindowedValue<Integer>> receiver = 
Mockito.mock(FnDataReceiver.class);
+    when(bundle.getInputReceiver()).thenReturn(receiver);
+
+    Exception expected = new Exception();
+    doThrow(expected).when(bundle).close();
+    thrown.expect(is(expected));
+    function.mapPartition(Collections.emptyList(), collector);
+  }
+
+  @Test
+  public void checksForRuntimeContextChanges() throws Exception {
+    FlinkExecutableStageFunction<Integer> function = 
getFunction(Collections.emptyMap());
+    function.open(new Configuration());
+    // Change runtime context.
+    function.setRuntimeContext(Mockito.mock(RuntimeContext.class));
+    thrown.expect(Matchers.instanceOf(IllegalStateException.class));
+    function.mapPartition(Collections.emptyList(), collector);
+  }
+
+  @Test
+  public void inputsAreSentInOrder() throws Exception {
 
 Review comment:
   Good point. I'll change it to only check that (exactly) the set of inputs 
are sent.

----------------------------------------------------------------
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: 100340)
    Time Spent: 3h 20m  (was: 3h 10m)

> FlinkRunner ExecutableStage batch operator
> ------------------------------------------
>
>                 Key: BEAM-2597
>                 URL: https://issues.apache.org/jira/browse/BEAM-2597
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-flink
>            Reporter: Kenneth Knowles
>            Assignee: Ben Sidhom
>            Priority: Major
>              Labels: portability
>          Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> This operator will execute user code in the context of an SDK harness by 
> constructing a ProcessBundleDescriptor from an ExecutableStage (physical 
> stage plan) and sending instructions/elements over the control and data 
> planes.



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

Reply via email to