scwhittle commented on code in PR #26063: URL: https://github.com/apache/beam/pull/26063#discussion_r1171078471
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java: ########## @@ -0,0 +1,164 @@ +/* + * 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.runners.dataflow.worker; + +import static org.apache.beam.runners.dataflow.util.Structs.getString; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.runners.dataflow.util.CloudObject; +import org.apache.beam.runners.dataflow.util.PropertyNames; +import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink; +import org.apache.beam.runners.dataflow.worker.windmill.Pubsub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) +}) +public class PubsubDynamicSink extends Sink<WindowedValue<PubsubMessage>> { + private final String timestampLabel; + private final String idLabel; + private final StreamingModeExecutionContext context; + // Function used to convert PCollection elements to PubsubMessage objects. Review Comment: remove comment? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java: ########## @@ -0,0 +1,164 @@ +/* + * 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.runners.dataflow.worker; + +import static org.apache.beam.runners.dataflow.util.Structs.getString; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.runners.dataflow.util.CloudObject; +import org.apache.beam.runners.dataflow.util.PropertyNames; +import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink; +import org.apache.beam.runners.dataflow.worker.windmill.Pubsub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) +}) +public class PubsubDynamicSink extends Sink<WindowedValue<PubsubMessage>> { + private final String timestampLabel; + private final String idLabel; + private final StreamingModeExecutionContext context; + // Function used to convert PCollection elements to PubsubMessage objects. + + PubsubDynamicSink(String timestampLabel, String idLabel, StreamingModeExecutionContext context) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.context = context; + } + + /** A {@link SinkFactory.Registrar} for pubsub sinks. */ + @AutoService(SinkFactory.Registrar.class) + public static class Registrar implements SinkFactory.Registrar { + + @Override + public Map<String, SinkFactory> factories() { + PubsubDynamicSink.Factory factory = new Factory(); + return ImmutableMap.of( + "PubsubDynamicSink", + factory, + "org.apache.beam.runners.dataflow.worker.PubsubDynamicSink", + factory); + } + } + + static class Factory implements SinkFactory { + @Override + public PubsubDynamicSink create( + CloudObject spec, + Coder<?> coder, + @Nullable PipelineOptions options, + @Nullable DataflowExecutionContext executionContext, + DataflowOperationContext operationContext) + throws Exception { + String timestampLabel = getString(spec, PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, ""); + String idLabel = getString(spec, PropertyNames.PUBSUB_ID_ATTRIBUTE, ""); + + return new PubsubDynamicSink( + timestampLabel, + idLabel, + (StreamingModeExecutionContext) + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull(executionContext)); + } + } + + @Override + public Sink.SinkWriter<WindowedValue<PubsubMessage>> writer() { + return new PubsubDynamicSink.PubsubWriter(); + } + + class PubsubWriter implements Sink.SinkWriter<WindowedValue<PubsubMessage>> { + private final Map<String, Windmill.PubSubMessageBundle.Builder> outputBuilders; + private final ByteStringOutputStream stream; // Kept across adds for buffer reuse. + + PubsubWriter() { + outputBuilders = Maps.newHashMap(); + stream = new ByteStringOutputStream(); + } + + public ByteString getDataFromMessage(PubsubMessage formatted, ByteStringOutputStream stream) + throws IOException { + Pubsub.PubsubMessage.Builder pubsubMessageBuilder = + Pubsub.PubsubMessage.newBuilder().setData(ByteString.copyFrom(formatted.getPayload())); + Map<String, String> attributeMap = formatted.getAttributeMap(); + if (attributeMap != null) { + pubsubMessageBuilder.putAllAttributes(attributeMap); + } + pubsubMessageBuilder.build().writeTo(stream); + return stream.toByteStringAndReset(); + } + + public void close(Windmill.PubSubMessageBundle.Builder outputBuilder) throws IOException { + context.getOutputBuilder().addPubsubMessages(outputBuilder); + outputBuilder.clear(); + } + + @Override + public long add(WindowedValue<PubsubMessage> data) throws IOException { + String dataTopic = + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( Review Comment: import static? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java: ########## @@ -268,9 +309,9 @@ public void startBundle(StartBundleContext c) throws Exception { public void processElement(ProcessContext c) throws Exception { List<OutgoingMessage> pubsubMessages = new ArrayList<>(publishBatchSize); int bytes = 0; - for (OutgoingMessage message : c.element().getValue()) { + for (OutgoingMessage message : c.element()) { if (!pubsubMessages.isEmpty() - && bytes + message.message().getData().size() > publishBatchBytes) { + && bytes + message.getMessage().getData().size() > publishBatchBytes) { Review Comment: this limit is currently across all topics but could be per-topic ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java: ########## @@ -268,9 +309,9 @@ public void startBundle(StartBundleContext c) throws Exception { public void processElement(ProcessContext c) throws Exception { List<OutgoingMessage> pubsubMessages = new ArrayList<>(publishBatchSize); int bytes = 0; - for (OutgoingMessage message : c.element().getValue()) { + for (OutgoingMessage message : c.element()) { if (!pubsubMessages.isEmpty() - && bytes + message.message().getData().size() > publishBatchBytes) { + && bytes + message.getMessage().getData().size() > publishBatchBytes) { Review Comment: another option would be to change so we group by topic+shard instead of just shard that would keep this simple here, and avoid sequential blocking publishes ########## website/www/site/layouts/shortcodes/flink_python_pipeline_options.html: ########## @@ -133,6 +132,11 @@ <td>Sets the behavior of reusing objects.</td> <td>Default: <code>false</code></td> </tr> +<tr> + <td><code>operator_chaining</code></td> Review Comment: this file still seems there ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java: ########## @@ -1263,45 +1292,52 @@ public Write<T> withPubsubRootUrl(String pubsubRootUrl) { @Override public PDone expand(PCollection<T> input) { - if (getTopicProvider() == null) { - throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform"); + if (getTopicProvider() == null && !getDynamicDestinations()) { + throw new IllegalStateException( + "need to set the topic of a PubsubIO.Write transform if not using " + + "dynamic topic destinations."); } + SerializableFunction<ValueInSingleWindow<T>, PubsubIO.PubsubTopic> topicFunction = + getTopicFunction(); + if (topicFunction == null && getTopicProvider() != null) { + topicFunction = v -> getTopicProvider().get(); + } + int maxMessageSize = PUBSUB_MESSAGE_MAX_TOTAL_SIZE; + if (input.isBounded() == PCollection.IsBounded.BOUNDED) { + maxMessageSize = + Math.min( + maxMessageSize, + MoreObjects.firstNonNull( + getMaxBatchBytesSize(), MAX_PUBLISH_BATCH_BYTE_SIZE_DEFAULT)); + } + PCollection<PubsubMessage> pubsubMessages = + input.apply( + ParDo.of(new PreparePubsubWriteDoFn<>(getFormatFn(), topicFunction, maxMessageSize))); switch (input.isBounded()) { case BOUNDED: - input.apply( + pubsubMessages.apply( Review Comment: enforce no dynamic destinations in batch? Or I think the BoundedWriter needs to group output by topic as publish is just all messages to single topic currently. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java: ########## @@ -0,0 +1,163 @@ +/* + * 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.runners.dataflow.worker; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.when; + +import avro.shaded.com.google.common.collect.Lists; Review Comment: don't use shaded version? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java: ########## @@ -0,0 +1,164 @@ +/* + * 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.runners.dataflow.worker; + +import static org.apache.beam.runners.dataflow.util.Structs.getString; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.runners.dataflow.util.CloudObject; +import org.apache.beam.runners.dataflow.util.PropertyNames; +import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink; +import org.apache.beam.runners.dataflow.worker.windmill.Pubsub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class PubsubDynamicSink extends Sink<WindowedValue<PubsubMessage>> { + private final String timestampLabel; + private final String idLabel; + private final StreamingModeExecutionContext context; + // Function used to convert PCollection elements to PubsubMessage objects. + + PubsubDynamicSink(String timestampLabel, String idLabel, StreamingModeExecutionContext context) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.context = context; + } + + /** A {@link SinkFactory.Registrar} for pubsub sinks. */ + @AutoService(SinkFactory.Registrar.class) + public static class Registrar implements SinkFactory.Registrar { + + @Override + public Map<String, SinkFactory> factories() { + PubsubDynamicSink.Factory factory = new Factory(); + return ImmutableMap.of( + "PubsubDynamicSink", + factory, + "org.apache.beam.runners.dataflow.worker.PubsubDynamicSink", + factory); + } + } + + public static class Factory implements SinkFactory { + @Override + public PubsubDynamicSink create( + CloudObject spec, + Coder<?> coder, + @Nullable PipelineOptions options, + @Nullable DataflowExecutionContext executionContext, + DataflowOperationContext operationContext) + throws Exception { + String timestampLabel = getString(spec, PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, ""); + String idLabel = getString(spec, PropertyNames.PUBSUB_ID_ATTRIBUTE, ""); + + return new PubsubDynamicSink( + timestampLabel, idLabel, (StreamingModeExecutionContext) executionContext); + } + } + + @Override + public Sink.SinkWriter<WindowedValue<PubsubMessage>> writer() { + return new PubsubDynamicSink.PubsubWriter(); + } + + class PubsubWriter implements Sink.SinkWriter<WindowedValue<PubsubMessage>> { + private Map<String, Windmill.PubSubMessageBundle.Builder> outputBuilders; + private ByteStringOutputStream stream; // Kept across adds for buffer reuse. + + PubsubWriter() { + outputBuilders = Maps.newHashMap(); + stream = new ByteStringOutputStream(); + } + + public ByteString getDataFromMessage(PubsubMessage formatted, ByteStringOutputStream stream) + throws IOException { + Pubsub.PubsubMessage.Builder pubsubMessageBuilder = + Pubsub.PubsubMessage.newBuilder().setData(ByteString.copyFrom(formatted.getPayload())); Review Comment: I wonder if we should separately look into switching to ByteString within beam's PubsubMessage. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java: ########## @@ -479,13 +523,78 @@ public ValueProvider<TopicPath> getTopicProvider() { @Override public PDone expand(PCollection<PubsubMessage> input) { - return input - .apply( - "Output Serialized PubsubMessage Proto", - MapElements.into(new TypeDescriptor<byte[]>() {}) - .via(new PubsubMessages.ParsePayloadAsPubsubMessageProto())) - .setCoder(ByteArrayCoder.of()) - .apply(new PubsubSink(this)); + if (topic != null) { + return input + .apply( + "Output Serialized PubsubMessage Proto", + MapElements.into(new TypeDescriptor<byte[]>() {}) + .via(new PubsubMessages.ParsePayloadAsPubsubMessageProto())) + .setCoder(ByteArrayCoder.of()) + .apply(new PubsubSink(this)); + } else { + // dynamic destinations. + return input + .apply( + "WithDynamicKeys", Review Comment: nit: WithDynamicTopics? DynamicDestinations? keys seems implementation detail -- 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]
