reuvenlax commented on code in PR #26063:
URL: https://github.com/apache/beam/pull/26063#discussion_r1170485056


##########
runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java:
##########
@@ -2341,6 +2347,97 @@ public void 
testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IO
     verifyGroupIntoBatchesOverrideBytes(p, true, true);
   }
 
+  @Test
+  public void testPubsubSinkOverride() throws IOException {
+    PipelineOptions options = buildPipelineOptions();
+    List<String> experiments =
+        new ArrayList<>(
+            ImmutableList.of(
+                GcpOptions.STREAMING_ENGINE_EXPERIMENT,
+                GcpOptions.WINDMILL_SERVICE_EXPERIMENT,
+                "use_runner_v2"));
+    DataflowPipelineOptions dataflowOptions = 
options.as(DataflowPipelineOptions.class);
+    dataflowOptions.setExperiments(experiments);
+    dataflowOptions.setStreaming(true);
+    Pipeline p = Pipeline.create(options);
+
+    List<PubsubMessage> testValues =
+        Arrays.asList(
+            new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), 
Collections.emptyMap()));
+    PCollection<PubsubMessage> input =
+        p.apply("CreateValuesBytes", Create.of(testValues))
+            .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
+    input.apply(PubsubIO.writeMessages().to("projects/project/topics/topic"));
+    p.run();
+
+    AtomicBoolean sawPubsubOverride = new AtomicBoolean(false);
+    p.traverseTopologically(
+        new PipelineVisitor.Defaults() {
+
+          @Override
+          public CompositeBehavior enterCompositeTransform(Node node) {

Review Comment:
   done



##########
runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java:
##########
@@ -2341,6 +2347,97 @@ public void 
testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IO
     verifyGroupIntoBatchesOverrideBytes(p, true, true);
   }
 
+  @Test
+  public void testPubsubSinkOverride() throws IOException {
+    PipelineOptions options = buildPipelineOptions();
+    List<String> experiments =
+        new ArrayList<>(
+            ImmutableList.of(
+                GcpOptions.STREAMING_ENGINE_EXPERIMENT,
+                GcpOptions.WINDMILL_SERVICE_EXPERIMENT,
+                "use_runner_v2"));
+    DataflowPipelineOptions dataflowOptions = 
options.as(DataflowPipelineOptions.class);
+    dataflowOptions.setExperiments(experiments);
+    dataflowOptions.setStreaming(true);
+    Pipeline p = Pipeline.create(options);
+
+    List<PubsubMessage> testValues =
+        Arrays.asList(
+            new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), 
Collections.emptyMap()));
+    PCollection<PubsubMessage> input =
+        p.apply("CreateValuesBytes", Create.of(testValues))
+            .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
+    input.apply(PubsubIO.writeMessages().to("projects/project/topics/topic"));
+    p.run();
+
+    AtomicBoolean sawPubsubOverride = new AtomicBoolean(false);
+    p.traverseTopologically(
+        new PipelineVisitor.Defaults() {
+
+          @Override
+          public CompositeBehavior enterCompositeTransform(Node node) {
+            if (node.getTransform() instanceof 
DataflowRunner.StreamingPubsubIOWrite) {
+              sawPubsubOverride.set(true);
+            }
+            return CompositeBehavior.ENTER_TRANSFORM;
+          }
+
+          @Override
+          public void visitPrimitiveTransform(@UnknownKeyFor @NonNull 
@Initialized Node node) {
+            if (node.getTransform() instanceof 
DataflowRunner.StreamingPubsubIOWrite) {
+              sawPubsubOverride.set(true);
+            }
+          }
+        });
+    assertTrue(sawPubsubOverride.get());
+  }
+
+  @Test
+  public void testPubsubSinkDynamicOverride() throws IOException {
+    PipelineOptions options = buildPipelineOptions();
+    List<String> experiments =
+        new ArrayList<>(
+            ImmutableList.of(
+                GcpOptions.STREAMING_ENGINE_EXPERIMENT,
+                GcpOptions.WINDMILL_SERVICE_EXPERIMENT,
+                "use_runner_v2"));
+    DataflowPipelineOptions dataflowOptions = 
options.as(DataflowPipelineOptions.class);
+    dataflowOptions.setExperiments(experiments);
+    dataflowOptions.setStreaming(true);
+    Pipeline p = Pipeline.create(options);
+
+    List<PubsubMessage> testValues =
+        Arrays.asList(
+            new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), 
Collections.emptyMap())
+                .withTopic(""));
+    PCollection<PubsubMessage> input =
+        p.apply("CreateValuesBytes", Create.of(testValues))
+            .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
+    input.apply(PubsubIO.writeMessagesDynamic());
+    p.run();
+
+    AtomicBoolean sawPubsubOverride = new AtomicBoolean(false);
+    p.traverseTopologically(
+        new PipelineVisitor.Defaults() {
+
+          @Override
+          public CompositeBehavior enterCompositeTransform(Node node) {

Review Comment:
   done



##########
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({

Review Comment:
   removed the nullness. Raw types suppression has to stay, as it's due to the 
base class



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

Review Comment:
   done



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

Review Comment:
   done



##########
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:
   Not sure - ByteString has copyFrom and readFrom. Don't see an obvious way to 
do this without a copy



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

Reply via email to