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

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

                Author: ASF GitHub Bot
            Created on: 30/Sep/21 23:04
            Start Date: 30/Sep/21 23:04
    Worklog Time Spent: 10m 
      Work Description: laraschmidt commented on a change in pull request 
#15540:
URL: https://github.com/apache/beam/pull/15540#discussion_r719827260



##########
File path: 
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
##########
@@ -3878,6 +3885,186 @@ public void 
testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc
     }
   }
 
+  @RunWith(JUnit4.class)
+  public static class ExceptionThrowingExecutionTest {
+    @Rule public final ExpectedException thrown = ExpectedException.none();
+
+    public static final String TEST_TRANSFORM_ID = "pTransformId";
+
+    /**
+     * A {@link DoFn} that outputs elements with timestamp equal to the input 
timestamp minus the
+     * input element.
+     */
+    private static class SkewingDoFn extends DoFn<String, String> {
+      private final Duration allowedSkew;
+
+      private SkewingDoFn(Duration allowedSkew) {
+        this.allowedSkew = allowedSkew;
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext context) {
+        Duration duration = new Duration(Long.valueOf(context.element()));
+        context.outputWithTimestamp(context.element(), 
context.timestamp().minus(duration));
+      }
+
+      @Override
+      public Duration getAllowedTimestampSkew() {
+        return allowedSkew;
+      }
+    }
+
+    @Test
+    public void testDoFnSkewNotAllowed() throws Exception {
+      Pipeline p = Pipeline.create();
+      PCollection<String> valuePCollection = p.apply(Create.of("0", "1"));
+      PCollection<String> outputPCollection =
+          valuePCollection.apply(TEST_TRANSFORM_ID, ParDo.of(new 
SkewingDoFn(Duration.ZERO)));
+
+      SdkComponents sdkComponents = SdkComponents.create(p.getOptions());
+      RunnerApi.Pipeline pProto = PipelineTranslation.toProto(p, 
sdkComponents);
+      String inputPCollectionId = 
sdkComponents.registerPCollection(valuePCollection);
+      String outputPCollectionId = 
sdkComponents.registerPCollection(outputPCollection);
+      RunnerApi.PTransform pTransform =
+          pProto
+              .getComponents()
+              .getTransformsOrThrow(
+                  pProto
+                      .getComponents()
+                      .getTransformsOrThrow(TEST_TRANSFORM_ID)
+                      .getSubtransforms(0));
+
+      List<WindowedValue<String>> mainOutputValues = new ArrayList<>();
+      MetricsContainerStepMap metricsContainerRegistry = new 
MetricsContainerStepMap();
+      PCollectionConsumerRegistry consumers =
+          new PCollectionConsumerRegistry(
+              metricsContainerRegistry, mock(ExecutionStateTracker.class));
+
+      consumers.register(
+          outputPCollectionId,
+          TEST_TRANSFORM_ID,
+          (FnDataReceiver) (FnDataReceiver<WindowedValue<String>>) 
mainOutputValues::add,
+          StringUtf8Coder.of());
+      PTransformFunctionRegistry startFunctionRegistry =
+          new PTransformFunctionRegistry(
+              mock(MetricsContainerStepMap.class), 
mock(ExecutionStateTracker.class), "start");
+      PTransformFunctionRegistry finishFunctionRegistry =
+          new PTransformFunctionRegistry(
+              mock(MetricsContainerStepMap.class), 
mock(ExecutionStateTracker.class), "finish");
+      List<ThrowingRunnable> teardownFunctions = new ArrayList<>();
+
+      new FnApiDoFnRunner.Factory<>()
+          .createRunnerForPTransform(
+              PipelineOptionsFactory.create(),
+              null /* beamFnDataClient */,
+              null /**/,
+              null /* beamFnTimerClient */,
+              TEST_TRANSFORM_ID,
+              pTransform,
+              Suppliers.ofInstance("57L")::get,
+              pProto.getComponents().getPcollectionsMap(),
+              pProto.getComponents().getCodersMap(),
+              pProto.getComponents().getWindowingStrategiesMap(),
+              consumers,
+              startFunctionRegistry,
+              finishFunctionRegistry,
+              null /* addResetFunction */,
+              teardownFunctions::add,
+              null /* addProgressRequestCallback */,
+              null /* splitListener */,
+              null /* bundleFinalizer */);
+
+      thrown.expect(UserCodeException.class);
+      thrown.expectMessage(
+          String.format("timestamp %s", new Instant(0).minus(new 
Duration(1L))));
+      thrown.expectMessage(
+          String.format(
+              "allowed skew (%s)",
+              PeriodFormat.getDefault().print(Duration.ZERO.toPeriod())));
+
+      Iterables.getOnlyElement(startFunctionRegistry.getFunctions()).run();
+      mainOutputValues.clear();
+
+      FnDataReceiver<WindowedValue<?>> mainInput =
+          consumers.getMultiplexingConsumer(inputPCollectionId);
+      mainInput.accept(valueInGlobalWindow("0"));
+      mainInput.accept(
+          timestampedValueInGlobalWindow("1", new Instant(0L)));
+    }
+
+    @Test
+    public void testDoFnSkewAllowed() throws Exception {
+      Pipeline p = Pipeline.create();
+      PCollection<String> valuePCollection = p.apply(Create.of("0", "3"));
+      PCollection<String> outputPCollection =
+          valuePCollection.apply(TEST_TRANSFORM_ID, ParDo.of(new 
SkewingDoFn(new Duration(5L))));
+
+      SdkComponents sdkComponents = SdkComponents.create(p.getOptions());
+      RunnerApi.Pipeline pProto = PipelineTranslation.toProto(p, 
sdkComponents);
+      String inputPCollectionId = 
sdkComponents.registerPCollection(valuePCollection);
+      String outputPCollectionId = 
sdkComponents.registerPCollection(outputPCollection);
+      RunnerApi.PTransform pTransform =
+          pProto
+              .getComponents()
+              .getTransformsOrThrow(
+                  pProto
+                      .getComponents()
+                      .getTransformsOrThrow(TEST_TRANSFORM_ID)
+                      .getSubtransforms(0));
+
+      List<WindowedValue<String>> mainOutputValues = new ArrayList<>();
+      MetricsContainerStepMap metricsContainerRegistry = new 
MetricsContainerStepMap();
+      PCollectionConsumerRegistry consumers =
+          new PCollectionConsumerRegistry(
+              metricsContainerRegistry, mock(ExecutionStateTracker.class));
+
+      consumers.register(
+          outputPCollectionId,
+          TEST_TRANSFORM_ID,
+          (FnDataReceiver) (FnDataReceiver<WindowedValue<String>>) 
mainOutputValues::add,
+          StringUtf8Coder.of());
+      PTransformFunctionRegistry startFunctionRegistry =
+          new PTransformFunctionRegistry(
+              mock(MetricsContainerStepMap.class), 
mock(ExecutionStateTracker.class), "start");
+      PTransformFunctionRegistry finishFunctionRegistry =
+          new PTransformFunctionRegistry(
+              mock(MetricsContainerStepMap.class), 
mock(ExecutionStateTracker.class), "finish");
+      List<ThrowingRunnable> teardownFunctions = new ArrayList<>();
+
+      new FnApiDoFnRunner.Factory<>()
+          .createRunnerForPTransform(
+              PipelineOptionsFactory.create(),
+              null /* beamFnDataClient */,
+              null /**/,

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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

    Worklog Id:     (was: 658649)
    Time Spent: 2h  (was: 1h 50m)

> Timer.setOutputTimestamp doesn't take into account for 
> DoFn#getAllowedTimestampSkew()
> -------------------------------------------------------------------------------------
>
>                 Key: BEAM-12931
>                 URL: https://issues.apache.org/jira/browse/BEAM-12931
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model
>            Reporter: Lara Schmidt
>            Assignee: Lara Schmidt
>            Priority: P2
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> A DoFn may emit elements with a timestamp up to 
> DoFn#getAllowedTimestampSkew() before the current element's timestamp. 
> However getAllowedTimestampSkew is not properly accounted for in looking at 
> the output timestamp of a timer.
> Context: 
> [https://lists.apache.org/thread.html/r7554658114ddde86c5d82e1c39fe7e1ef587fe926b8e406d1130d501%40%3Cdev.beam.apache.org%3E]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to