dpcollins-google commented on code in PR #23861:
URL: https://github.com/apache/beam/pull/23861#discussion_r1006978707


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java:
##########
@@ -68,58 +70,109 @@
    * }</pre>
    */
   public static <T> WithTimestamps<T> of(SerializableFunction<T, Instant> fn) {
-    return new WithTimestamps<>(fn, Duration.ZERO);
+    return new WithTimestamps<>(fn, null, null);
   }
 
   ///////////////////////////////////////////////////////////////////
 
   private final SerializableFunction<T, Instant> fn;
-  private final Duration allowedTimestampSkew;
+  // If allowedTimestampSkew is not null, use the deprecated behavior which 
emits data as late.
+  private final @Nullable Duration allowedTimestampSkew;
+  // If watermarkDelay is not null, emit data within the watermark delay bound 
as on time.
+  private final @Nullable Duration watermarkDelay;
 
-  private WithTimestamps(SerializableFunction<T, Instant> fn, Duration 
allowedTimestampSkew) {
+  private WithTimestamps(
+      SerializableFunction<T, Instant> fn,
+      @Nullable Duration allowedTimestampSkew,
+      @Nullable Duration watermarkDelay) {
+    checkArgument(
+        allowedTimestampSkew == null || watermarkDelay == null, "Both delays 
cannot be non-null.");

Review Comment:
   its private, I could write 3 constructors one for each use case where each 
is called once, but that feels like overkill



##########
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> {

Review Comment:
   Its an implementation detail (package private) with not much implementation, 
I'm not sure this is necessary



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