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

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

                Author: ASF GitHub Bot
            Created on: 22/May/19 22:48
            Start Date: 22/May/19 22:48
    Worklog Time Spent: 10m 
      Work Description: ibzib commented on pull request #8558: [BEAM-7131] 
Spark: cache executable stage output to prevent re-computation
URL: https://github.com/apache/beam/pull/8558#discussion_r286666450
 
 

 ##########
 File path: 
runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPortableExecutionTest.java
 ##########
 @@ -159,8 +162,166 @@ public void process(ProcessContext context) {
             pipelineProto,
             options.as(SparkPipelineOptions.class));
     jobInvocation.start();
-    while (jobInvocation.getState() != Enum.DONE) {
-      Thread.sleep(1000);
-    }
+    Assert.assertEquals(Enum.DONE, jobInvocation.getState());
+  }
+
+  /**
+   * Verifies that each executable stage runs exactly once, even if that 
executable stage has
+   * multiple immediate outputs. While re-computation may be necessary in the 
event of failure,
+   * re-computation of a whole executable stage is expensive and can cause 
unexpected behavior when
+   * the executable stage has side effects (BEAM-7131).
+   *
+   * <pre>
+   *    |-> B -> GBK
+   * A -|
+   *    |-> C -> GBK
+   * </pre>
+   */
+  @Test(timeout = 120_000)
+  public void testExecStageWithMultipleOutputs() throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    options.setRunner(CrashingRunner.class);
+    options
+        .as(PortablePipelineOptions.class)
+        .setDefaultEnvironmentType(Environments.ENVIRONMENT_EMBEDDED);
+    Pipeline pipeline = Pipeline.create(options);
+    String path =
+        FileSystems.getDefault()
+            .getPath(temporaryFolder.getRoot().getAbsolutePath(), 
UUID.randomUUID().toString())
+            .toString();
+    File file = new File(path);
+    PCollection<String> a =
+        pipeline
+            .apply("impulse", Impulse.create())
+            .apply(
+                "A",
+                ParDo.of(
+                    new DoFn<byte[], String>() {
+                      @ProcessElement
+                      public void process(ProcessContext context) throws 
Exception {
+                        context.output("A");
+                        // ParDos A, B, and C will all be fused together into 
the same executable
+                        // stage. This check verifies that stage is not run 
more than once by
+                        // enacting a side effect via the local file system.
+                        Assert.assertTrue(
+                            String.format(
+                                "Create file %s failed (ParDo A should only 
have been run once).",
+                                path),
+                            file.createNewFile());
+                      }
+                    }));
+    PCollection<KV<String, String>> b =
+        a.apply(
+            "B",
+            ParDo.of(
+                new DoFn<String, KV<String, String>>() {
+                  @ProcessElement
+                  public void process(ProcessContext context) {
+                    context.output(KV.of(context.element(), "B"));
+                  }
+                }));
+    PCollection<KV<String, String>> c =
+        a.apply(
+            "C",
+            ParDo.of(
+                new DoFn<String, KV<String, String>>() {
+                  @ProcessElement
+                  public void process(ProcessContext context) {
+                    context.output(KV.of(context.element(), "C"));
+                  }
+                }));
+    // Use GBKs to force re-computation of executable stage unless cached.
+    b.apply(GroupByKey.create());
+    c.apply(GroupByKey.create());
+    RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline);
+    JobInvocation jobInvocation =
+        SparkJobInvoker.createJobInvocation(
+            "testExecStageWithMultipleOutputs",
+            "testExecStageWithMultipleOutputsRetrievalToken",
+            sparkJobExecutor,
+            pipelineProto,
+            options.as(SparkPipelineOptions.class));
+    jobInvocation.start();
+    Assert.assertEquals(Enum.DONE, jobInvocation.getState());
+    Assert.assertTrue(file.exists());
+  }
+
+  /**
+   * Verifies that each executable stage runs exactly once, even if that 
executable stage has
+   * multiple downstream consumers. While re-computation may be necessary in 
the event of failure,
+   * re-computation of a whole executable stage is expensive and can cause 
unexpected behavior when
+   * the executable stage has side effects (BEAM-7131).
+   *
+   * <pre>
+   *           |-> B
+   * A -> GBK -|
+   *           |-> C
+   * </pre>
+   */
+  @Test(timeout = 120_000)
+  public void testExecStageWithMultipleConsumers() throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    options.setRunner(CrashingRunner.class);
+    options
+        .as(PortablePipelineOptions.class)
+        .setDefaultEnvironmentType(Environments.ENVIRONMENT_EMBEDDED);
+    Pipeline pipeline = Pipeline.create(options);
+    String path =
+        FileSystems.getDefault()
+            .getPath(temporaryFolder.getRoot().getAbsolutePath(), 
UUID.randomUUID().toString())
+            .toString();
+    File file = new File(path);
+    PCollection<KV<String, Integer>> a =
+        pipeline
+            .apply("impulse", Impulse.create())
+            .apply(
+                "A",
+                ParDo.of(
+                    new DoFn<byte[], KV<String, Integer>>() {
 
 Review comment:
   Done
 
----------------------------------------------------------------
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: 247116)
    Time Spent: 4h 10m  (was: 4h)

> Spark portable runner appears to be repeating work (in TFX example)
> -------------------------------------------------------------------
>
>                 Key: BEAM-7131
>                 URL: https://issues.apache.org/jira/browse/BEAM-7131
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-spark
>            Reporter: Kyle Weaver
>            Assignee: Kyle Weaver
>            Priority: Major
>          Time Spent: 4h 10m
>  Remaining Estimate: 0h
>
> I've been trying to run the TFX Chicago taxi example [1] on the Spark 
> portable runner. TFDV works fine, but the preprocess step 
> (preprocess_flink.sh [2]) fails with the following error:
> RuntimeError: AlreadyExistsError: file already exists [while running 
> 'WriteTransformFn/WriteTransformFn']
> Assets are being written multiple times to different temp directories, which 
> is okay, but the error occurs when they are copied to the same permanent 
> output directory. Specifically, the copy tree operation in transform_fn_io.py 
> [3] is run twice with the same output directory. The error doesn't occur when 
> that code is modified to allow overwriting existing files, but that's only a 
> shallow fix. While the TF transform should probably be made idempotent, this 
> is also an issue with the Spark runner, which shouldn't be repeating work 
> like this regularly (in the absence of a failure condition).
> [1] [https://github.com/tensorflow/tfx/tree/master/tfx/examples/chicago_taxi]
> [2] 
> [https://github.com/tensorflow/tfx/blob/master/tfx/examples/chicago_taxi/preprocess_flink.sh]
> [3] 
> [https://github.com/tensorflow/transform/blob/master/tensorflow_transform/beam/tft_beam_io/transform_fn_io.py#L33-L45]



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

Reply via email to