Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

#21431 Pubsub dynamic topic destinations #26063

Merged
merged 3 commits into from
Apr 22, 2023

Conversation

reuvenlax
Copy link
Contributor

No description provided.

@github-actions
Copy link
Contributor

github-actions bot commented Apr 1, 2023

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @robertwb for label java.
R: @chamikaramj for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@github-actions github-actions bot added core and removed core labels Apr 8, 2023
@reuvenlax reuvenlax force-pushed the pubsub_dynamic_destinations branch from ba4dd0b to e11a5ac Compare April 9, 2023 18:07
@github-actions github-actions bot added core and removed core labels Apr 9, 2023
@reuvenlax
Copy link
Contributor Author

Run Spotless PreCommit

@reuvenlax reuvenlax changed the title Pubsub dynamic destinations #21431 Pubsub dynamic topic destinations Apr 11, 2023
@reuvenlax reuvenlax force-pushed the pubsub_dynamic_destinations branch from 96ed323 to 39b56fc Compare April 11, 2023 00:17
@github-actions github-actions bot added the core label Apr 11, 2023
@reuvenlax
Copy link
Contributor Author

R: @Naireen

@github-actions github-actions bot removed the core label Apr 11, 2023
@github-actions
Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@reuvenlax
Copy link
Contributor Author

R: @scwhittle

public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription";
public static final String PUBSUB_SUBSCRIPTION_OVERRIDE = "pubsub_subscription_runtime_override";
public static final String PUBSUB_TIMESTAMP_ATTRIBUTE = "pubsub_timestamp_label";
public static final String PUBSUB_TOPIC = "pubsub_topic";
public static final String PUBSUB_TOPIC_OVERRIDE = "pubsub_topic_runtime_override";

public static final String PUBSUB_DYNAMIC_DESTINATIONS = "pubsub_with_dynamic_destinations";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: pubsub_dynamic_destinations to match enum? ie remove "with"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This has to match the string on the Dataflow backend, which is pubsub_with_dynamic_destinations

new PipelineVisitor.Defaults() {

@Override
public CompositeBehavior enterCompositeTransform(Node node) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it should only be a composite or a primitive right?
can you simplify this test and remove the unnecessary override here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

new PipelineVisitor.Defaults() {

@Override
public CompositeBehavior enterCompositeTransform(Node node) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
import org.checkerframework.checker.nullness.qual.Nullable;

@SuppressWarnings({
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

since this is new, can you avoid supressing these?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}
}

public static class Factory implements SinkFactory {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can this be private or package visible? or perhaps even inlined in factories() above?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -1350,23 +1385,15 @@ public void startBundle(StartBundleContext c) throws IOException {
@ProcessElement
public void processElement(ProcessContext c) throws IOException, SizeLimitExceededException {
PubsubMessage message = getFormatFn().apply(c.element());
int messageSize = validateAndGetPubsubMessageSize(message);
if (messageSize > maxPublishBatchByteSize) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

keep this as sanity check?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, done

} else {
Map<TopicPath, List<OutgoingMessage>> messagesPerTopic = Maps.newHashMap();
for (OutgoingMessage message : messages) {
TopicPath topicPath = PubsubClient.topicPathFromPath(message.topic());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could do this translation during publish loop so it's per topic instead of per message

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -495,6 +604,10 @@ static class PubsubSink extends PTransform<PCollection<byte[]>, PDone> {
this.outer = outer;
}

boolean isDynamic() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove? seems PubsubUnboundedSink would be used if this was true?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -133,6 +132,11 @@
<td>Sets the behavior of reusing objects.</td>
<td>Default: <code>false</code></td>
</tr>
<tr>
<td><code>operatorChaining</code></td>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

revert this file?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -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>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this file still seems there

@reuvenlax reuvenlax force-pushed the pubsub_dynamic_destinations branch from 6660fdc to b394cf4 Compare April 18, 2023 19:33
private final String timestampLabel;
private final String idLabel;
private final StreamingModeExecutionContext context;
// Function used to convert PCollection elements to PubsubMessage objects.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove comment?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

public ByteString getDataFromMessage(PubsubMessage formatted, ByteStringOutputStream stream)
throws IOException {
Pubsub.PubsubMessage.Builder pubsubMessageBuilder =
Pubsub.PubsubMessage.newBuilder().setData(ByteString.copyFrom(formatted.getPayload()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if we should separately look into switching to ByteString within beam's PubsubMessage.

@Override
public long add(WindowedValue<PubsubMessage> data) throws IOException {
String dataTopic =
org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

import static?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.when;

import avro.shaded.com.google.common.collect.Lists;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't use shaded version?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

if (!pubsubMessages.isEmpty()
&& bytes + message.message().getData().size() > publishBatchBytes) {
&& bytes + message.getMessage().getData().size() > publishBatchBytes) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this limit is currently across all topics but could be per-topic

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added grouping by topic+shard

// dynamic destinations.
return input
.apply(
"WithDynamicKeys",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: WithDynamicTopics? DynamicDestinations?
keys seems implementation detail

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -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>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this file still seems there

switch (input.isBounded()) {
case BOUNDED:
input.apply(
pubsubMessages.apply(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point - fixed BoundedWriter

@github-actions github-actions bot removed the website label Apr 19, 2023
}

public void testDynamicTopics(boolean isBounded) throws IOException {
List<OutgoingMessage> outgoing =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: name expectedOutgoing, it wasn't obvious that the test factory for publish took these as expectations when created. I think naming would help.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -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;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could have a single map by topic, reduces lookup and prevents possibility of entry in one map and not the other.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

pubsubClient.close();
pubsubClient = null;
}

private void publish() throws IOException {
PubsubTopic topic = getTopicProvider().get();
for (Map.Entry<PubsubTopic, List<OutgoingMessage>> entry : output.entrySet()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use visitor instead of iterator?
forEach(e -> publish(e.getKey(), e.getValue());

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unfortunately this does not work for function references that throw exceptions. I would have to wrap a try/catch inside the lambda (i.e. catch the IOException and rethrow as a RuntimeException), which gets ugly.

publish(pubsubTopic, currentTopicOutput);
currentTopicOutput.clear();
currentTopicOutputBytes = 0;
currentOutputBytes.put(pubsubTopic, 0L);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

think the single map would make this simpler to since you would be updating the value in the entry directly

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

// 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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"cannot be null" is inconsistent with using NullableCoder

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@Override
public PubsubMessage decode(InputStream inStream) throws IOException {
byte[] payload = PAYLOAD_CODER.decode(inStream);
Map<String, String> attributes = ATTRIBUTES_CODER.decode(inStream);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add Nullable annotation?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


/** A coder for PubsubMessage including the topic from the PubSub server. */
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can this be removed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@reuvenlax
Copy link
Contributor Author

Run Java PreCommit

Copy link
Contributor

@scwhittle scwhittle left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM with one comment

}

@FinishBundle
public void finishBundle() throws IOException {
publish();
output = null;
currentOutputBytes = null;
pubsubClient.close();
pubsubClient = null;
}

private void publish() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

inline this in finishBundle? otherwise the topic specific publish should be called

@reuvenlax reuvenlax force-pushed the pubsub_dynamic_destinations branch from 7809f08 to 1cb9649 Compare April 22, 2023 01:48
@reuvenlax reuvenlax force-pushed the pubsub_dynamic_destinations branch from 1cb9649 to ca67490 Compare April 22, 2023 02:16
@reuvenlax reuvenlax merged commit f549fd3 into apache:master Apr 22, 2023
copybara-service bot pushed a commit to GoogleCloudPlatform/DataflowTemplates that referenced this pull request May 1, 2023
@kennknowles
Copy link
Member

In authoring #30604 some test from this PR failed. Specifically something that appears to expect client-side overrides (aka the v1 path) to have happened when using v2. Does that make sense to you all?

}
}
});
assertTrue(sawPubsubOverride.get());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should never be using client-side overrides for v2, where we want it all to happen in the service. Is there necessary functionality in the override? It happens after the v2 pipeline is created anyhow I think...

@reuvenlax
Copy link
Contributor Author

reuvenlax commented Mar 12, 2024 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants