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

Adding support for PubSub Lite in Schema Transforms #24275

Merged
merged 3 commits into from
Dec 3, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions sdks/java/io/google-cloud-platform/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,9 @@ dependencies {
testImplementation library.java.joda_time
testImplementation library.java.google_cloud_spanner_test
testRuntimeOnly library.java.slf4j_jdk14
// everit_json is needed for PubsubLite SchemaTransform tests that rely on JSON-schema translation.
permitUnusedDeclared library.java.everit_json_schema
provided library.java.everit_json_schema
}

// Don't pull in newer versions of the checker framework from dependencies.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,167 @@
/*
* 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.pubsublite;

import com.google.auto.value.AutoValue;
import com.google.cloud.pubsublite.CloudRegionOrZone;
import com.google.cloud.pubsublite.ProjectId;
import com.google.cloud.pubsublite.SubscriptionName;
import com.google.cloud.pubsublite.SubscriptionPath;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.schemas.utils.AvroUtils;
import org.apache.beam.sdk.schemas.utils.JsonUtils;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
import org.checkerframework.checker.initialization.qual.Initialized;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.checkerframework.checker.nullness.qual.UnknownKeyFor;

public class PubsubLiteReadSchemaTransformProvider
extends TypedSchemaTransformProvider<
PubsubLiteReadSchemaTransformProvider.PubsubLiteReadSchemaTransformConfiguration> {

@Override
protected @UnknownKeyFor @NonNull @Initialized Class<PubsubLiteReadSchemaTransformConfiguration>
configurationClass() {
return PubsubLiteReadSchemaTransformConfiguration.class;
}

@Override
public @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
PubsubLiteReadSchemaTransformConfiguration configuration) {
final Schema beamSchema =
Objects.equals(configuration.getDataFormat(), "JSON")
? JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema())
: AvroUtils.toBeamSchema(
new org.apache.avro.Schema.Parser().parse(configuration.getSchema()));
final SerializableFunction<byte[], Row> valueMapper =
Objects.equals(configuration.getDataFormat(), "JSON")
? JsonUtils.getJsonBytesToRowFunction(beamSchema)
: AvroUtils.getAvroBytesToRowFunction(beamSchema);
return new SchemaTransform() {
@Override
public @UnknownKeyFor @NonNull @Initialized PTransform<
@UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
@UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
buildTransform() {
return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
String project = configuration.getProject();
if (Strings.isNullOrEmpty(project)) {
project = input.getPipeline().getOptions().as(GcpOptions.class).getProject();
}
if (project == null) {
throw new IllegalArgumentException(
"Unable to infer the project to read from Pubsub Lite. Please provide a project.");
}
return PCollectionRowTuple.of(
"output",
input
.getPipeline()
.apply(
PubsubLiteIO.read(
SubscriberOptions.newBuilder()
.setSubscriptionPath(
SubscriptionPath.newBuilder()
.setLocation(
CloudRegionOrZone.parse(configuration.getLocation()))
.setProject(ProjectId.of(project))
.setName(
SubscriptionName.of(
configuration.getSubscriptionName()))
.build())
.build()))
.apply(
MapElements.into(TypeDescriptors.rows())
.via(
seqMess ->
valueMapper.apply(
seqMess.getMessage().getData().toByteArray())))
.setRowSchema(beamSchema));
}
};
}
};
}

@Override
public @UnknownKeyFor @NonNull @Initialized String identifier() {
return "beam:schematransform:org.apache.beam:pubsublite_read:v1";
}

@Override
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
inputCollectionNames() {
return Collections.emptyList();
}

@Override
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
outputCollectionNames() {
return Collections.singletonList("output");
}

@AutoValue
@DefaultSchema(AutoValueSchema.class)
public abstract static class PubsubLiteReadSchemaTransformConfiguration {
public abstract String getDataFormat();

public abstract String getSchema();

public abstract @Nullable String getProject();

public abstract String getSubscriptionName();

public abstract String getLocation();

public static Builder builder() {
return new AutoValue_PubsubLiteReadSchemaTransformProvider_PubsubLiteReadSchemaTransformConfiguration
.Builder();
}

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setDataFormat(String dataFormat);

public abstract Builder setSchema(String schema);

public abstract Builder setProject(String project);

public abstract Builder setSubscriptionName(String subscriptionName);

public abstract Builder setLocation(String location);

public abstract PubsubLiteReadSchemaTransformConfiguration build();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import com.google.protobuf.ByteString;
import java.util.ArrayDeque;
import java.util.Deque;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
Expand All @@ -49,6 +50,8 @@
import org.apache.beam.sdk.io.gcp.pubsub.TestPubsubSignal;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.utils.JsonUtils;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import org.apache.beam.sdk.transforms.Create;
Expand All @@ -58,6 +61,9 @@
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
import org.joda.time.Duration;
import org.junit.After;
Expand All @@ -73,6 +79,8 @@ public class ReadWriteIT {
private static final Logger LOG = LoggerFactory.getLogger(ReadWriteIT.class);
private static final CloudZone ZONE = CloudZone.parse("us-central1-b");
private static final int MESSAGE_COUNT = 90;
private static final Schema SAMPLE_BEAM_SCHEMA =
Schema.builder().addStringField("numberInString").addInt32Field("numberInInt").build();

@Rule public transient TestPubsubSignal signal = TestPubsubSignal.create();
@Rule public transient TestPipeline pipeline = TestPipeline.create();
Expand Down Expand Up @@ -112,6 +120,7 @@ private TopicPath createTopic(ProjectId id) throws Exception {
LOG.error("Failed to clean up topic.", t);
}
});
LOG.info("Creating topic named {}", toReturn);
try (AdminClient client = newAdminClient()) {
client.createTopic(topic.build()).get();
}
Expand All @@ -138,6 +147,7 @@ private SubscriptionPath createSubscription(TopicPath topic) throws Exception {
LOG.error("Failed to clean up subscription.", t);
}
});
LOG.info("Creating subscription named {} from topic {}", toReturn, topic);
try (AdminClient client = newAdminClient()) {
client.createSubscription(subscription.build(), BacklogLocation.BEGINNING).get();
}
Expand Down Expand Up @@ -168,6 +178,39 @@ public Iterable<Integer> apply(Void input) {
}
}

public static void writeJsonMessages(TopicPath topicPath, Pipeline pipeline) {
final SimpleFunction<Row, String> rowToJsonFn =
JsonUtils.getRowToJsonStringsFunction(SAMPLE_BEAM_SCHEMA);
pipeline
.apply(Create.of((Void) null))
.apply("createIndexes", new CustomCreate())
.apply(
"format to rows",
MapElements.via(
new SimpleFunction<Integer, Row>(
index ->
Row.withSchema(SAMPLE_BEAM_SCHEMA)
.addValue(Objects.requireNonNull(index).toString())
.addValue(index)
.build()) {}))
.setRowSchema(SAMPLE_BEAM_SCHEMA)
.apply(
"createMessages",
MapElements.via(
new SimpleFunction<Row, PubSubMessage>(
beamRow ->
Message.builder()
.setData(
ByteString.copyFromUtf8(
Objects.requireNonNull(rowToJsonFn.apply(beamRow))))
.build()
.toProto()) {}))
.apply("addUuids", PubsubLiteIO.addUuids())
.apply(
"writeMessages",
PubsubLiteIO.write(PublisherOptions.newBuilder().setTopicPath(topicPath).build()));
}

public static void writeMessages(TopicPath topicPath, Pipeline pipeline) {
PCollection<Void> trigger = pipeline.apply(Create.of((Void) null));
PCollection<Integer> indexes = trigger.apply("createIndexes", new CustomCreate());
Expand Down Expand Up @@ -219,6 +262,81 @@ public static SerializableFunction<Set<Integer>, Boolean> testIds() {
};
}

@Test
public void testPubsubLiteReadWithSchemaTransform() throws Exception {
pipeline.getOptions().as(StreamingOptions.class).setStreaming(true);
pipeline.getOptions().as(TestPipelineOptions.class).setBlockOnRun(false);

TopicPath topic = createTopic(getProject(pipeline.getOptions()));
SubscriptionPath subscription = null;
Exception lastException = null;
for (int i = 0; i < 30; ++i) {
// Sleep for topic creation to propagate.
Thread.sleep(1000);
try {
subscription = createSubscription(topic);
break;
} catch (Exception e) {
lastException = e;
LOG.info("Retrying exception on subscription creation.", e);
}
}
if (subscription == null) {
throw lastException;
}

// Publish some messages
writeJsonMessages(topic, pipeline);

// Read some messages. They should be deduplicated by the time we see them, so there should be
// exactly numMessages, one for every index in [0,MESSAGE_COUNT).
PCollection<Row> messages =
PCollectionRowTuple.empty(pipeline)
.apply(
new PubsubLiteReadSchemaTransformProvider()
.from(
PubsubLiteReadSchemaTransformProvider
.PubsubLiteReadSchemaTransformConfiguration.builder()
.setDataFormat("JSON")
.setSchema(
"{\n"
+ " \"properties\": {\n"
+ " \"numberInString\": {\n"
+ " \"type\": \"string\"\n"
+ " },\n"
+ " \"numberInInt\": {\n"
+ " \"type\": \"integer\"\n"
+ " }\n"
+ " }\n"
+ "}")
.setSubscriptionName(subscription.name().value())
.setLocation(subscription.location().toString())
.build())
.buildTransform())
.get("output");
PCollection<Integer> ids =
messages.apply(
"get ints",
MapElements.into(TypeDescriptors.integers())
.via(
row -> {
return Objects.requireNonNull(row.getInt64("numberInInt")).intValue();
}));
ids.apply("PubsubSignalTest", signal.signalSuccessWhen(BigEndianIntegerCoder.of(), testIds()));
Supplier<Void> start = signal.waitForStart(Duration.standardMinutes(5));
pipeline.apply(signal.signalStart());
PipelineResult job = pipeline.run();
start.get();
LOG.info("Running!");
signal.waitForSuccess(Duration.standardMinutes(5));
// A runner may not support cancel
try {
job.cancel();
} catch (UnsupportedOperationException exc) {
// noop
}
}

@Test
public void testReadWrite() throws Exception {
pipeline.getOptions().as(StreamingOptions.class).setStreaming(true);
Expand Down