1717 */
1818package org .apache .beam .sdk .io .gcp .pubsub ;
1919
20+ import static org .apache .beam .sdk .io .gcp .pubsub .PubsubIO .ENABLE_CUSTOM_PUBSUB_SINK ;
21+ import static org .apache .beam .sdk .io .gcp .pubsub .PubsubIO .ENABLE_CUSTOM_PUBSUB_SOURCE ;
22+
2023import com .google .auto .service .AutoService ;
2124import java .util .Collections ;
2225import java .util .Map ;
3336import org .apache .beam .sdk .io .gcp .pubsub .PubsubClient .SubscriptionPath ;
3437import org .apache .beam .sdk .io .gcp .pubsub .PubsubClient .TopicPath ;
3538import org .apache .beam .sdk .io .gcp .pubsub .PubsubUnboundedSource .PubsubSource ;
39+ import org .apache .beam .sdk .options .ExperimentalOptions ;
3640import org .apache .beam .sdk .options .ValueProvider ;
3741import org .apache .beam .sdk .options .ValueProvider .NestedValueProvider ;
3842import org .apache .beam .sdk .runners .AppliedPTransform ;
@@ -59,6 +63,10 @@ public String getUrn() {
5963 @ Override
6064 public RunnerApi .FunctionSpec translate (
6165 AppliedPTransform <?, ?, Unbounded <?>> transform , SdkComponents components ) {
66+ if (ExperimentalOptions .hasExperiment (
67+ transform .getPipeline ().getOptions (), ENABLE_CUSTOM_PUBSUB_SOURCE )) {
68+ return null ;
69+ }
6270 if (!(transform .getTransform ().getSource () instanceof PubsubUnboundedSource .PubsubSource )) {
6371 return null ;
6472 }
@@ -111,6 +119,10 @@ public String getUrn() {
111119 public RunnerApi .FunctionSpec translate (
112120 AppliedPTransform <?, ?, PubsubUnboundedSink .PubsubSink > transform ,
113121 SdkComponents components ) {
122+ if (ExperimentalOptions .hasExperiment (
123+ transform .getPipeline ().getOptions (), ENABLE_CUSTOM_PUBSUB_SINK )) {
124+ return null ;
125+ }
114126 PubSubWritePayload .Builder payloadBuilder = PubSubWritePayload .newBuilder ();
115127 ValueProvider <TopicPath > topicProvider =
116128 Preconditions .checkStateNotNull (transform .getTransform ().outer .getTopicProvider ());
@@ -145,6 +157,10 @@ public String getUrn() {
145157 public RunnerApi .FunctionSpec translate (
146158 AppliedPTransform <?, ?, PubsubUnboundedSink .PubsubDynamicSink > transform ,
147159 SdkComponents components ) {
160+ if (ExperimentalOptions .hasExperiment (
161+ transform .getPipeline ().getOptions (), ENABLE_CUSTOM_PUBSUB_SINK )) {
162+ return null ;
163+ }
148164 PubSubWritePayload .Builder payloadBuilder = PubSubWritePayload .newBuilder ();
149165 if (transform .getTransform ().outer .getTimestampAttribute () != null ) {
150166 payloadBuilder .setTimestampAttribute (
0 commit comments