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 2022/07/11 19:05:45 UTC

[GitHub] [beam] johnjcasey commented on a diff in pull request #22216: BEAM-13592 Add getOrderingKey in o.a.b.sdk.io.gcp.pubsub.PubsubMessage

johnjcasey commented on code in PR #22216:
URL: https://github.com/apache/beam/pull/22216#discussion_r918260784


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java:
##########
@@ -524,6 +524,16 @@ public static Read<PubsubMessage> readMessagesWithAttributesAndMessageId() {
         .build();
   }
 
+  /**
+   * Returns A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream. The
+   * messages will contain a {@link PubsubMessage#getPayload() payload}, {@link
+   * PubsubMessage#getAttributeMap() attributes}, along with the {@link PubsubMessage#getMessageId()
+   * messageId} and {PubsubMessage#getOrderingKey() orderingKey} from PubSub.
+   */
+  public static Read<PubsubMessage> readMessagesWithAllAttributesAndMessageIdAndOrderingKey() {

Review Comment:
   Can we remove "all" from the message name to keep it consistent with the above method?
   Also, based on the above method, do we need .setNeedsAttributes(true) here?
   Finally, should we name the coder PubsubMessageCoder, given that the above method calls it PubsubMessageWithAttributesAndMessageIdCoder?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageCoder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.io.gcp.pubsub;
+
+import com.google.protobuf.Timestamp;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** A coder for PubsubMessage including all fields of a PubSub message from server. */
+@SuppressWarnings({
+  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class PubsubMessageCoder extends CustomCoder<PubsubMessage> {
+  // A message's payload cannot be null
+  private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
+  // A message's attributes can be null.
+  private static final Coder<Map<String, String>> ATTRIBUTES_CODER =

Review Comment:
   can the attribute map be null, or can elements within the map be null?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageCoder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.io.gcp.pubsub;
+
+import com.google.protobuf.Timestamp;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** A coder for PubsubMessage including all fields of a PubSub message from server. */
+@SuppressWarnings({
+  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class PubsubMessageCoder extends CustomCoder<PubsubMessage> {
+  // A message's payload cannot be null
+  private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
+  // A message's attributes can be null.
+  private static final Coder<Map<String, String>> ATTRIBUTES_CODER =
+      NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+  // A message's messageId cannot be null
+  private static final Coder<String> MESSAGE_ID_CODER = StringUtf8Coder.of();
+  // A message's publish time, populated by server
+  private static final Coder<Timestamp> PUBLISH_TIME_CODER = ProtoCoder.of(Timestamp.class);
+  // A message's ordering key can be null
+  private static final Coder<String> ORDERING_KEY_CODER = NullableCoder.of(StringUtf8Coder.of());
+
+  public static Coder<PubsubMessage> of(TypeDescriptor<PubsubMessage> ignored) {
+    return of();
+  }
+
+  public static PubsubMessageCoder of() {
+    return new PubsubMessageCoder();
+  }
+
+  @Override
+  public void encode(PubsubMessage value, OutputStream outStream) throws IOException {
+    PAYLOAD_CODER.encode(value.getPayload(), outStream);
+    ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream);
+    MESSAGE_ID_CODER.encode(value.getMessageId(), outStream);
+    // TODO(discuss what to do with publish_time field)

Review Comment:
   What needs to be discussed here?



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