reuvenlax commented on code in PR #23861:
URL: https://github.com/apache/beam/pull/23861#discussion_r1008970520


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithWatermarkDelayFn.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.sdk.transforms;
+
+import java.util.Arrays;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement;
+import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
+import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Adds a watermark a fixed delay behind the extracted timestamp of the source 
element. Does not
+ * allow the watermark to pass the timestamp of the source element.
+ */
+@BoundedPerElement
+class WithWatermarkDelayFn<T> extends DoFn<T, T> {
+
+  private final SerializableFunction<T, Instant> fn;
+  private final Duration watermarkDelay;
+
+  WithWatermarkDelayFn(SerializableFunction<T, Instant> fn, Duration 
watermarkDelay) {
+    this.fn = fn;
+    this.watermarkDelay = watermarkDelay;
+  }
+
+  @ProcessElement
+  public void processElement(
+      @Element T input,
+      @Timestamp Instant inputTimestamp,
+      RestrictionTracker<Integer, Integer> restrictionTracker,
+      OutputReceiver<T> output,
+      ManualWatermarkEstimator<Instant> estimator) {
+    Instant ts = fn.apply(input);
+    output.outputWithTimestamp(input, ts);

Review Comment:
   What if ts < watermark - watermarkDelay?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithWatermarkDelayFn.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.sdk.transforms;
+
+import java.util.Arrays;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement;
+import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
+import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Adds a watermark a fixed delay behind the extracted timestamp of the source 
element. Does not
+ * allow the watermark to pass the timestamp of the source element.
+ */
+@BoundedPerElement
+class WithWatermarkDelayFn<T> extends DoFn<T, T> {
+
+  private final SerializableFunction<T, Instant> fn;
+  private final Duration watermarkDelay;
+
+  WithWatermarkDelayFn(SerializableFunction<T, Instant> fn, Duration 
watermarkDelay) {
+    this.fn = fn;
+    this.watermarkDelay = watermarkDelay;
+  }
+
+  @ProcessElement
+  public void processElement(
+      @Element T input,
+      @Timestamp Instant inputTimestamp,
+      RestrictionTracker<Integer, Integer> restrictionTracker,
+      OutputReceiver<T> output,
+      ManualWatermarkEstimator<Instant> estimator) {
+    Instant ts = fn.apply(input);
+    output.outputWithTimestamp(input, ts);
+    Instant watermark = ts.minus(watermarkDelay);
+    // Don't advance the watermark past the input element timestamp.
+    watermark = Collections.min(Arrays.asList(watermark, inputTimestamp));
+    if (watermark.isAfter(estimator.currentWatermark())) {
+      estimator.setWatermark(watermark);

Review Comment:
   I'm not sure this really does what you want. There is no guarantee on the 
order of input elements and watermarks are monotonic.. If (to take a worst 
case) the elements show up in reverse timestamp order, you'll lock the 
watermark in at the value of that first element, and all the subsequent 
setWatermark calls will be noops (i.e. the elements being output will be late 
data again).
   
   What usually protects against this is the fact that the unprocessed elements 
upstream hold the watermark back. However in this case you don't have that, 
since the source watermark is effectively in a different time domain.
   
   I'm not sure whether there is a correct way to change the time domain mid 
stream - it may always require being done at the source. 



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithWatermarkDelayFn.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.sdk.transforms;
+
+import java.util.Arrays;
+import java.util.Collections;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement;
+import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
+import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+@BoundedPerElement
+class WithWatermarkDelayFn<T> extends DoFn<T, T> {
+
+  private final SerializableFunction<T, Instant> fn;
+  private final Duration watermarkDelay;
+
+  WithWatermarkDelayFn(SerializableFunction<T, Instant> fn, Duration 
watermarkDelay) {
+    this.fn = fn;
+    this.watermarkDelay = watermarkDelay;
+  }
+
+  @ProcessElement
+  public void processElement(
+      @Element T input,
+      @Timestamp Instant inputTimestamp,
+      RestrictionTracker<Integer, Integer> restrictionTracker,
+      OutputReceiver<T> output,
+      ManualWatermarkEstimator<Instant> estimator) {

Review Comment:
   Yes - you can also use stateful dofn



-- 
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]

Reply via email to