scwhittle commented on code in PR #26063:
URL: https://github.com/apache/beam/pull/26063#discussion_r1172252496
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java:
##########
@@ -668,6 +672,74 @@ public void testReadMessagesWithCoderAndParseFn() {
List<String> outputs = ImmutableList.of("foo", "bar");
PAssert.that(read).containsInAnyOrder(outputs);
- readPipeline.run();
+ pipeline.run();
+ }
+
+ @Test
+ public void testDynamicTopicsBounded() throws IOException {
+ testDynamicTopics(true);
+ }
+
+ @Test
+ public void testDynamicTopicsUnbounded() throws IOException {
+ testDynamicTopics(false);
+ }
+
+ public void testDynamicTopics(boolean isBounded) throws IOException {
+ List<OutgoingMessage> outgoing =
Review Comment:
nit: name expectedOutgoing, it wasn't obvious that the test factory for
publish took these as expectations when created. I think naming would help.
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java:
##########
@@ -1355,9 +1377,10 @@ public void populateDisplayData(DisplayData.Builder
builder) {
* <p>Public so can be suppressed by runners.
*/
public class PubsubBoundedWriter extends DoFn<PubsubMessage, Void> {
- private transient List<OutgoingMessage> output;
+ private transient Map<PubsubTopic, List<OutgoingMessage>> output;
Review Comment:
could have a single map by topic, reduces lookup and prevents possibility of
entry in one map and not the other.
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java:
##########
@@ -1390,11 +1413,25 @@ public void processElement(@Element PubsubMessage
message, @Timestamp Instant ti
PreparePubsubWriteDoFn.validatePubsubMessageSize(message,
maxPublishBatchSize);
byte[] payload = message.getPayload();
int messageSize = payload.length;
+
+ PubsubTopic pubsubTopic;
+ if (getTopicProvider() != null) {
+ pubsubTopic = getTopicProvider().get();
+ } else {
+ pubsubTopic =
+
PubsubTopic.fromPath(Preconditions.checkArgumentNotNull(message.getTopic()));
+ }
// Checking before adding the message stops us from violating max
batch size or bytes
- if (output.size() >= maxPublishBatchSize
- || (!output.isEmpty()
- && (currentOutputBytes + messageSize) >=
maxPublishBatchByteSize)) {
- publish();
+ List<OutgoingMessage> currentTopicOutput =
+ output.computeIfAbsent(pubsubTopic, t -> Lists.newArrayList());
+ long currentTopicOutputBytes =
currentOutputBytes.getOrDefault(pubsubTopic, 0L);
+ if (currentTopicOutput.size() >= maxPublishBatchSize
+ || (!currentTopicOutput.isEmpty()
+ && (currentTopicOutputBytes + messageSize) >=
maxPublishBatchByteSize)) {
+ publish(pubsubTopic, currentTopicOutput);
+ currentTopicOutput.clear();
+ currentTopicOutputBytes = 0;
+ currentOutputBytes.put(pubsubTopic, 0L);
Review Comment:
think the single map would make this simpler to since you would be updating
the value in the entry directly
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithTopicCoder.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 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.values.TypeDescriptor;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** A coder for PubsubMessage including the topic from the PubSub server. */
+@SuppressWarnings({
+ "nullness" // TODO(https://github.com/apache/beam/issues/20497)
Review Comment:
can this be removed?
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithTopicCoder.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 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.values.TypeDescriptor;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** A coder for PubsubMessage including the topic from the PubSub server. */
+@SuppressWarnings({
+ "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class PubsubMessageWithTopicCoder 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
Review Comment:
"cannot be null" is inconsistent with using NullableCoder
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithTopicCoder.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 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.values.TypeDescriptor;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** A coder for PubsubMessage including the topic from the PubSub server. */
+@SuppressWarnings({
+ "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class PubsubMessageWithTopicCoder 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 =
NullableCoder.of(StringUtf8Coder.of());
+
+ private static final Coder<String> TOPIC_CODER =
NullableCoder.of(StringUtf8Coder.of());
+
+ public static Coder<PubsubMessage> of(TypeDescriptor<PubsubMessage> ignored)
{
+ return of();
+ }
+
+ public static PubsubMessageWithAttributesAndMessageIdCoder of() {
+ return new PubsubMessageWithAttributesAndMessageIdCoder();
+ }
+
+ @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);
+ TOPIC_CODER.encode(value.getTopic(), outStream);
+ }
+
+ @Override
+ public PubsubMessage decode(InputStream inStream) throws IOException {
+ byte[] payload = PAYLOAD_CODER.decode(inStream);
+ Map<String, String> attributes = ATTRIBUTES_CODER.decode(inStream);
Review Comment:
add Nullable annotation?
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java:
##########
@@ -1410,31 +1447,34 @@ public void processElement(@Element PubsubMessage
message, @Timestamp Instant ti
}
// NOTE: The record id is always null.
- output.add(
+ currentTopicOutput.add(
OutgoingMessage.of(
msgBuilder.build(), timestamp.getMillis(), null,
message.getTopic()));
- currentOutputBytes += messageSize;
+ currentOutputBytes.put(pubsubTopic, currentTopicOutputBytes +
messageSize);
}
@FinishBundle
public void finishBundle() throws IOException {
- if (!output.isEmpty()) {
- publish();
- }
+ publish();
output = null;
- currentOutputBytes = 0;
+ currentOutputBytes = null;
pubsubClient.close();
pubsubClient = null;
}
private void publish() throws IOException {
- PubsubTopic topic = getTopicProvider().get();
+ for (Map.Entry<PubsubTopic, List<OutgoingMessage>> entry :
output.entrySet()) {
Review Comment:
use visitor instead of iterator?
forEach(e -> publish(e.getKey(), e.getValue());
--
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]