You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/07/15 18:16:11 UTC

[GitHub] [beam] y1chi commented on a change in pull request #15137: [BEAM-2914] Add TimestampPrefixingWindowCoder as beam:coder:custom_window:v1.

y1chi commented on a change in pull request #15137:
URL: https://github.com/apache/beam/pull/15137#discussion_r670701960



##########
File path: model/pipeline/src/main/proto/beam_runner_api.proto
##########
@@ -939,6 +939,15 @@ message StandardCoders {
     // Experimental.
     STATE_BACKED_ITERABLE = 9 [(beam_urn) = "beam:coder:state_backed_iterable:v1"];
 
+
+    // Encodes an arbitrary user defined window, the coder extracts the required
+    // maxTimestamp(inclusive) parameter for the window and prefix it to the
+    // bytes encoded from the window by using user defined and registered window
+    // coder.
+    //
+    // Components: Coder for the custom window type.
+    CUSTOM_WINDOW = 16 [(beam_urn) = "beam:coder:custom_window:v1"];

Review comment:
       Done.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
##########
@@ -345,6 +347,11 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co
       byte[] shardId = ((String) kvMap.get("shardId")).getBytes(StandardCharsets.ISO_8859_1);
       return ShardedKey.of(
           convertValue(kvMap.get("key"), coderSpec.getComponents().get(0), keyCoder), shardId);
+    } else if (s.equals(getUrn(StandardCoders.Enum.CUSTOM_WINDOW))) {
+      Map<String, Object> kvMap = (Map<String, Object>) value;
+      Instant end = new Instant(((Number) kvMap.get("end")).longValue());
+      Duration span = Duration.millis(((Number) kvMap.get("span")).longValue());
+      return new IntervalWindow(end.minus(span), span);

Review comment:
       Done.

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoder.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.coders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+
+/**
+ * A {@link TimestampPrefixingWindowCoder} wraps arbitrary user custom window coder. While encoding
+ * the custom window type, it extracts the maxTimestamp(inclusive) of the window and prefix it to
+ * the encoded bytes of the window using the user custom window coder.
+ *
+ * @param <T> The custom window type.
+ */
+public class TimestampPrefixingWindowCoder<T> extends StructuredCoder<T> {
+  private final Coder<T> windowCoder;
+
+  public static <T> TimestampPrefixingWindowCoder<T> of(Coder<T> windowCoder) {
+    return new TimestampPrefixingWindowCoder<>(windowCoder);
+  }
+
+  TimestampPrefixingWindowCoder(Coder<T> windowCoder) {
+    this.windowCoder = windowCoder;
+  }
+
+  @Override
+  public void encode(T value, OutputStream outStream) throws CoderException, IOException {
+    BoundedWindow window = (BoundedWindow) value;
+    if (window == null) {
+      throw new CoderException("Cannot encode null window");
+    }
+    InstantCoder.of().encode(window.maxTimestamp(), outStream);
+    windowCoder.encode(value, outStream);
+  }
+
+  @Override
+  public T decode(InputStream inStream) throws CoderException, IOException {
+    InstantCoder.of().decode(inStream);
+    return windowCoder.decode(inStream);
+  }
+
+  @Override
+  public List<? extends Coder<?>> getCoderArguments() {
+    return Lists.newArrayList(windowCoder);
+  }
+
+  @Override
+  public void verifyDeterministic() throws NonDeterministicException {

Review comment:
       Done.

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.coders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+public class TimestampPrefixingWindowCoderTest {
+
+  private static class CustomWindow extends IntervalWindow {
+    private boolean isBig;
+
+    CustomWindow(Instant start, Instant end, boolean isBig) {
+      super(start, end);
+      this.isBig = isBig;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      CustomWindow that = (CustomWindow) o;
+      return super.equals(o) && this.isBig == that.isBig;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(super.hashCode(), isBig);
+    }
+  }
+
+  private static class CustomWindowCoder extends CustomCoder<CustomWindow> {
+
+    private static final CustomWindowCoder INSTANCE = new CustomWindowCoder();
+    private static final Coder<IntervalWindow> INTERVAL_WINDOW_CODER = IntervalWindow.getCoder();
+    private static final VarIntCoder VAR_INT_CODER = VarIntCoder.of();
+
+    public static CustomWindowCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(CustomWindow window, OutputStream outStream) throws IOException {
+      INTERVAL_WINDOW_CODER.encode(window, outStream);
+      VAR_INT_CODER.encode(window.isBig ? 1 : 0, outStream);

Review comment:
       Done.

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.coders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+public class TimestampPrefixingWindowCoderTest {
+
+  private static class CustomWindow extends IntervalWindow {
+    private boolean isBig;
+
+    CustomWindow(Instant start, Instant end, boolean isBig) {
+      super(start, end);
+      this.isBig = isBig;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      CustomWindow that = (CustomWindow) o;
+      return super.equals(o) && this.isBig == that.isBig;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(super.hashCode(), isBig);
+    }
+  }
+
+  private static class CustomWindowCoder extends CustomCoder<CustomWindow> {
+
+    private static final CustomWindowCoder INSTANCE = new CustomWindowCoder();
+    private static final Coder<IntervalWindow> INTERVAL_WINDOW_CODER = IntervalWindow.getCoder();
+    private static final VarIntCoder VAR_INT_CODER = VarIntCoder.of();
+
+    public static CustomWindowCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(CustomWindow window, OutputStream outStream) throws IOException {
+      INTERVAL_WINDOW_CODER.encode(window, outStream);
+      VAR_INT_CODER.encode(window.isBig ? 1 : 0, outStream);
+    }
+
+    @Override
+    public CustomWindow decode(InputStream inStream) throws IOException {
+      IntervalWindow superWindow = INTERVAL_WINDOW_CODER.decode(inStream);
+      boolean isBig = VAR_INT_CODER.decode(inStream) != 0;
+      return new CustomWindow(superWindow.start(), superWindow.end(), isBig);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      INTERVAL_WINDOW_CODER.verifyDeterministic();
+      VAR_INT_CODER.verifyDeterministic();
+    }
+  }
+
+  @Test
+  public void testEncodeAndDecode() throws Exception {
+    List<IntervalWindow> intervalWindowsToTest =
+        Lists.newArrayList(
+            new IntervalWindow(new Instant(0L), new Instant(1L)),
+            new IntervalWindow(new Instant(100L), new Instant(200L)),
+            new IntervalWindow(new Instant(0L), BoundedWindow.TIMESTAMP_MAX_VALUE));
+    TimestampPrefixingWindowCoder<IntervalWindow> coder1 =
+        TimestampPrefixingWindowCoder.of(IntervalWindow.getCoder());
+    for (IntervalWindow window : intervalWindowsToTest) {
+      CoderProperties.coderDecodeEncodeEqual(coder1, window);
+    }
+
+    GlobalWindow globalWindow = GlobalWindow.INSTANCE;
+    TimestampPrefixingWindowCoder<GlobalWindow> coder2 =
+        TimestampPrefixingWindowCoder.of(GlobalWindow.Coder.INSTANCE);
+    CoderProperties.coderDecodeEncodeEqual(coder2, globalWindow);
+
+    List<CustomWindow> customWindowsToTest =
+        Lists.newArrayList(
+            new CustomWindow(new Instant(0L), new Instant(1L), true),
+            new CustomWindow(new Instant(100L), new Instant(200L), false),
+            new CustomWindow(new Instant(0L), BoundedWindow.TIMESTAMP_MAX_VALUE, true));
+    TimestampPrefixingWindowCoder<CustomWindow> coder3 =
+        TimestampPrefixingWindowCoder.of(CustomWindowCoder.of());
+    for (CustomWindow window : customWindowsToTest) {
+      CoderProperties.coderDecodeEncodeEqual(coder3, window);
+    }
+  }

Review comment:
       Done.

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoder.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.coders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+
+/**
+ * A {@link TimestampPrefixingWindowCoder} wraps arbitrary user custom window coder. While encoding
+ * the custom window type, it extracts the maxTimestamp(inclusive) of the window and prefix it to
+ * the encoded bytes of the window using the user custom window coder.
+ *
+ * @param <T> The custom window type.
+ */
+public class TimestampPrefixingWindowCoder<T> extends StructuredCoder<T> {

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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org