scwhittle commented on code in PR #31608: URL: https://github.com/apache/beam/pull/31608#discussion_r2063270829
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFn.java: ########## @@ -179,6 +201,16 @@ public void process( null); reportedLineage = topic; } + if (!usesOrderingKey && !Strings.isNullOrEmpty(message.getOrderingKey())) { + if (!logOrderingKeyUnconfigured) { + LOG.warn( + "Encountered Pubsub message with ordering key but this sink was not configured to " + + "retain ordering keys, so they will be dropped. Please set #withOrderingKeys()."); + + logOrderingKeyUnconfigured = true; Review Comment: I think this might show up as racy with thread sanitizers. Maybe just mark it volatile to avoid that? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java: ########## @@ -220,6 +234,16 @@ public void populateDisplayData(DisplayData.Builder builder) { /** Publish messages to Pubsub in batches. */ @VisibleForTesting static class WriterFn extends DoFn<Iterable<OutgoingMessage>, Void> { + private class OutgoingData { + int bytes; + List<OutgoingMessage> messages; + + OutgoingData() { + this.bytes = 0; + this.messages = new ArrayList<>(publishBatchSize); Review Comment: I don't think we want to reserve full batch size. if we have unique ordering keys we're adding 1K overhead to each one. Seems ok just to use normal array list growth to handle appending ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java: ########## @@ -1710,51 +1735,44 @@ public void startBundle(StartBundleContext c) throws IOException { public void processElement(@Element PubsubMessage message, @Timestamp Instant timestamp) throws IOException, SizeLimitExceededException { // Validate again here just as a sanity check. + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 PreparePubsubWriteDoFn.validatePubsubMessageSize(message, maxPublishBatchByteSize); - byte[] payload = message.getPayload(); - int messageSize = payload.length; + // NOTE: The record id is always null. + final OutgoingMessage msg = + OutgoingMessage.of(message, timestamp.getMillis(), null, message.getTopic()); + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 + final int messageSize = msg.getMessage().getData().size(); - PubsubTopic pubsubTopic; + final 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 - OutgoingData currentTopicOutput = - output.computeIfAbsent(pubsubTopic, t -> new OutgoingData()); - if (currentTopicOutput.messages.size() >= maxPublishBatchSize - || (!currentTopicOutput.messages.isEmpty() - && (currentTopicOutput.bytes + messageSize) >= maxPublishBatchByteSize)) { - publish(pubsubTopic, currentTopicOutput.messages); - currentTopicOutput.messages.clear(); - currentTopicOutput.bytes = 0; + pubsubTopic = PubsubTopic.fromPath(Preconditions.checkArgumentNotNull(msg.topic())); } - Map<String, String> attributes = message.getAttributeMap(); - String orderingKey = message.getOrderingKey(); - - com.google.pubsub.v1.PubsubMessage.Builder msgBuilder = - com.google.pubsub.v1.PubsubMessage.newBuilder() - .setData(ByteString.copyFrom(payload)) - .putAllAttributes(attributes); - - if (orderingKey != null) { - msgBuilder.setOrderingKey(orderingKey); + // Checking before adding the message stops us from violating max batch size or bytes + String orderingKey = getPublishWithOrderingKey() ? msg.getMessage().getOrderingKey() : ""; + final OutgoingData currentTopicAndOrderingKeyOutput = + output.computeIfAbsent(KV.of(pubsubTopic, orderingKey), t -> new OutgoingData()); + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 + if (currentTopicAndOrderingKeyOutput.messages.size() >= maxPublishBatchSize + || (!currentTopicAndOrderingKeyOutput.messages.isEmpty() + && (currentTopicAndOrderingKeyOutput.bytes + messageSize) + >= maxPublishBatchByteSize)) { + publish(pubsubTopic, currentTopicAndOrderingKeyOutput.messages); + currentTopicAndOrderingKeyOutput.messages.clear(); + currentTopicAndOrderingKeyOutput.bytes = 0; } - // NOTE: The record id is always null. - currentTopicOutput.messages.add( - OutgoingMessage.of( - msgBuilder.build(), timestamp.getMillis(), null, message.getTopic())); - currentTopicOutput.bytes += messageSize; + currentTopicAndOrderingKeyOutput.messages.add(msg); + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 Review Comment: think you could remove this reference, others seem sufficient ########## sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java: ########## @@ -303,6 +307,7 @@ public void sendMoreThanOneBatchByByteSize() throws IOException { TIMESTAMP_ATTRIBUTE, ID_ATTRIBUTE, NUM_SHARDS, + false, Review Comment: add a test to this file with true? verify that differnet ordering keys are published separately ########## sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteIT.java: ########## @@ -102,4 +109,55 @@ public void testBoundedWriteMessageWithAttributes() { .apply(PubsubIO.writeMessages().to(testTopic.getPath())); pipeline.run(); } + + @Test + public void testBoundedWriteMessageWithAttributesAndMessageIdAndOrderingKey() throws IOException { + TopicPath testTopicPath = + PubsubClient.topicPathFromName( + project, "pubsub-write-ordering-key-" + Instant.now().getMillis()); + pubsubClient.createTopic(testTopicPath); + SubscriptionPath testSubscriptionPath = + pubsubClient.createRandomSubscription( + PubsubClient.projectPathFromId(project), testTopicPath, 10); + + byte[] payload = RandomStringUtils.randomAscii(1_000_000).getBytes(StandardCharsets.UTF_8); + Map<String, String> attributes = + ImmutableMap.<String, String>builder() + .put("id", "1") + .put("description", RandomStringUtils.randomAscii(100)) + .build(); + + PubsubMessage outgoingMessage = + new PubsubMessage(payload, attributes, "test_message", "111222"); Review Comment: would be a better test to publish a different key as well and verify they aren't batched ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java: ########## @@ -1710,51 +1735,44 @@ public void startBundle(StartBundleContext c) throws IOException { public void processElement(@Element PubsubMessage message, @Timestamp Instant timestamp) throws IOException, SizeLimitExceededException { // Validate again here just as a sanity check. + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 Review Comment: Add a short summary of issue so it's easier to eyeball in code here ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java: ########## @@ -1710,51 +1735,44 @@ public void startBundle(StartBundleContext c) throws IOException { public void processElement(@Element PubsubMessage message, @Timestamp Instant timestamp) throws IOException, SizeLimitExceededException { // Validate again here just as a sanity check. + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 PreparePubsubWriteDoFn.validatePubsubMessageSize(message, maxPublishBatchByteSize); - byte[] payload = message.getPayload(); - int messageSize = payload.length; + // NOTE: The record id is always null. + final OutgoingMessage msg = + OutgoingMessage.of(message, timestamp.getMillis(), null, message.getTopic()); + // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 Review Comment: ditto ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageSchemaCoder.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; + +/** + * Provides a {@link SchemaCoder} for {@link PubsubMessage}, including the topic and all fields of a + * PubSub message from server. + * + * <p>{@link SchemaCoder} is used so that fields can be added in the future without breaking update + * compatibility. Review Comment: should we add a comment here that this should be preferred in new code to PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.java could also update comment for that class to steer users to this new coder. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageSchemaCoder.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; + +/** + * Provides a {@link SchemaCoder} for {@link PubsubMessage}, including the topic and all fields of a + * PubSub message from server. + * + * <p>{@link SchemaCoder} is used so that fields can be added in the future without breaking update + * compatibility. + */ +public class PubsubMessageSchemaCoder { + private static final Schema PUBSUB_MESSAGE_SCHEMA = + Schema.builder() + .addByteArrayField("payload") + .addNullableStringField("topic") + .addNullableMapField("attributes", Schema.FieldType.STRING, Schema.FieldType.STRING) + .addNullableStringField("message_id") + .addNullableStringField("ordering_key") + .build(); + + private static final SerializableFunction<PubsubMessage, Row> TO_ROW = + (PubsubMessage message) -> { + Map<String, Object> fieldValues = new HashMap<>(); + fieldValues.put("payload", message.getPayload()); + + String topic = message.getTopic(); + if (topic != null) { + fieldValues.put("topic", topic); + } + Map<String, String> attributeMap = message.getAttributeMap(); + if (attributeMap != null) { + fieldValues.put("attributes", attributeMap); + } + String messageId = message.getMessageId(); + if (messageId != null) { + fieldValues.put("message_id", messageId); + } + String orderingKey = message.getOrderingKey(); + if (orderingKey != null) { + fieldValues.put("ordering_key", orderingKey); + } + return Row.withSchema(PUBSUB_MESSAGE_SCHEMA).withFieldValues(fieldValues).build(); Review Comment: nit seems like it would be more efficient to add the fields to the Row.Builder one by one instead of building up the map. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageSchemaCoder.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; + +/** + * Provides a {@link SchemaCoder} for {@link PubsubMessage}, including the topic and all fields of a + * PubSub message from server. + * + * <p>{@link SchemaCoder} is used so that fields can be added in the future without breaking update + * compatibility. + */ +public class PubsubMessageSchemaCoder { + private static final Schema PUBSUB_MESSAGE_SCHEMA = + Schema.builder() + .addByteArrayField("payload") + .addNullableStringField("topic") + .addNullableMapField("attributes", Schema.FieldType.STRING, Schema.FieldType.STRING) + .addNullableStringField("message_id") + .addNullableStringField("ordering_key") + .build(); + + private static final SerializableFunction<PubsubMessage, Row> TO_ROW = + (PubsubMessage message) -> { + Map<String, Object> fieldValues = new HashMap<>(); + fieldValues.put("payload", message.getPayload()); + + String topic = message.getTopic(); + if (topic != null) { + fieldValues.put("topic", topic); + } + Map<String, String> attributeMap = message.getAttributeMap(); + if (attributeMap != null) { + fieldValues.put("attributes", attributeMap); + } + String messageId = message.getMessageId(); + if (messageId != null) { + fieldValues.put("message_id", messageId); + } + String orderingKey = message.getOrderingKey(); + if (orderingKey != null) { + fieldValues.put("ordering_key", orderingKey); + } + return Row.withSchema(PUBSUB_MESSAGE_SCHEMA).withFieldValues(fieldValues).build(); + }; + + private static final SerializableFunction<Row, PubsubMessage> FROM_ROW = + (Row row) -> { + PubsubMessage message = + new PubsubMessage( + Preconditions.checkNotNull(row.getBytes("payload")), + row.getMap("attributes"), + row.getString("message_id"), + row.getString("ordering_key")); + + String topic = row.getString("topic"); Review Comment: nullable -- 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-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org