-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-8932][Cleanup] Move external PubsubIO hooks outside of PubsubIO. #10476
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
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
138 changes: 138 additions & 0 deletions
138
...o/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalRead.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,138 @@ | ||
| /* | ||
| * 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 com.google.auto.service.AutoService; | ||
| import com.google.protobuf.ByteString; | ||
| import java.util.Map; | ||
| import javax.annotation.Nullable; | ||
| import org.apache.beam.sdk.annotations.Experimental; | ||
| import org.apache.beam.sdk.coders.ByteArrayCoder; | ||
| import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; | ||
| import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubSubscription; | ||
| import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubTopic; | ||
| import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; | ||
| import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; | ||
| import org.apache.beam.sdk.transforms.ExternalTransformBuilder; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.transforms.SimpleFunction; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; | ||
|
|
||
| /** Exposes {@link PubsubIO.Read} as an external transform for cross-language usage. */ | ||
| @Experimental | ||
| @AutoService(ExternalTransformRegistrar.class) | ||
| public final class ExternalRead implements ExternalTransformRegistrar { | ||
| public ExternalRead() {} | ||
|
|
||
| public static final String URN = "beam:external:java:pubsub:read:v1"; | ||
|
|
||
| @Override | ||
| public Map<String, Class<? extends ExternalTransformBuilder>> knownBuilders() { | ||
| return ImmutableMap.of(URN, ReadBuilder.class); | ||
| } | ||
|
|
||
| /** Parameters class to expose the transform to an external SDK. */ | ||
| public static class Configuration { | ||
| @Nullable private String topic; | ||
| @Nullable private String subscription; | ||
| @Nullable private String idAttribute; | ||
| @Nullable private String timestampAttribute; | ||
| private boolean needsAttributes; | ||
|
|
||
| public void setTopic(@Nullable String topic) { | ||
| this.topic = topic; | ||
| } | ||
|
|
||
| public void setSubscription(@Nullable String subscription) { | ||
| this.subscription = subscription; | ||
| } | ||
|
|
||
| public void setIdLabel(@Nullable String idAttribute) { | ||
| this.idAttribute = idAttribute; | ||
| } | ||
|
|
||
| public void setTimestampAttribute(@Nullable String timestampAttribute) { | ||
| this.timestampAttribute = timestampAttribute; | ||
| } | ||
|
|
||
| public void setWithAttributes(Boolean needsAttributes) { | ||
| // we must use Boolean instead of boolean because the external payload system | ||
| // inspects the native type of each coder urn, and BooleanCoder wants Boolean. | ||
| this.needsAttributes = needsAttributes; | ||
| } | ||
| } | ||
|
|
||
| public static class ReadBuilder | ||
| implements ExternalTransformBuilder<Configuration, PBegin, PCollection<byte[]>> { | ||
| public ReadBuilder() {} | ||
|
|
||
| @Override | ||
| public PTransform<PBegin, PCollection<byte[]>> buildExternal(Configuration config) { | ||
| PubsubIO.Read.Builder<byte[]> readBuilder = new AutoValue_PubsubIO_Read.Builder<>(); | ||
| readBuilder.setCoder(ByteArrayCoder.of()); | ||
| if (config.topic != null) { | ||
| StaticValueProvider<String> topic = StaticValueProvider.of(config.topic); | ||
| readBuilder.setTopicProvider(NestedValueProvider.of(topic, PubsubTopic::fromPath)); | ||
| } | ||
| if (config.subscription != null) { | ||
| StaticValueProvider<String> subscription = StaticValueProvider.of(config.subscription); | ||
| readBuilder.setSubscriptionProvider( | ||
| NestedValueProvider.of(subscription, PubsubSubscription::fromPath)); | ||
| } | ||
| if (config.idAttribute != null) { | ||
| readBuilder.setIdAttribute(config.idAttribute); | ||
| } | ||
| if (config.timestampAttribute != null) { | ||
| readBuilder.setTimestampAttribute(config.timestampAttribute); | ||
| } | ||
| readBuilder.setNeedsAttributes(config.needsAttributes); | ||
| if (config.needsAttributes) { | ||
| readBuilder.setParseFn(new ParsePayloadAsPubsubMessageProto()); | ||
| } else { | ||
| readBuilder.setParseFn( | ||
| new SimpleFunction<PubsubMessage, byte[]>() { | ||
| @Override | ||
| public byte[] apply(PubsubMessage input) { | ||
| return input.getPayload(); | ||
| } | ||
| }); | ||
| } | ||
| readBuilder.setNeedsMessageId(false); | ||
| return readBuilder.build(); | ||
| } | ||
| } | ||
|
|
||
| // Convert the PubsubMessage to a PubsubMessage proto, then return its serialized representation. | ||
| private static class ParsePayloadAsPubsubMessageProto | ||
| extends SimpleFunction<PubsubMessage, byte[]> { | ||
| @Override | ||
| public byte[] apply(PubsubMessage input) { | ||
| Map<String, String> attributes = input.getAttributeMap(); | ||
| com.google.pubsub.v1.PubsubMessage.Builder message = | ||
| com.google.pubsub.v1.PubsubMessage.newBuilder() | ||
| .setData(ByteString.copyFrom(input.getPayload())); | ||
| // TODO(BEAM-8085) this should not be null | ||
| if (attributes != null) { | ||
| message.putAllAttributes(attributes); | ||
| } | ||
| return message.build().toByteArray(); | ||
| } | ||
| } | ||
| } |
102 changes: 102 additions & 0 deletions
102
.../google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalWrite.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,102 @@ | ||
| /* | ||
| * 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 com.google.auto.service.AutoService; | ||
| import com.google.protobuf.InvalidProtocolBufferException; | ||
| import java.util.Map; | ||
| import javax.annotation.Nullable; | ||
| import org.apache.beam.sdk.annotations.Experimental; | ||
| import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; | ||
| import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubTopic; | ||
| import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; | ||
| import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; | ||
| import org.apache.beam.sdk.transforms.ExternalTransformBuilder; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.transforms.SimpleFunction; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.PDone; | ||
| import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; | ||
|
|
||
| /** Exposes {@link PubsubIO.Write} as an external transform for cross-language usage. */ | ||
| @Experimental | ||
| @AutoService(ExternalTransformRegistrar.class) | ||
| public final class ExternalWrite implements ExternalTransformRegistrar { | ||
| public ExternalWrite() {} | ||
|
|
||
| public static final String URN = "beam:external:java:pubsub:write:v1"; | ||
|
|
||
| @Override | ||
| public Map<String, Class<? extends ExternalTransformBuilder>> knownBuilders() { | ||
| return ImmutableMap.of(URN, WriteBuilder.class); | ||
| } | ||
|
|
||
| /** Parameters class to expose the transform to an external SDK. */ | ||
| public static class Configuration { | ||
| private String topic; | ||
| @Nullable private String idAttribute; | ||
| @Nullable private String timestampAttribute; | ||
|
|
||
| public void setTopic(String topic) { | ||
| this.topic = topic; | ||
| } | ||
|
|
||
| public void setIdLabel(@Nullable String idAttribute) { | ||
| this.idAttribute = idAttribute; | ||
| } | ||
|
|
||
| public void setTimestampAttribute(@Nullable String timestampAttribute) { | ||
| this.timestampAttribute = timestampAttribute; | ||
| } | ||
| } | ||
|
|
||
| public static class WriteBuilder | ||
| implements ExternalTransformBuilder<Configuration, PCollection<byte[]>, PDone> { | ||
| public WriteBuilder() {} | ||
|
|
||
| @Override | ||
| public PTransform<PCollection<byte[]>, PDone> buildExternal(Configuration config) { | ||
| PubsubIO.Write.Builder<byte[]> writeBuilder = new AutoValue_PubsubIO_Write.Builder<>(); | ||
| writeBuilder.setFormatFn(new FormatFn()); | ||
| if (config.topic != null) { | ||
| StaticValueProvider<String> topic = StaticValueProvider.of(config.topic); | ||
| writeBuilder.setTopicProvider(NestedValueProvider.of(topic, PubsubTopic::fromPath)); | ||
| } | ||
| if (config.idAttribute != null) { | ||
| writeBuilder.setIdAttribute(config.idAttribute); | ||
| } | ||
| if (config.timestampAttribute != null) { | ||
| writeBuilder.setTimestampAttribute(config.timestampAttribute); | ||
| } | ||
| return writeBuilder.build(); | ||
| } | ||
| } | ||
|
|
||
| private static class FormatFn extends SimpleFunction<byte[], PubsubMessage> { | ||
| @Override | ||
| public PubsubMessage apply(byte[] input) { | ||
| try { | ||
| com.google.pubsub.v1.PubsubMessage message = | ||
| com.google.pubsub.v1.PubsubMessage.parseFrom(input); | ||
| return new PubsubMessage(message.getData().toByteArray(), message.getAttributesMap()); | ||
| } catch (InvalidProtocolBufferException e) { | ||
| throw new RuntimeException("Could not decode Pubsub message", e); | ||
| } | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.