diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 6f25803b8483..6e6fa4fd5fb1 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -102,8 +102,6 @@ dependencies { permitUnusedDeclared library.java.flogger_system_backend implementation library.java.google_api_client // Ensures SequencedMessage availability for Spotless - implementation library.java.proto_google_cloud_pubsublite_v1 - permitUnusedDeclared library.java.proto_google_cloud_pubsublite_v1 implementation library.java.google_api_services_dataflow implementation library.java.google_api_services_storage permitUnusedDeclared library.java.google_api_services_storage // BEAM-11761 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 775e7b91de93..f34b74866575 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -96,7 +96,6 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink; import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.SubscribeTransform; import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -665,8 +664,6 @@ private List getOverrides(boolean streaming) { // and only needed when KafkaIO is used in the pipeline. } - overridesBuilder.add(SubscribeTransform.V1_READ_OVERRIDE); - if (!hasExperiment(options, "enable_file_dynamic_sharding")) { overridesBuilder.add( PTransformOverride.of( diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index afbc87f8eeba..49633a0a47a1 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -103,12 +103,8 @@ dependencies { implementation library.java.proto_google_cloud_bigtable_v2 implementation library.java.google_api_services_bigquery permitUnusedDeclared library.java.google_api_services_bigquery - implementation library.java.proto_google_cloud_pubsublite_v1 - implementation library.java.google_cloud_pubsublite implementation project(":sdks:java:io:mongodb") implementation library.java.avro - implementation library.java.protobuf_java - implementation library.java.protobuf_java_util provided project(":sdks:java:io:parquet") provided library.java.jackson_dataformat_xml permitUnusedDeclared library.java.jackson_dataformat_xml diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteSubscriptionTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteSubscriptionTable.java deleted file mode 100644 index 5f46472d2b58..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteSubscriptionTable.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; -import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; -import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; -import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.Row; - -class PubsubLiteSubscriptionTable extends SchemaBaseBeamTable { - private final SubscriptionPath subscription; - private final PTransform, PCollection> transform; - - PubsubLiteSubscriptionTable( - Schema schema, - SubscriptionPath subscription, - PTransform, PCollection> transform) { - super(schema); - this.subscription = subscription; - this.transform = transform; - } - - @Override - public PCollection buildIOReader(PBegin begin) { - return begin - .apply( - "Read Pub/Sub Lite", - PubsubLiteIO.read( - SubscriberOptions.newBuilder().setSubscriptionPath(subscription).build())) - .apply("Transform to Row", transform); - } - - @Override - public POutput buildIOWriter(PCollection input) { - throw new UnsupportedOperationException( - "You cannot write to a Pub/Sub Lite subscription: you must write to a topic."); - } - - @Override - public IsBounded isBounded() { - return IsBounded.UNBOUNDED; - } - - @Override - public BeamTableStatistics getTableStatistics(PipelineOptions options) { - return BeamTableStatistics.UNBOUNDED_UNKNOWN; - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProvider.java deleted file mode 100644 index cdc6a4c05dff..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProvider.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.auto.service.AutoService; -import com.google.auto.value.AutoOneOf; -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import java.util.Optional; -import org.apache.beam.sdk.extensions.sql.TableUtils; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; -import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.EquivalenceNullablePolicy; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.schemas.io.DeadLetteredTransform; -import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; -import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializers; -import org.apache.beam.sdk.transforms.MapElements; -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.Row; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** - * Pub/Sub Lite table provider. - * - *

Pub/Sub Lite tables may be constructed with: - * - *

{@code
- * CREATE EXTERNAL TABLE tableName(
- *     message_key BYTES [NOT NULL],  // optional, always present on read
- *     publish_timestamp TIMESTAMP [NOT NULL],  // optional, readable tables only, always present on read
- *     event_timestamp TIMESTAMP [NOT NULL],  // optional, null if not present in readable table, unset in message if null in writable table. NOT NULL enforces field presence on read
- *     attributes ARRAY>>,  // optional, null values never present on reads or handled on writes
- *     payload BYTES | ROW<[INSERT SCHEMA HERE]>,
- * )
- * TYPE pubsublite
- * // For writable tables
- * LOCATION 'projects/[PROJECT]/locations/[CLOUD ZONE]/topics/[TOPIC]'
- * // For readable tables
- * LOCATION 'projects/[PROJECT]/locations/[CLOUD ZONE]/subscriptions/[SUBSCRIPTION]'
- * TBLPROPERTIES '{
- *     "deadLetterQueue": "[DLQ_KIND]:[DLQ_ID]",  // optional
- *     "format": "[FORMAT]",  // optional
- *     // format params
- * }'
- * }
- */ -@AutoService(TableProvider.class) -public class PubsubLiteTableProvider extends InMemoryMetaTableProvider { - @Override - public String getTableType() { - return "pubsublite"; - } - - private static Optional getSerializer(Schema schema, ObjectNode properties) { - if (schema.getField("payload").getType().equals(FieldType.BYTES)) { - checkArgument( - !properties.has("format"), - "Must not set the 'format' property if not unpacking payload."); - return Optional.empty(); - } - String format = properties.path("format").asText("json"); - return Optional.of( - PayloadSerializers.getSerializer(format, schema, TableUtils.convertNode2Map(properties))); - } - - private static void checkFieldHasType(Field field, FieldType type) { - checkArgument( - type.equivalent(field.getType(), EquivalenceNullablePolicy.WEAKEN), - String.format("'%s' field must have schema matching '%s'.", field.getName(), type)); - } - - private static void validateSchema(Schema schema) { - checkArgument( - schema.hasField(RowHandler.PAYLOAD_FIELD), - "Must provide a 'payload' field for Pub/Sub Lite."); - for (Field field : schema.getFields()) { - switch (field.getName()) { - case RowHandler.ATTRIBUTES_FIELD: - checkFieldHasType(field, RowHandler.ATTRIBUTES_FIELD_TYPE); - break; - case RowHandler.EVENT_TIMESTAMP_FIELD: - case RowHandler.PUBLISH_TIMESTAMP_FIELD: - checkFieldHasType(field, FieldType.DATETIME); - break; - case RowHandler.MESSAGE_KEY_FIELD: - checkFieldHasType(field, FieldType.BYTES); - break; - case RowHandler.PAYLOAD_FIELD: - checkArgument( - FieldType.BYTES.equivalent(field.getType(), EquivalenceNullablePolicy.WEAKEN) - || field.getType().getTypeName().equals(TypeName.ROW), - String.format( - "'%s' field must either have a 'BYTES NOT NULL' or 'ROW' schema.", - field.getName())); - break; - default: - throw new IllegalArgumentException( - String.format( - "'%s' field is invalid at the top level for Pub/Sub Lite.", field.getName())); - } - } - } - - @AutoOneOf(Location.Kind.class) - abstract static class Location { - enum Kind { - TOPIC, - SUBSCRIPTION - } - - abstract Kind getKind(); - - abstract TopicPath topic(); - - abstract SubscriptionPath subscription(); - - static Location parse(String location) { - if (location.contains("/topics/")) { - return AutoOneOf_PubsubLiteTableProvider_Location.topic(TopicPath.parse(location)); - } - if (location.contains("/subscriptions/")) { - return AutoOneOf_PubsubLiteTableProvider_Location.subscription( - SubscriptionPath.parse(location)); - } - throw new IllegalArgumentException( - String.format( - "Location '%s' does not correspond to either a Pub/Sub Lite topic or subscription.", - location)); - } - } - - private static RowHandler getRowHandler( - Schema schema, Optional optionalSerializer) { - if (optionalSerializer.isPresent()) { - return new RowHandler(schema, optionalSerializer.get()); - } - return new RowHandler(schema); - } - - private static - PTransform, PCollection> addDlqIfPresent( - SimpleFunction transform, ObjectNode properties) { - if (properties.has("deadLetterQueue")) { - return new DeadLetteredTransform<>(transform, properties.get("deadLetterQueue").asText()); - } - return MapElements.via(transform); - } - - @Override - public BeamSqlTable buildBeamSqlTable(Table table) { - checkArgument(table.getType().equals(getTableType())); - validateSchema(table.getSchema()); - Optional serializer = - getSerializer(table.getSchema(), table.getProperties()); - Location location = Location.parse(checkArgumentNotNull(table.getLocation())); - RowHandler rowHandler = getRowHandler(table.getSchema(), serializer); - - switch (location.getKind()) { - case TOPIC: - checkArgument( - !table.getSchema().hasField(RowHandler.PUBLISH_TIMESTAMP_FIELD), - "May not write to publish timestamp, this field is read-only."); - return new PubsubLiteTopicTable( - table.getSchema(), - location.topic(), - addDlqIfPresent( - SimpleFunction.fromSerializableFunctionWithOutputType( - rowHandler::rowToMessage, TypeDescriptor.of(PubSubMessage.class)), - table.getProperties())); - case SUBSCRIPTION: - return new PubsubLiteSubscriptionTable( - table.getSchema(), - location.subscription(), - addDlqIfPresent( - SimpleFunction.fromSerializableFunctionWithOutputType( - rowHandler::messageToRow, TypeDescriptor.of(Row.class)), - table.getProperties())); - default: - throw new IllegalArgumentException("Invalid kind for location: " + location.getKind()); - } - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTopicTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTopicTable.java deleted file mode 100644 index 3cb9f51b4097..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTopicTable.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; -import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; -import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.Row; - -class PubsubLiteTopicTable extends SchemaBaseBeamTable { - private final TopicPath topic; - private final PTransform, PCollection> transform; - - PubsubLiteTopicTable( - Schema schema, - TopicPath topic, - PTransform, PCollection> transform) { - super(schema); - this.topic = topic; - this.transform = transform; - } - - @Override - public PCollection buildIOReader(PBegin begin) { - throw new UnsupportedOperationException( - "You cannot read from a Pub/Sub Lite topic: you must create a subscription first."); - } - - @Override - public POutput buildIOWriter(PCollection input) { - return input - .apply("Transform to PubSubMessage", transform) - .apply( - "Write Pub/Sub Lite", - PubsubLiteIO.write(PublisherOptions.newBuilder().setTopicPath(topic).build())); - } - - @Override - public IsBounded isBounded() { - return IsBounded.UNBOUNDED; - } - - @Override - public BeamTableStatistics getTableStatistics(PipelineOptions options) { - return BeamTableStatistics.UNBOUNDED_UNKNOWN; - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandler.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandler.java deleted file mode 100644 index 15a0b4732657..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandler.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import static org.apache.beam.sdk.schemas.transforms.Cast.castRow; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import com.google.cloud.pubsublite.proto.AttributeValues; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import com.google.protobuf.ByteString; -import com.google.protobuf.util.Timestamps; -import java.io.Serializable; -import java.util.Collection; -import java.util.stream.Collectors; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.joda.time.Instant; -import org.joda.time.ReadableDateTime; - -class RowHandler implements Serializable { - private static final long serialVersionUID = 6827681678454156L; - - static final String PUBLISH_TIMESTAMP_FIELD = "publish_timestamp"; - static final String MESSAGE_KEY_FIELD = "message_key"; - static final String EVENT_TIMESTAMP_FIELD = "event_timestamp"; - static final String ATTRIBUTES_FIELD = "attributes"; - static final String PAYLOAD_FIELD = "payload"; - - static final String ATTRIBUTES_KEY_FIELD = "key"; - static final String ATTRIBUTES_VALUES_FIELD = "values"; - - static final Schema ATTRIBUTES_ENTRY_SCHEMA = - Schema.builder() - .addStringField(ATTRIBUTES_KEY_FIELD) - .addArrayField(ATTRIBUTES_VALUES_FIELD, FieldType.BYTES) - .build(); - static final Schema.FieldType ATTRIBUTES_FIELD_TYPE = - Schema.FieldType.array(FieldType.row(ATTRIBUTES_ENTRY_SCHEMA)); - - private final Schema schema; - private final @Nullable PayloadSerializer payloadSerializer; - - RowHandler(Schema schema) { - checkArgument(schema.getField(PAYLOAD_FIELD).getType().equals(FieldType.BYTES)); - this.schema = schema; - this.payloadSerializer = null; - } - - RowHandler(Schema schema, @Nonnull PayloadSerializer payloadSerializer) { - this.schema = schema; - this.payloadSerializer = payloadSerializer; - checkArgument(schema.getField(PAYLOAD_FIELD).getType().getTypeName().equals(TypeName.ROW)); - } - - /* Convert a message to a row. If Schema payload field is a Row type, payloadSerializer is required. */ - Row messageToRow(SequencedMessage message) { - // Transform this to a FieldValueBuilder, because otherwise individual withFieldValue calls will - // not mutate the original object. - Row.FieldValueBuilder builder = Row.withSchema(schema).withFieldValues(ImmutableMap.of()); - if (schema.hasField(PUBLISH_TIMESTAMP_FIELD)) { - builder.withFieldValue( - PUBLISH_TIMESTAMP_FIELD, - Instant.ofEpochMilli(Timestamps.toMillis(message.getPublishTime()))); - } - if (schema.hasField(MESSAGE_KEY_FIELD)) { - builder.withFieldValue(MESSAGE_KEY_FIELD, message.getMessage().getKey().toByteArray()); - } - if (schema.hasField(EVENT_TIMESTAMP_FIELD) && message.getMessage().hasEventTime()) { - builder.withFieldValue( - EVENT_TIMESTAMP_FIELD, - Instant.ofEpochMilli(Timestamps.toMillis(message.getMessage().getEventTime()))); - } - if (schema.hasField(ATTRIBUTES_FIELD)) { - ImmutableList.Builder listBuilder = ImmutableList.builder(); - message - .getMessage() - .getAttributesMap() - .forEach( - (key, values) -> { - Row entry = - Row.withSchema(ATTRIBUTES_ENTRY_SCHEMA) - .withFieldValue(ATTRIBUTES_KEY_FIELD, key) - .withFieldValue( - ATTRIBUTES_VALUES_FIELD, - values.getValuesList().stream() - .map(ByteString::toByteArray) - .collect(Collectors.toList())) - .build(); - listBuilder.add(entry); - }); - builder.withFieldValue(ATTRIBUTES_FIELD, listBuilder.build()); - } - if (payloadSerializer == null) { - builder.withFieldValue(PAYLOAD_FIELD, message.getMessage().getData().toByteArray()); - } else { - builder.withFieldValue( - PAYLOAD_FIELD, - payloadSerializer.deserialize(message.getMessage().getData().toByteArray())); - } - return builder.build(); - } - - /* Convert a row to a message. If Schema payload field is a Row type, payloadSerializer is required. */ - PubSubMessage rowToMessage(Row row) { - row = castRow(row, row.getSchema(), schema); - PubSubMessage.Builder builder = PubSubMessage.newBuilder(); - if (schema.hasField(MESSAGE_KEY_FIELD)) { - byte[] bytes = row.getBytes(MESSAGE_KEY_FIELD); - if (bytes != null) { - builder.setKey(ByteString.copyFrom(bytes)); - } - } - if (schema.hasField(EVENT_TIMESTAMP_FIELD)) { - ReadableDateTime time = row.getDateTime(EVENT_TIMESTAMP_FIELD); - if (time != null) { - builder.setEventTime(Timestamps.fromMillis(time.getMillis())); - } - } - if (schema.hasField(ATTRIBUTES_FIELD)) { - Collection attributes = row.getArray(ATTRIBUTES_FIELD); - if (attributes != null) { - attributes.forEach( - entry -> { - AttributeValues.Builder valuesBuilder = AttributeValues.newBuilder(); - Collection values = - checkArgumentNotNull(entry.getArray(ATTRIBUTES_VALUES_FIELD)); - values.forEach(bytes -> valuesBuilder.addValues(ByteString.copyFrom(bytes))); - builder.putAttributes( - checkArgumentNotNull(entry.getString(ATTRIBUTES_KEY_FIELD)), - valuesBuilder.build()); - }); - } - } - if (payloadSerializer == null) { - byte[] payload = row.getBytes(PAYLOAD_FIELD); - if (payload != null) { - builder.setData(ByteString.copyFrom(payload)); - } - } else { - Row payload = row.getRow(PAYLOAD_FIELD); - if (payload != null) { - builder.setData(ByteString.copyFrom(payloadSerializer.serialize(payload))); - } - } - return builder.build(); - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/package-info.java deleted file mode 100644 index 8f6741957b1a..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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. - */ - -/** Provides abstractions for schema-aware IOs. */ -@DefaultAnnotation(NonNull.class) -package org.apache.beam.sdk.extensions.sql.meta.provider.pubsublite; - -import edu.umd.cs.findbugs.annotations.DefaultAnnotation; -import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProviderTest.java deleted file mode 100644 index 4cdad7180b8c..000000000000 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/PubsubLiteTableProviderTest.java +++ /dev/null @@ -1,239 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import static com.google.cloud.pubsublite.internal.testing.UnitTestExamples.example; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.TopicPath; -import java.util.Map; -import java.util.function.Function; -import org.apache.beam.sdk.extensions.sql.TableUtils; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class PubsubLiteTableProviderTest { - private static final PubsubLiteTableProvider PROVIDER = new PubsubLiteTableProvider(); - private static final Schema FULL_WRITE_SCHEMA = - Schema.builder() - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .addDateTimeField(RowHandler.EVENT_TIMESTAMP_FIELD) - .addArrayField( - RowHandler.ATTRIBUTES_FIELD, FieldType.row(RowHandler.ATTRIBUTES_ENTRY_SCHEMA)) - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .build(); - private static final Schema FULL_READ_SCHEMA = - Schema.builder() - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .addDateTimeField(RowHandler.EVENT_TIMESTAMP_FIELD) - .addArrayField( - RowHandler.ATTRIBUTES_FIELD, FieldType.row(RowHandler.ATTRIBUTES_ENTRY_SCHEMA)) - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addDateTimeField(RowHandler.PUBLISH_TIMESTAMP_FIELD) - .build(); - - private static BeamSqlTable makeTable( - Schema schema, String location, Map properties) { - Table table = - Table.builder() - .type(PROVIDER.getTableType()) - .name("testTable") - .schema(schema) - .location(location) - .properties(TableUtils.getObjectMapper().valueToTree(properties)) - .build(); - return PROVIDER.buildBeamSqlTable(table); - } - - @Test - public void invalidSchemas() { - Function tableMaker = - schema -> makeTable(schema, example(SubscriptionPath.class).toString(), ImmutableMap.of()); - // No payload - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder().addDateTimeField(RowHandler.EVENT_TIMESTAMP_FIELD).build())); - // Bad payload type - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply(Schema.builder().addDateTimeField(RowHandler.PAYLOAD_FIELD).build())); - // Bad field name - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addByteArrayField("my-random-field") - .build())); - // Bad attributes type - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addByteArrayField(RowHandler.ATTRIBUTES_FIELD) - .build())); - // Bad attributes field names - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addRowField( - RowHandler.ATTRIBUTES_FIELD, - Schema.builder() - .addStringField(RowHandler.ATTRIBUTES_KEY_FIELD) - .addArrayField("badValues", FieldType.BYTES) - .build()) - .build())); - // Bad event timestamp type - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addByteArrayField(RowHandler.EVENT_TIMESTAMP_FIELD) - .build())); - // Bad publish timestamp type - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addByteArrayField(RowHandler.PUBLISH_TIMESTAMP_FIELD) - .build())); - // Bad message key type - assertThrows( - IllegalArgumentException.class, - () -> - tableMaker.apply( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addStringField(RowHandler.MESSAGE_KEY_FIELD) - .build())); - } - - @Test - public void bytesWithFormatInvalid() { - assertThrows( - IllegalArgumentException.class, - () -> - makeTable( - Schema.builder().addByteArrayField(RowHandler.PAYLOAD_FIELD).build(), - example(SubscriptionPath.class).toString(), - ImmutableMap.of("format", "json"))); - } - - @Test - public void invalidLocations() { - Function tableMaker = - location -> makeTable(FULL_WRITE_SCHEMA, location, ImmutableMap.of()); - - // Just nonsense - assertThrows(IllegalArgumentException.class, () -> tableMaker.apply("my-location")); - // CPS topic - assertThrows(ApiException.class, () -> tableMaker.apply("projects/abc/topics/def")); - // CPS subscription - assertThrows(ApiException.class, () -> tableMaker.apply("projects/abc/subscriptions/def")); - } - - @Test - public void validTopicTables() { - BeamSqlTable basic = - makeTable(FULL_WRITE_SCHEMA, example(TopicPath.class).toString(), ImmutableMap.of()); - assertTrue(basic instanceof PubsubLiteTopicTable); - BeamSqlTable row = - makeTable( - Schema.builder() - .addRowField( - RowHandler.PAYLOAD_FIELD, Schema.builder().addStringField("abc").build()) - .build(), - example(TopicPath.class).toString(), - ImmutableMap.of("format", "json")); // Defaults to json - assertTrue(row instanceof PubsubLiteTopicTable); - BeamSqlTable dlq = - makeTable( - Schema.builder() - .addRowField( - RowHandler.PAYLOAD_FIELD, Schema.builder().addStringField("abc").build()) - .build(), - example(TopicPath.class).toString(), - ImmutableMap.of( - "deadLetterQueue", "pubsub:projects/abc/topics/def")); // Defaults to json - assertTrue(dlq instanceof PubsubLiteTopicTable); - } - - @Test - @SuppressWarnings("argument") - public void topicTableCannotRead() { - BeamSqlTable basic = - makeTable(FULL_WRITE_SCHEMA, example(TopicPath.class).toString(), ImmutableMap.of()); - assertThrows(UnsupportedOperationException.class, () -> basic.buildIOReader(null)); - } - - @Test - public void validSubscriptionTables() { - BeamSqlTable basic = - makeTable(FULL_READ_SCHEMA, example(SubscriptionPath.class).toString(), ImmutableMap.of()); - assertTrue(basic instanceof PubsubLiteSubscriptionTable); - BeamSqlTable row = - makeTable( - Schema.builder() - .addRowField( - RowHandler.PAYLOAD_FIELD, Schema.builder().addStringField("abc").build()) - .build(), - example(SubscriptionPath.class).toString(), - ImmutableMap.of("format", "json")); - assertTrue(row instanceof PubsubLiteSubscriptionTable); - BeamSqlTable dlq = - makeTable( - Schema.builder() - .addRowField( - RowHandler.PAYLOAD_FIELD, Schema.builder().addStringField("abc").build()) - .build(), - example(SubscriptionPath.class).toString(), - ImmutableMap.of("format", "json", "deadLetterQueue", "pubsub:projects/abc/topics/def")); - assertTrue(dlq instanceof PubsubLiteSubscriptionTable); - } - - @Test - @SuppressWarnings("argument") - public void subscriptionTableCannotWrite() { - BeamSqlTable basic = - makeTable(FULL_READ_SCHEMA, example(SubscriptionPath.class).toString(), ImmutableMap.of()); - assertThrows(UnsupportedOperationException.class, () -> basic.buildIOWriter(null)); - } -} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandlerTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandlerTest.java deleted file mode 100644 index 8e7fecd54800..000000000000 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsublite/RowHandlerTest.java +++ /dev/null @@ -1,237 +0,0 @@ -/* - * 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.extensions.sql.meta.provider.pubsublite; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doThrow; -import static org.mockito.MockitoAnnotations.openMocks; - -import com.google.cloud.pubsublite.proto.AttributeValues; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import com.google.protobuf.ByteString; -import com.google.protobuf.util.Timestamps; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.joda.time.Instant; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mock; - -@RunWith(JUnit4.class) -@SuppressWarnings("initialization.fields.uninitialized") -public class RowHandlerTest { - private static final Schema FULL_WRITE_SCHEMA = - Schema.builder() - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .addField(RowHandler.EVENT_TIMESTAMP_FIELD, FieldType.DATETIME.withNullable(true)) - .addArrayField( - RowHandler.ATTRIBUTES_FIELD, FieldType.row(RowHandler.ATTRIBUTES_ENTRY_SCHEMA)) - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .build(); - private static final Schema FULL_READ_SCHEMA = - Schema.builder() - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .addField(RowHandler.EVENT_TIMESTAMP_FIELD, FieldType.DATETIME.withNullable(true)) - .addArrayField( - RowHandler.ATTRIBUTES_FIELD, FieldType.row(RowHandler.ATTRIBUTES_ENTRY_SCHEMA)) - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addDateTimeField(RowHandler.PUBLISH_TIMESTAMP_FIELD) - .build(); - - @Mock public PayloadSerializer serializer; - - @Before - public void setUp() { - openMocks(this); - } - - @Test - public void constructionFailures() { - // Row payload without serializer - assertThrows( - IllegalArgumentException.class, - () -> - new RowHandler( - Schema.builder() - .addRowField( - RowHandler.PAYLOAD_FIELD, Schema.builder().addStringField("abc").build()) - .build())); - // Bytes payload with serializer - assertThrows( - IllegalArgumentException.class, - () -> - new RowHandler( - Schema.builder().addByteArrayField(RowHandler.PAYLOAD_FIELD).build(), serializer)); - } - - @Test - public void messageToRowFailures() { - { - Schema payloadSchema = Schema.builder().addStringField("def").build(); - RowHandler rowHandler = - new RowHandler( - Schema.builder().addRowField(RowHandler.PAYLOAD_FIELD, payloadSchema).build(), - serializer); - doThrow(new IllegalArgumentException("")).when(serializer).deserialize(any()); - assertThrows( - IllegalArgumentException.class, - () -> rowHandler.messageToRow(SequencedMessage.getDefaultInstance())); - } - // Schema requires event time, missing in message - { - RowHandler rowHandler = - new RowHandler( - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addDateTimeField(RowHandler.EVENT_TIMESTAMP_FIELD) - .build()); - assertThrows( - IllegalArgumentException.class, - () -> rowHandler.messageToRow(SequencedMessage.getDefaultInstance())); - } - } - - @Test - public void rowToMessageFailures() { - Schema payloadSchema = Schema.builder().addStringField("def").build(); - Schema schema = Schema.builder().addRowField(RowHandler.PAYLOAD_FIELD, payloadSchema).build(); - RowHandler rowHandler = new RowHandler(schema, serializer); - // badRow cannot be cast to schema - Schema badRowSchema = Schema.builder().addStringField("xxx").build(); - Row badRow = - Row.withSchema(badRowSchema).attachValues(Row.withSchema(badRowSchema).attachValues("abc")); - assertThrows(IllegalArgumentException.class, () -> rowHandler.rowToMessage(badRow)); - - Row goodRow = - Row.withSchema(schema).addValue(Row.withSchema(payloadSchema).attachValues("abc")).build(); - doThrow(new IllegalArgumentException("")).when(serializer).serialize(any()); - assertThrows(IllegalArgumentException.class, () -> rowHandler.rowToMessage(goodRow)); - } - - @Test - public void reorderRowToMessage() { - Schema schema = - Schema.builder() - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .build(); - Schema rowSchema = - Schema.builder() - .addByteArrayField(RowHandler.PAYLOAD_FIELD) - .addByteArrayField(RowHandler.MESSAGE_KEY_FIELD) - .build(); - RowHandler rowHandler = new RowHandler(schema); - Row row = Row.withSchema(rowSchema).attachValues("abc".getBytes(UTF_8), "def".getBytes(UTF_8)); - PubSubMessage expected = - PubSubMessage.newBuilder() - .setData(ByteString.copyFromUtf8("abc")) - .setKey(ByteString.copyFromUtf8("def")) - .build(); - assertEquals(expected, rowHandler.rowToMessage(row)); - } - - @Test - public void fullRowToMessage() { - RowHandler rowHandler = new RowHandler(FULL_WRITE_SCHEMA); - Instant now = Instant.now(); - Row row = - Row.withSchema(FULL_WRITE_SCHEMA) - .withFieldValue(RowHandler.MESSAGE_KEY_FIELD, "val1".getBytes(UTF_8)) - .withFieldValue(RowHandler.PAYLOAD_FIELD, "val2".getBytes(UTF_8)) - .withFieldValue(RowHandler.EVENT_TIMESTAMP_FIELD, now) - .withFieldValue( - RowHandler.ATTRIBUTES_FIELD, - ImmutableList.of( - Row.withSchema(RowHandler.ATTRIBUTES_ENTRY_SCHEMA) - .attachValues( - "key1", - ImmutableList.of("attr1".getBytes(UTF_8), "attr2".getBytes(UTF_8))), - Row.withSchema(RowHandler.ATTRIBUTES_ENTRY_SCHEMA) - .attachValues("key2", ImmutableList.of("attr3".getBytes(UTF_8))))) - .build(); - PubSubMessage expected = - PubSubMessage.newBuilder() - .setKey(ByteString.copyFromUtf8("val1")) - .setData(ByteString.copyFromUtf8("val2")) - .setEventTime(Timestamps.fromMillis(now.getMillis())) - .putAttributes( - "key1", - AttributeValues.newBuilder() - .addValues(ByteString.copyFromUtf8("attr1")) - .addValues(ByteString.copyFromUtf8("attr2")) - .build()) - .putAttributes( - "key2", - AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("attr3")).build()) - .build(); - assertEquals(expected, rowHandler.rowToMessage(row)); - } - - @Test - public void fullMessageToRow() { - RowHandler rowHandler = new RowHandler(FULL_READ_SCHEMA); - Instant event = Instant.now(); - Instant publish = Instant.now(); - PubSubMessage userMessage = - PubSubMessage.newBuilder() - .setKey(ByteString.copyFromUtf8("val1")) - .setData(ByteString.copyFromUtf8("val2")) - .setEventTime(Timestamps.fromMillis(event.getMillis())) - .putAttributes( - "key1", - AttributeValues.newBuilder() - .addValues(ByteString.copyFromUtf8("attr1")) - .addValues(ByteString.copyFromUtf8("attr2")) - .build()) - .putAttributes( - "key2", - AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("attr3")).build()) - .build(); - SequencedMessage sequencedMessage = - SequencedMessage.newBuilder() - .setMessage(userMessage) - .setPublishTime(Timestamps.fromMillis(publish.getMillis())) - .build(); - Row expected = - Row.withSchema(FULL_READ_SCHEMA) - .withFieldValue(RowHandler.MESSAGE_KEY_FIELD, "val1".getBytes(UTF_8)) - .withFieldValue(RowHandler.PAYLOAD_FIELD, "val2".getBytes(UTF_8)) - .withFieldValue(RowHandler.EVENT_TIMESTAMP_FIELD, event) - .withFieldValue(RowHandler.PUBLISH_TIMESTAMP_FIELD, publish) - .withFieldValue( - RowHandler.ATTRIBUTES_FIELD, - ImmutableList.of( - Row.withSchema(RowHandler.ATTRIBUTES_ENTRY_SCHEMA) - .attachValues( - "key1", - ImmutableList.of("attr1".getBytes(UTF_8), "attr2".getBytes(UTF_8))), - Row.withSchema(RowHandler.ATTRIBUTES_ENTRY_SCHEMA) - .attachValues("key2", ImmutableList.of("attr3".getBytes(UTF_8))))) - .build(); - assertEquals(expected, rowHandler.messageToRow(sequencedMessage)); - } -} diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index 5dd3f9bb761d..260740fbcf09 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -88,7 +88,6 @@ dependencies { permitUnusedDeclared library.java.google_cloud_core_grpc // BEAM-11761 implementation library.java.google_cloud_datastore_v1_proto_client implementation library.java.google_cloud_firestore - implementation library.java.google_cloud_pubsublite // GCP PubSub client is used in TestPubSub implementation library.java.google_cloud_pubsub implementation library.java.google_cloud_spanner @@ -119,8 +118,6 @@ dependencies { implementation library.java.grpc_xds permitUnusedDeclared library.java.grpc_xds // BEAM-11761 implementation library.java.grpc_google_cloud_pubsub_v1 - implementation library.java.grpc_google_cloud_pubsublite_v1 - permitUnusedDeclared library.java.grpc_google_cloud_pubsublite_v1 // BEAM-11761 implementation library.java.guava implementation library.java.http_client implementation library.java.hamcrest @@ -139,7 +136,6 @@ dependencies { implementation library.java.proto_google_cloud_datastore_v1 implementation library.java.proto_google_cloud_firestore_v1 implementation library.java.proto_google_cloud_pubsub_v1 - implementation library.java.proto_google_cloud_pubsublite_v1 implementation library.java.proto_google_cloud_spanner_admin_database_v1 implementation library.java.proto_google_cloud_spanner_v1 implementation library.java.proto_google_common_protos @@ -171,7 +167,7 @@ dependencies { testImplementation library.java.google_cloud_spanner_test testImplementation library.java.google_cloud_bigtable_emulator testRuntimeOnly library.java.slf4j_jdk14 - // everit_json is needed for PubsubLite SchemaTransform tests that rely on JSON-schema translation. + // everit_json is needed for Pubsub SchemaTransform that relies on JSON-schema translation. permitUnusedDeclared library.java.everit_json_schema provided library.java.everit_json_schema } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/CloudPubsubTransforms.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/CloudPubsubTransforms.java deleted file mode 100644 index 1140c11c2767..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/CloudPubsubTransforms.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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 static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.fromCpsPublishTransformer; -import static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.toCpsPublishTransformer; -import static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.toCpsSubscribeTransformer; - -import com.google.cloud.pubsublite.Message; -import com.google.cloud.pubsublite.cloudpubsub.KeyExtractor; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessages; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** A class providing transforms between Cloud Pub/Sub and Pub/Sub Lite message types. */ -public final class CloudPubsubTransforms { - private CloudPubsubTransforms() {} - /** - * Ensure that all messages that pass through can be converted to Cloud Pub/Sub messages using the - * standard transformation methods in the client library. - * - *

Will fail the pipeline if a message has multiple attributes per key. - */ - public static PTransform, PCollection> - ensureUsableAsCloudPubsub() { - return new PTransform, PCollection>() { - @Override - public PCollection expand(PCollection input) { - return input.apply( - MapElements.into(TypeDescriptor.of(PubSubMessage.class)) - .via( - message -> { - Object unused = - toCpsPublishTransformer().transform(Message.fromProto(message)); - return message; - })); - } - }; - } - - /** - * Transform messages read from Pub/Sub Lite to their equivalent Cloud Pub/Sub Message that would - * have been read from PubsubIO. - * - *

Will fail the pipeline if a message has multiple attributes per map key. - */ - public static PTransform, PCollection> - toCloudPubsubMessages() { - return new PTransform, PCollection>() { - @Override - public PCollection expand(PCollection input) { - return input.apply( - MapElements.into(TypeDescriptor.of(PubsubMessage.class)) - .via( - message -> - PubsubMessages.fromProto( - toCpsSubscribeTransformer() - .transform( - com.google.cloud.pubsublite.SequencedMessage.fromProto( - message))))); - } - }; - } - - /** - * Transform messages publishable using PubsubIO to their equivalent Pub/Sub Lite publishable - * message. - */ - public static PTransform, PCollection> - fromCloudPubsubMessages() { - return new PTransform, PCollection>() { - @Override - public PCollection expand(PCollection input) { - return input.apply( - MapElements.into(TypeDescriptor.of(PubSubMessage.class)) - .via( - message -> - fromCpsPublishTransformer(KeyExtractor.DEFAULT) - .transform(PubsubMessages.toProto(message)) - .toProto())); - } - }; - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java deleted file mode 100644 index be9eba9b6140..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.TopicPath; -import com.google.errorprone.annotations.CanIgnoreReturnValue; -import java.io.Serializable; - -/** Options needed for a Pub/Sub Lite Publisher. */ -@AutoValue -public abstract class PublisherOptions implements Serializable { - private static final long serialVersionUID = 275311613L; - - // Required parameters. - public abstract TopicPath topicPath(); - - public static Builder newBuilder() { - return new AutoValue_PublisherOptions.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - // Required parameters. - @CanIgnoreReturnValue - public abstract Builder setTopicPath(TopicPath path); - - public abstract PublisherOptions build(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java deleted file mode 100644 index 1f71baa31ff2..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.cloud.pubsublite.proto.PubSubMessage; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.AddUuidsTransform; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.PubsubLiteSink; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.SubscribeTransform; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.UuidDeduplicationTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; - -/** - * I/O transforms for reading from Google Pub/Sub Lite. - * - *

For the differences between this and Google Pub/Sub, please refer to the product documentation. - * - *

Updates to the I/O connector code

- * - * For any significant updates to this I/O connector, please consider involving corresponding code - * reviewers mentioned - * here. - */ -public final class PubsubLiteIO { - private PubsubLiteIO() {} - - /** - * Read messages from Pub/Sub Lite. These messages may contain duplicates if the publisher - * retried, which the PubsubLiteIO write method will do. Use the dedupe transform to remove these - * duplicates. - * - *

Note that this will not pick up newly added partitions. To pick up new partitions, drain and - * restart the pipeline. - * - *

{@code
-   * Pipeline p = ...;
-   *
-   * SubscriptionPath subscriptionPath =
-   *         SubscriptionPath.newBuilder()
-   *             .setLocation(zone)
-   *             .setProjectNumber(projectNum)
-   *             .setName(subscriptionName)
-   *             .build();
-   *
-   * PCollection messages = p.apply(PubsubLiteIO.read(SubscriberOptions.newBuilder()
-   *     .setSubscriptionPath(subscriptionPath)
-   *     .build()), "read");
-   * }
- */ - public static PTransform> read(SubscriberOptions options) { - return new SubscribeTransform(options); - } - - /** - * Remove duplicates from the PTransform from a read. Assumes by default that the uuids were added - * by a call to PubsubLiteIO.addUuids() when published. - * - *
{@code
-   * PCollection messages = ... (above) ...;
-   * messages = messages.apply(PubsubLiteIO.deduplicate(
-   *     UuidDeduplicationOptions.newBuilder().build()));
-   *
-   * }
- */ - public static PTransform, PCollection> - deduplicate(UuidDeduplicationOptions options) { - return new UuidDeduplicationTransform(options); - } - - /** - * Add Uuids to to-be-published messages that ensures that uniqueness is maintained. - * - *
{@code
-   * PCollection messages = ...;
-   * messages = messages.apply(PubsubLiteIO.addUuids());
-   *
-   * }
- */ - public static PTransform, PCollection> addUuids() { - return new AddUuidsTransform(); - } - - /** - * Write messages to Pub/Sub Lite. - * - *
{@code
-   * TopicPath topicPath =
-   *         TopicPath.newBuilder()
-   *             .setProjectNumber(projectNum)
-   *             .setLocation(zone)
-   *             .setName(topicName)
-   *             .build();
-   *
-   * PCollection messages = ...;
-   * messages.apply(PubsubLiteIO.write(
-   *     PublisherOptions.newBuilder().setTopicPath(topicPath).build());
-   *
-   * }
- */ - public static PTransform, PDone> write(PublisherOptions options) { - return new PTransform, PDone>() { - @Override - public PDone expand(PCollection input) { - PubsubLiteSink sink = new PubsubLiteSink(options); - input.apply("Write", ParDo.of(sink)); - return PDone.in(input.getPipeline()); - } - }; - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java deleted file mode 100644 index ede1bb0a99b0..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.SubscriptionPath; -import com.google.errorprone.annotations.CanIgnoreReturnValue; -import java.io.Serializable; - -@AutoValue -public abstract class SubscriberOptions implements Serializable { - private static final long serialVersionUID = 269598118L; - - // Required parameters. - public abstract SubscriptionPath subscriptionPath(); - - public static Builder newBuilder() { - return new AutoValue_SubscriberOptions.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - public abstract static class Builder { - // Required parameters. - @CanIgnoreReturnValue - public abstract Builder setSubscriptionPath(SubscriptionPath path); - - public abstract SubscriberOptions build(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java deleted file mode 100644 index 8beffc6fda7e..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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 static com.google.cloud.pubsublite.internal.UncheckedApiPreconditions.checkArgument; - -import com.google.auto.value.AutoValue; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import com.google.errorprone.annotations.CanIgnoreReturnValue; -import com.google.protobuf.ByteString; -import java.io.Serializable; -import java.util.List; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.Uuid; -import org.apache.beam.sdk.state.TimeDomain; -import org.apache.beam.sdk.transforms.Deduplicate; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.joda.time.Duration; - -/** Options for deduplicating Pub/Sub Lite messages based on the UUID they were published with. */ -@AutoValue -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public abstract class UuidDeduplicationOptions implements Serializable { - private static final long serialVersionUID = 9837489720893L; - - public static final SerializableFunction DEFAULT_UUID_EXTRACTOR = - message -> { - checkArgument( - message.getMessage().getAttributesMap().containsKey(Uuid.DEFAULT_ATTRIBUTE), - "Uuid attribute missing."); - List attributes = - message.getMessage().getAttributesMap().get(Uuid.DEFAULT_ATTRIBUTE).getValuesList(); - checkArgument(attributes.size() == 1, "Duplicate Uuid attribute values exist."); - return Uuid.of(attributes.get(0)); - }; - - public static final TimeDomain DEFAULT_TIME_DOMAIN = TimeDomain.EVENT_TIME; - public static final Duration DEFAULT_DEDUPLICATE_DURATION = Duration.standardDays(1); - - // All parameters are optional. - public abstract SerializableFunction uuidExtractor(); - - public abstract Deduplicate.KeyedValues deduplicate(); - - @SuppressWarnings("CheckReturnValue") - public static Builder newBuilder() { - Builder builder = new AutoValue_UuidDeduplicationOptions.Builder(); - builder.setUuidExtractor(DEFAULT_UUID_EXTRACTOR); - builder.setDeduplicate( - Deduplicate.keyedValues() - .withTimeDomain(DEFAULT_TIME_DOMAIN) - .withDuration(DEFAULT_DEDUPLICATE_DURATION)); - return builder; - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setUuidExtractor( - SerializableFunction uuidExtractor); - - /** - * Set the deduplication transform. - * - *
{@code
-     * UuidDeduplicationOptions.Builder builder = UuidDeduplicationOptions.newBuilder();
-     * builder.setDeduplicate(Deduplicate.keyedValues()
-     *     .withTimeDomain(TimeDomain.PROCESSING_TIME));
-     * }
- */ - @CanIgnoreReturnValue - public abstract Builder setDeduplicate( - Deduplicate.KeyedValues deduplicate); - - public abstract UuidDeduplicationOptions build(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/AddUuidsTransform.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/AddUuidsTransform.java deleted file mode 100644 index 72e1b7ce055c..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/AddUuidsTransform.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.proto.AttributeValues; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** A transform to add UUIDs to each message to be written to Pub/Sub Lite. */ -public class AddUuidsTransform - extends PTransform, PCollection> { - private static PubSubMessage addUuid(PubSubMessage message) { - PubSubMessage.Builder builder = message.toBuilder(); - builder.putAttributes( - Uuid.DEFAULT_ATTRIBUTE, - AttributeValues.newBuilder().addValues(Uuid.random().value()).build()); - return builder.build(); - } - - @Override - public PCollection expand(PCollection input) { - PCollection withUuids = - input.apply( - "AddUuids", - MapElements.into(new TypeDescriptor() {}) - .via(AddUuidsTransform::addUuid)); - // Reshuffle into 1000 buckets to avoid having unit-sized bundles under high throughput. - return withUuids.apply( - "ShuffleToPersist", Reshuffle.viaRandomKey().withNumBuckets(1000)); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ApiServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ApiServices.java deleted file mode 100644 index 94db1d9d2b02..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ApiServices.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.internal; - -import com.google.api.core.ApiService; -import java.util.concurrent.TimeUnit; - -class ApiServices { - - private ApiServices() {} - - static AutoCloseable asCloseable(ApiService service) { - return () -> service.stopAsync().awaitTerminated(1, TimeUnit.MINUTES); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitter.java deleted file mode 100644 index 67310da34ac3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitter.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.Offset; - -interface BlockingCommitter extends AutoCloseable { - - void commitOffset(Offset offset); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitterImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitterImpl.java deleted file mode 100644 index 406ebb428920..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitterImpl.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; - -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.internal.wire.Committer; -import java.util.concurrent.TimeUnit; - -public class BlockingCommitterImpl implements BlockingCommitter { - - private final Committer committer; - - BlockingCommitterImpl(Committer committer) { - if (!committer.isRunning()) { - throw new IllegalStateException( - "Committer passed to BlockingCommitter which is not running.", committer.failureCause()); - } - this.committer = committer; - } - - @Override - public void commitOffset(Offset offset) { - if (!committer.isRunning()) { - throw new IllegalStateException( - "Committer not running when commitOffset called.", committer.failureCause()); - } - try { - committer.commitOffset(offset).get(1, TimeUnit.MINUTES); - } catch (Exception e) { - throw toCanonical(e).underlying; - } - } - - @Override - public void close() throws Exception { - committer.stopAsync().awaitTerminated(1, TimeUnit.MINUTES); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CheckpointMarkImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CheckpointMarkImpl.java deleted file mode 100644 index 0f8d5f329f2f..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CheckpointMarkImpl.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import com.google.cloud.pubsublite.Offset; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Optional; -import java.util.function.Supplier; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class CheckpointMarkImpl implements CheckpointMark { - - private final Logger logger = LoggerFactory.getLogger(CheckpointMarkImpl.class); - - final Offset offset; - - private final Optional> committer; - - CheckpointMarkImpl(Offset offset, Supplier committer) { - this.offset = offset; - this.committer = Optional.of(committer); - } - - /** Internal-only constructor for deserialization. */ - private CheckpointMarkImpl(Offset offset) { - this.offset = offset; - this.committer = Optional.empty(); - } - - static Coder coder() { - return new AtomicCoder() { - @Override - public void encode(CheckpointMarkImpl value, OutputStream outStream) throws IOException { - VarLongCoder.of().encode(value.offset.value(), outStream); - } - - @Override - public CheckpointMarkImpl decode(InputStream inStream) throws IOException { - return new CheckpointMarkImpl(Offset.of(VarLongCoder.of().decode(inStream))); - } - }; - } - - @Override - public void finalizeCheckpoint() { - try { - checkState(committer.isPresent()); - committer.get().get().commitOffset(offset); - } catch (Exception e) { - logger.warn("Failed to finalize checkpoint.", e); - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java deleted file mode 100644 index 9853e24e8ff3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.internal.wire.SystemExecutors; -import java.util.function.Supplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** A class that safely ensures an object of type T is cleaned up before it is garbage collected. */ -class CloserReference implements Supplier { - - private static final Logger LOG = LoggerFactory.getLogger(CloserReference.class); - - private final T object; - - public static CloserReference of(T object) { - return new CloserReference<>(object); - } - - @Override - public T get() { - return object; - } - - private CloserReference(T object) { - this.object = object; - } - - private static class Closer implements Runnable { - - private final AutoCloseable object; - - private Closer(AutoCloseable object) { - this.object = object; - } - - @Override - public void run() { - try { - object.close(); - } catch (Exception e) { - LOG.warn("Failed to close resource with class: " + object.getClass().getCanonicalName(), e); - } - } - } - - @SuppressWarnings({"deprecation", "Finalize"}) - @Override - protected void finalize() { - SystemExecutors.getFuturesExecutor().execute(new Closer(object)); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/DlqProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/DlqProvider.java deleted file mode 100644 index 8ea2f76fd46e..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/DlqProvider.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.service.AutoService; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.proto.AttributeValues; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.protobuf.ByteString; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; -import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; -import org.apache.beam.sdk.schemas.io.Failure; -import org.apache.beam.sdk.schemas.io.GenericDlqProvider; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.TypeDescriptor; - -@Internal -@AutoService(GenericDlqProvider.class) -public class DlqProvider implements GenericDlqProvider { - @Override - public String identifier() { - return "pubsublite"; - } - - @Override - public PTransform, PDone> newDlqTransform(String config) { - return new DlqTransform(TopicPath.parse(config)); - } - - private static class DlqTransform extends PTransform, PDone> { - private final TopicPath topic; - - DlqTransform(TopicPath topic) { - this.topic = topic; - } - - @Override - public PDone expand(PCollection input) { - return input - .apply( - "Failure to PubSubMessage", - MapElements.into(TypeDescriptor.of(PubSubMessage.class)) - .via(DlqTransform::getMessage)) - .apply( - "Write Failures to Pub/Sub Lite", - PubsubLiteIO.write(PublisherOptions.newBuilder().setTopicPath(topic).build())); - } - - private static PubSubMessage getMessage(Failure failure) { - PubSubMessage.Builder builder = PubSubMessage.newBuilder(); - builder.putAttributes( - "beam-dlq-error", - AttributeValues.newBuilder() - .addValues(ByteString.copyFromUtf8(failure.getError())) - .build()); - builder.setData(ByteString.copyFrom(failure.getPayload())); - return builder.build(); - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformConfig.java deleted file mode 100644 index d0ba67ea0552..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformConfig.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; -import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; -import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; -import org.apache.beam.sdk.io.gcp.pubsublite.UuidDeduplicationOptions; -import org.apache.beam.sdk.transforms.ExternalTransformBuilder; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; - -class ExternalTransformConfig { - private ExternalTransformConfig() {} - - public static class WriteConfig { - private final PublisherOptions.Builder builder = PublisherOptions.newBuilder(); - private boolean addUuids = false; - - public void setTopicPath(String path) { - builder.setTopicPath(TopicPath.parse(path)); - } - - public void setAddUuids(Boolean addUuids) { - this.addUuids = addUuids; - } - } - - public static class WriteExternalBuilder - implements ExternalTransformBuilder, PDone> { - @Override - public PTransform, PDone> buildExternal(WriteConfig configuration) { - PublisherOptions options = configuration.builder.build(); - boolean addUuids = configuration.addUuids; - return new PTransform, PDone>() { - @Override - public PDone expand(PCollection input) { - PCollection messages = - input.apply(new ProtoFromBytes<>(PubSubMessage::parseFrom)); - if (addUuids) { - messages = messages.apply(PubsubLiteIO.addUuids()); - } - return messages.apply(PubsubLiteIO.write(options)); - } - }; - } - } - - public static class ReadConfig { - private final SubscriberOptions.Builder builder = SubscriberOptions.newBuilder(); - private boolean deduplicate = false; - - public void setSubscriptionPath(String path) { - builder.setSubscriptionPath(SubscriptionPath.parse(path)); - } - - public void setDeduplicate(Boolean deduplicate) { - this.deduplicate = deduplicate; - } - } - - public static class ReadExternalBuilder - implements ExternalTransformBuilder> { - @Override - public PTransform> buildExternal(ReadConfig configuration) { - SubscriberOptions options = configuration.builder.build(); - boolean deduplicate = configuration.deduplicate; - return new PTransform>() { - @Override - public PCollection expand(PBegin input) { - PCollection messages = input.apply(PubsubLiteIO.read(options)); - if (deduplicate) { - messages = - messages.apply( - PubsubLiteIO.deduplicate(UuidDeduplicationOptions.newBuilder().build())); - } - return messages.apply(new ProtoToBytes<>()); - } - }; - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformRegistrarImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformRegistrarImpl.java deleted file mode 100644 index 9b7b5b8b51ce..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ExternalTransformRegistrarImpl.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.service.AutoService; -import java.util.Map; -import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.ExternalTransformConfig.ReadExternalBuilder; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.ExternalTransformConfig.WriteExternalBuilder; -import org.apache.beam.sdk.transforms.ExternalTransformBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -@AutoService(ExternalTransformRegistrar.class) -public class ExternalTransformRegistrarImpl implements ExternalTransformRegistrar { - public static final String WRITE_URN = "beam:transform:org.apache.beam:pubsublite_write:v1"; - public static final String READ_URN = "beam:transform:org.apache.beam:pubsublite_read:v1"; - - @Override - public Map> knownBuilderInstances() { - return ImmutableMap.of( - WRITE_URN, new WriteExternalBuilder(), - READ_URN, new ReadExternalBuilder()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReader.java deleted file mode 100644 index 292d3e19a8fb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReader.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.internal; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; - -interface InitialOffsetReader { - Offset read() throws ApiException; -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReaderImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReaderImpl.java deleted file mode 100644 index ddee9b704996..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/InitialOffsetReaderImpl.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static java.util.concurrent.TimeUnit.MINUTES; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.internal.CursorClient; -import java.util.Map; - -class InitialOffsetReaderImpl implements InitialOffsetReader { - private final CursorClient client; - private final SubscriptionPath subscription; - private final Partition partition; - - InitialOffsetReaderImpl( - CursorClient unownedCursorClient, SubscriptionPath subscription, Partition partition) { - this.client = unownedCursorClient; - this.subscription = subscription; - this.partition = partition; - } - - @Override - public Offset read() throws ApiException { - try { - Map results = client.listPartitionCursors(subscription).get(1, MINUTES); - return results.getOrDefault(partition, Offset.of(0)); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/LimitingTopicBacklogReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/LimitingTopicBacklogReader.java deleted file mode 100644 index 43a7d1d20e4b..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/LimitingTopicBacklogReader.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.proto.ComputeMessageStatsResponse; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Ticker; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache; - -final class LimitingTopicBacklogReader implements TopicBacklogReader { - private final TopicBacklogReader underlying; - private final LoadingCache backlogCache; - - @GuardedBy("this") - @Nullable - private Offset currentRequestOffset = null; - - @SuppressWarnings("method.invocation") - LimitingTopicBacklogReader(TopicBacklogReader underlying, Ticker ticker) { - this.underlying = underlying; - backlogCache = - CacheBuilder.newBuilder() - .ticker(ticker) - .maximumSize(1) - .expireAfterWrite(1, TimeUnit.MINUTES) - .refreshAfterWrite(10, TimeUnit.SECONDS) - .build( - new CacheLoader() { - @Override - public ComputeMessageStatsResponse load(String val) { - return loadFromUnderlying(); - } - }); - } - - @SuppressWarnings("argument") - private synchronized ComputeMessageStatsResponse loadFromUnderlying() { - return underlying.computeMessageStats(checkNotNull(currentRequestOffset)); - } - - @Override - public synchronized ComputeMessageStatsResponse computeMessageStats(Offset offset) - throws ApiException { - currentRequestOffset = offset; - try { - // There is only a single entry in the cache. - return backlogCache.get("cache"); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - @Override - public void close() throws Exception { - underlying.close(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactory.java deleted file mode 100644 index 89be581cdff0..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactory.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.internal; - -import java.io.Serializable; - -/** - * A ManagedFactory produces instances and tears down any produced instances when it is itself - * closed. - * - *

close() should never be called on produced instances. - */ -public interface ManagedFactory extends AutoCloseable, Serializable { - - T create(SubscriptionPartition subscriptionPartition); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactoryImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactoryImpl.java deleted file mode 100644 index 6f60483effcd..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ManagedFactoryImpl.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.internal; - -import java.util.HashMap; -import java.util.Map; -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; -import org.apache.beam.sdk.transforms.SerializableFunction; - -public class ManagedFactoryImpl implements ManagedFactory { - - private final SerializableFunction newInstance; - - @GuardedBy("this") - private final Map instances = new HashMap<>(); - - ManagedFactoryImpl(SerializableFunction newInstance) { - this.newInstance = newInstance; - } - - @Override - public synchronized T create(SubscriptionPartition subscriptionPartition) { - return instances.computeIfAbsent(subscriptionPartition, newInstance::apply); - } - - @Override - public synchronized void close() throws Exception { - @Nullable Exception e = null; - for (AutoCloseable c : instances.values()) { - try { - c.close(); - } catch (Exception e2) { - if (e == null) { - e = e2; - } else { - e.addSuppressed(e2); - } - } - } - if (e != null) { - throw e; - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriber.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriber.java deleted file mode 100644 index 1454581703ac..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriber.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.internal; - -import com.google.api.core.ApiService; -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import java.util.Optional; - -interface MemoryBufferedSubscriber extends ApiService { - - /** - * Get the current fetch offset of this subscriber. This offset will be less than or equal to all - * future messages returned by this object. - */ - Offset fetchOffset(); - - /** - * Notify this subscriber that all messages that have been removed with `pop` should no longer be - * counted against the memory budget. - * - *

Acquire a new memory buffer and allow any bytes which are now available to be sent to this - * job. - */ - void rebuffer() throws ApiException; - - /** Return the head message from the buffer if it exists, or empty() otherwise. */ - Optional peek(); - - /** Remove the head message from the buffer. Throws if no messages exist in the buffer. */ - void pop(); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriberImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriberImpl.java deleted file mode 100644 index 92d3c0126d81..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryBufferedSubscriberImpl.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.internal; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.internal.CheckedApiException; -import com.google.cloud.pubsublite.internal.ProxyService; -import com.google.cloud.pubsublite.internal.wire.Subscriber; -import com.google.cloud.pubsublite.proto.FlowControlRequest; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import java.util.ArrayDeque; -import java.util.List; -import java.util.Optional; -import java.util.Queue; -import java.util.function.Consumer; -import java.util.function.Function; -import org.apache.beam.sdk.io.gcp.pubsublite.internal.MemoryLimiter.Block; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class MemoryBufferedSubscriberImpl extends ProxyService implements MemoryBufferedSubscriber { - private static final Logger LOG = LoggerFactory.getLogger(MemoryBufferedSubscriberImpl.class); - - private final Partition partition; - private final MemoryLimiter limiter; - private final Subscriber subscriber; - private long targetMemory; - private Offset fetchOffset; - private Block memBlock; - - private long bytesOutstandingToServer = 0; - private long bytesOutstanding = 0; - private final Queue messages = new ArrayDeque<>(); - private boolean shutdown = false; - - // onReceive will not be called inline as subscriber is not started. - // addServices is intended to be called from the constructor. - @SuppressWarnings({"methodref.receiver.bound", "method.invocation"}) - public MemoryBufferedSubscriberImpl( - Partition partition, - Offset startOffset, - MemoryLimiter limiter, - Function>, Subscriber> subscriberFactory) { - this.partition = partition; - this.fetchOffset = startOffset; - this.limiter = limiter; - this.targetMemory = limiter.maxBlockSize(); - this.subscriber = subscriberFactory.apply(this::onReceive); - addServices(this.subscriber); - memBlock = limiter.claim(targetMemory); - } - - @Override - protected synchronized void start() throws CheckedApiException { - bytesOutstandingToServer += memBlock.claimed(); - bytesOutstanding += memBlock.claimed(); - subscriber.allowFlow( - FlowControlRequest.newBuilder() - .setAllowedBytes(memBlock.claimed()) - .setAllowedMessages(Long.MAX_VALUE) - .build()); - } - - @Override - protected synchronized void stop() { - if (shutdown) { - return; - } - shutdown = true; - memBlock.close(); - } - - @Override - protected synchronized void handlePermanentError(CheckedApiException e) { - stop(); - } - - private synchronized void onReceive(List batch) { - if (shutdown) { - return; - } - for (SequencedMessage message : batch) { - bytesOutstandingToServer -= message.getSizeBytes(); - } - messages.addAll(batch); - } - - @Override - public synchronized Offset fetchOffset() { - return fetchOffset; - } - - @Override - public synchronized void rebuffer() throws ApiException { - if (shutdown) { - return; - } - if (bytesOutstandingToServer < (targetMemory / 3)) { - // Server is delivering lots of data, increase the target so that it is not throttled. - targetMemory = Math.min(limiter.maxBlockSize(), targetMemory * 2); - } else if (bytesOutstandingToServer > (2 * targetMemory / 3)) { - // Server is delivering little data, decrease the target so that memory can be used for other - // users of the limiter. - targetMemory = Math.max(limiter.minBlockSize(), targetMemory / 2); - } - long claimTarget = Math.max(bytesOutstanding, targetMemory); - memBlock.close(); - memBlock = limiter.claim(claimTarget); - long toAllow = Math.max(memBlock.claimed() - bytesOutstanding, 0); - if (toAllow > 0) { - bytesOutstanding += toAllow; - bytesOutstandingToServer += toAllow; - try { - subscriber.allowFlow(FlowControlRequest.newBuilder().setAllowedBytes(toAllow).build()); - } catch (CheckedApiException e) { - throw e.underlying; - } - } else { - LOG.debug( - "Not claiming memory: partition {} outstanding {} to server {} target {} claimed {} messages {}", - partition, - bytesOutstanding, - bytesOutstandingToServer, - targetMemory, - memBlock.claimed(), - messages.size()); - } - } - - @Override - public synchronized Optional peek() { - return Optional.ofNullable(messages.peek()); - } - - @Override - public synchronized void pop() { - SequencedMessage message = messages.remove(); - bytesOutstanding -= message.getSizeBytes(); - fetchOffset = Offset.of(message.getCursor().getOffset() + 1); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiter.java deleted file mode 100644 index 02cee192cba3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.internal; - -/** - * A class which tracks blocks of memory which have been given out, and tries to limit total memory - * size. - */ -interface MemoryLimiter { - Block claim(long toAcquire); - - long minBlockSize(); - - long maxBlockSize(); - - interface Block extends AutoCloseable { - long claimed(); - - @Override - void close(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java deleted file mode 100644 index 753c45e0c1e3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import javax.annotation.concurrent.GuardedBy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class MemoryLimiterImpl implements MemoryLimiter { - private static final Logger LOG = LoggerFactory.getLogger(MemoryLimiterImpl.class); - private final long minBlockSize; - private final long maxBlockSize; - private final long maxAvailable; - - @GuardedBy("this") - private long available; - - public MemoryLimiterImpl(long minBlockSize, long maxBlockSize, long maxAvailable) { - this.minBlockSize = minBlockSize; - this.maxBlockSize = maxBlockSize; - this.maxAvailable = maxAvailable; - this.available = maxAvailable; - } - - @Override - public synchronized Block claim(long toAcquire) { - toAcquire = Math.max(Math.min(toAcquire, available / 2), minBlockSize); - available -= toAcquire; - return new Block(toAcquire); - } - - @Override - public long minBlockSize() { - return minBlockSize; - } - - @Override - public long maxBlockSize() { - return maxBlockSize; - } - - private synchronized void release(long toRelease) { - available += toRelease; - checkState(available <= maxAvailable); - } - - public class Block implements MemoryLimiter.Block { - public final long claimed; - private boolean released = false; - - private Block(long claimed) { - this.claimed = claimed; - } - - @Override - public long claimed() { - return claimed; - } - - @Override - public void close() { - checkState(!released); - released = true; - release(claimed); - } - - @Override - @SuppressWarnings("Finalize") - public void finalize() { - if (!released) { - LOG.error("Failed to release memory block- likely SDF implementation error."); - close(); - } - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteProgress.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteProgress.java deleted file mode 100644 index bc28c1fa32ed..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteProgress.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.value.AutoValue; -import com.google.cloud.pubsublite.Offset; - -/** A representation of progress through a Pub/Sub lite partition. */ -@AutoValue -abstract class OffsetByteProgress { - static OffsetByteProgress of(Offset lastOffset, long batchBytes) { - return new AutoValue_OffsetByteProgress(lastOffset, batchBytes); - } - /** The last offset of the messages received. */ - abstract Offset lastOffset(); - - abstract long batchBytes(); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRange.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRange.java deleted file mode 100644 index 62cf96f1f1f8..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRange.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.value.AutoValue; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.io.range.OffsetRange; - -@AutoValue -@DefaultCoder(OffsetByteRangeCoder.class) -abstract class OffsetByteRange { - abstract OffsetRange getRange(); - - abstract long getByteCount(); - - static OffsetByteRange of(OffsetRange range, long byteCount) { - return new AutoValue_OffsetByteRange(range, byteCount); - } - - static OffsetByteRange of(OffsetRange range) { - return of(range, 0); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeCoder.java deleted file mode 100644 index 0e96d38a051e..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeCoder.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.internal; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderProviders; -import org.apache.beam.sdk.coders.DelegateCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.range.OffsetRange; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TypeDescriptor; - -public class OffsetByteRangeCoder extends AtomicCoder { - private static final Coder CODER = - DelegateCoder.of( - KvCoder.of(OffsetRange.Coder.of(), VarLongCoder.of()), - OffsetByteRangeCoder::toKv, - OffsetByteRangeCoder::fromKv); - - private static KV toKv(OffsetByteRange value) { - return KV.of(value.getRange(), value.getByteCount()); - } - - private static OffsetByteRange fromKv(KV kv) { - return OffsetByteRange.of(kv.getKey(), kv.getValue()); - } - - @Override - public void encode(OffsetByteRange value, OutputStream outStream) throws IOException { - CODER.encode(value, outStream); - } - - @Override - public OffsetByteRange decode(InputStream inStream) throws IOException { - return CODER.decode(inStream); - } - - public static CoderProvider getCoderProvider() { - return CoderProviders.forCoder( - TypeDescriptor.of(OffsetByteRange.class), new OffsetByteRangeCoder()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeTracker.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeTracker.java deleted file mode 100644 index 43233294fa54..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/OffsetByteRangeTracker.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.proto.ComputeMessageStatsResponse; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.range.OffsetRange; -import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; -import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; - -/** - * OffsetByteRangeTracker is an unbounded restriction tracker for Pub/Sub lite partitions that - * tracks offsets for checkpointing and bytes for progress. - */ -class OffsetByteRangeTracker extends TrackerWithProgress { - private final TopicBacklogReader unownedBacklogReader; - private final OffsetRangeTracker rangeTracker; - private long bytes; - private @Nullable Long lastClaimed; - - public OffsetByteRangeTracker(OffsetByteRange range, TopicBacklogReader unownedBacklogReader) { - checkArgument( - range.getRange().getTo() == Long.MAX_VALUE, - "May only construct OffsetByteRangeTracker with an unbounded range with no progress."); - checkArgument( - range.getByteCount() == 0L, - "May only construct OffsetByteRangeTracker with an unbounded range with no progress."); - this.unownedBacklogReader = unownedBacklogReader; - this.rangeTracker = new OffsetRangeTracker(range.getRange()); - this.bytes = range.getByteCount(); - } - - @Override - public IsBounded isBounded() { - return IsBounded.UNBOUNDED; - } - - @Override - public boolean tryClaim(OffsetByteProgress position) { - if (!rangeTracker.tryClaim(position.lastOffset().value())) { - return false; - } - lastClaimed = position.lastOffset().value(); - bytes += position.batchBytes(); - return true; - } - - @Override - public OffsetByteRange currentRestriction() { - return OffsetByteRange.of(rangeTracker.currentRestriction(), bytes); - } - - private long nextOffset() { - checkState(lastClaimed == null || lastClaimed < Long.MAX_VALUE); - return lastClaimed == null ? currentRestriction().getRange().getFrom() : lastClaimed + 1; - } - - @Override - public @Nullable SplitResult trySplit(double fractionOfRemainder) { - // Cannot split a bounded range. This should already be completely claimed. - if (rangeTracker.currentRestriction().getTo() != Long.MAX_VALUE) { - return null; - } - @Nullable SplitResult ranges = rangeTracker.trySplit(fractionOfRemainder); - if (ranges == null) { - return null; - } - checkArgument(rangeTracker.currentRestriction().equals(ranges.getPrimary())); - return SplitResult.of( - currentRestriction(), OffsetByteRange.of(checkArgumentNotNull(ranges.getResidual()))); - } - - @Override - public void checkDone() throws IllegalStateException { - rangeTracker.checkDone(); - } - - @Override - public Progress getProgress() { - ComputeMessageStatsResponse stats = - this.unownedBacklogReader.computeMessageStats(Offset.of(nextOffset())); - return Progress.from(bytes, stats.getMessageBytes()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerPublisherCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerPublisherCache.java deleted file mode 100644 index ee76169dd545..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerPublisherCache.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.MessageMetadata; -import com.google.cloud.pubsublite.internal.Publisher; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; - -/** - * A shared cache per-worker instance of Pub/Sub Lite publishers. - * - *

Pub/Sub Lite publishers connect to all available partitions: it would be a pessimization for - * all instances of the PubsubLiteSink to do this. - */ -final class PerServerPublisherCache { - private PerServerPublisherCache() {} - - static final ServiceCache> PUBLISHER_CACHE = - new ServiceCache<>(); - - static { - Runtime.getRuntime().addShutdownHook(new Thread(PUBLISHER_CACHE::close)); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerSubscriberCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerSubscriberCache.java deleted file mode 100644 index 130ecc87e8e4..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerServerSubscriberCache.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.internal; - -/** - * A shared cache per-worker instance of Pub/Sub Lite subscribers. - * - *

This allows subscribers to buffer in the background while processElement is not running. - */ -final class PerServerSubscriberCache { - private PerServerSubscriberCache() {} - - static final ServiceCache CACHE = - new ServiceCache<>(); - - static { - Runtime.getRuntime().addShutdownHook(new Thread(CACHE::close)); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerSubscriptionPartitionSdf.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerSubscriptionPartitionSdf.java deleted file mode 100644 index e7b387209fed..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PerSubscriptionPartitionSdf.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.internal.ExtractStatus; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.io.range.OffsetRange; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.SerializableBiFunction; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class PerSubscriptionPartitionSdf extends DoFn { - - private static final Logger LOG = LoggerFactory.getLogger(PerSubscriptionPartitionSdf.class); - private final ManagedFactory backlogReaderFactory; - private final ManagedFactory committerFactory; - private final SubscriptionPartitionProcessorFactory processorFactory; - private final SerializableFunction - offsetReaderFactory; - private final SerializableBiFunction - trackerFactory; - - PerSubscriptionPartitionSdf( - ManagedFactory backlogReaderFactory, - ManagedFactory committerFactory, - SerializableFunction offsetReaderFactory, - SerializableBiFunction - trackerFactory, - SubscriptionPartitionProcessorFactory processorFactory) { - this.backlogReaderFactory = backlogReaderFactory; - this.committerFactory = committerFactory; - this.processorFactory = processorFactory; - this.offsetReaderFactory = offsetReaderFactory; - this.trackerFactory = trackerFactory; - } - - @Teardown - public void teardown() throws Exception { - try (AutoCloseable c1 = committerFactory; - AutoCloseable c2 = backlogReaderFactory) {} - } - - /** - * The initial watermark state is not allowed to return less than the element's input timestamp. - * - *

The polling logic for identifying new partitions will export all preexisting partitions with - * very old (EPOCH) initial watermarks, and any new partitions with a recent watermark likely to - * be before all messages that could exist on that partition given the polling delay. - */ - @GetInitialWatermarkEstimatorState - public Instant getInitialWatermarkState(@Timestamp Instant elementTimestamp) { - return elementTimestamp; - } - - @NewWatermarkEstimator - public MonotonicallyIncreasing newWatermarkEstimator(@WatermarkEstimatorState Instant state) { - return new MonotonicallyIncreasing(state); - } - - @ProcessElement - public ProcessContinuation processElement( - RestrictionTracker tracker, - @Element SubscriptionPartition subscriptionPartition, - OutputReceiver receiver) - throws Exception { - LOG.debug("Starting process for {} at {}", subscriptionPartition, Instant.now()); - SubscriptionPartitionProcessor processor = - processorFactory.newProcessor(subscriptionPartition, tracker, receiver); - ProcessContinuation result = processor.run(); - LOG.debug("Starting commit for {} at {}", subscriptionPartition, Instant.now()); - // TODO(dpcollins-google): Move commits to a bundle finalizer for drain correctness - processor - .lastClaimed() - .ifPresent( - lastClaimed -> { - try { - committerFactory - .create(subscriptionPartition) - .commitOffset(Offset.of(lastClaimed.value() + 1)); - } catch (Exception e) { - throw ExtractStatus.toCanonical(e).underlying; - } - }); - LOG.debug("Finishing process for {} at {}", subscriptionPartition, Instant.now()); - return result; - } - - @GetInitialRestriction - public OffsetByteRange getInitialRestriction( - @Element SubscriptionPartition subscriptionPartition) { - Offset offset = offsetReaderFactory.apply(subscriptionPartition).read(); - return OffsetByteRange.of(new OffsetRange(offset.value(), Long.MAX_VALUE /* open interval */)); - } - - @NewTracker - public TrackerWithProgress newTracker( - @Element SubscriptionPartition subscriptionPartition, @Restriction OffsetByteRange range) { - return trackerFactory.apply(backlogReaderFactory.create(subscriptionPartition), range); - } - - @GetSize - public double getSize( - @Element SubscriptionPartition subscriptionPartition, - @Restriction OffsetByteRange restriction) { - if (restriction.getRange().getTo() != Long.MAX_VALUE) { - return restriction.getByteCount(); - } - return newTracker(subscriptionPartition, restriction).getProgress().getWorkRemaining(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoFromBytes.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoFromBytes.java deleted file mode 100644 index f3cd693947eb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoFromBytes.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.internal; - -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.Message; -import java.io.Serializable; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.PCollection; - -public class ProtoFromBytes - extends PTransform, PCollection> { - interface ProtoParser extends Serializable { - T2 parse(byte[] bytes) throws InvalidProtocolBufferException; - } - - private final ProtoParser parser; - - ProtoFromBytes(ProtoParser parser) { - this.parser = parser; - } - - @Override - public PCollection expand(PCollection input) { - return input.apply( - MapElements.via( - new SimpleFunction() { - @Override - public T apply(byte[] bytes) { - try { - return parser.parse(bytes); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - })); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoToBytes.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoToBytes.java deleted file mode 100644 index 553ee3398cd2..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ProtoToBytes.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.internal; - -import com.google.protobuf.Message; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.PCollection; - -public class ProtoToBytes - extends PTransform, PCollection> { - @Override - public PCollection expand(PCollection input) { - return input.apply( - MapElements.via( - new SimpleFunction() { - @Override - public byte[] apply(T message) { - return message.toByteArray(); - } - })); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PublisherAssembler.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PublisherAssembler.java deleted file mode 100644 index 91f0d02c2ba5..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PublisherAssembler.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.cloudpubsub.PublisherSettings.DEFAULT_BATCHING_SETTINGS; -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static com.google.cloud.pubsublite.internal.wire.ServiceClients.addDefaultSettings; -import static com.google.cloud.pubsublite.internal.wire.ServiceClients.getCallContext; - -import com.google.api.gax.rpc.ApiCallContext; -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.AdminClient; -import com.google.cloud.pubsublite.AdminClientSettings; -import com.google.cloud.pubsublite.MessageMetadata; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.internal.Publisher; -import com.google.cloud.pubsublite.internal.wire.PartitionCountWatchingPublisherSettings; -import com.google.cloud.pubsublite.internal.wire.PartitionPublisherFactory; -import com.google.cloud.pubsublite.internal.wire.PubsubContext; -import com.google.cloud.pubsublite.internal.wire.PubsubContext.Framework; -import com.google.cloud.pubsublite.internal.wire.RoutingMetadata; -import com.google.cloud.pubsublite.internal.wire.SinglePartitionPublisherBuilder; -import com.google.cloud.pubsublite.v1.AdminServiceClient; -import com.google.cloud.pubsublite.v1.AdminServiceSettings; -import com.google.cloud.pubsublite.v1.PublisherServiceClient; -import com.google.cloud.pubsublite.v1.PublisherServiceSettings; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; - -class PublisherAssembler { - private static final Framework FRAMEWORK = Framework.of("BEAM"); - - private final PublisherOptions options; - - PublisherAssembler(PublisherOptions options) { - this.options = options; - } - - private AdminClient newAdminClient() throws ApiException { - try { - return AdminClient.create( - AdminClientSettings.newBuilder() - .setServiceClient( - AdminServiceClient.create( - addDefaultSettings( - options.topicPath().location().extractRegion(), - AdminServiceSettings.newBuilder()))) - .setRegion(options.topicPath().location().extractRegion()) - .build()); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - private PublisherServiceClient newServiceClient() { - try { - return PublisherServiceClient.create( - addDefaultSettings( - options.topicPath().location().extractRegion(), - PublisherServiceSettings.newBuilder())); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - private PartitionPublisherFactory getPartitionPublisherFactory() throws ApiException { - PublisherServiceClient client = newServiceClient(); - return new PartitionPublisherFactory() { - @Override - public com.google.cloud.pubsublite.internal.Publisher newPublisher( - Partition partition) throws ApiException { - SinglePartitionPublisherBuilder.Builder singlePartitionBuilder = - SinglePartitionPublisherBuilder.newBuilder() - .setTopic(options.topicPath()) - .setPartition(partition) - .setBatchingSettings(DEFAULT_BATCHING_SETTINGS) - .setStreamFactory( - responseStream -> { - ApiCallContext context = - getCallContext( - PubsubContext.of(FRAMEWORK), - RoutingMetadata.of(options.topicPath(), partition)); - return client.publishCallable().splitCall(responseStream, context); - }); - return singlePartitionBuilder.build(); - } - - @Override - public void close() { - client.close(); - } - }; - } - - Publisher newPublisher() throws ApiException { - return PartitionCountWatchingPublisherSettings.newBuilder() - .setTopic(options.topicPath()) - .setPublisherFactory(getPartitionPublisherFactory()) - .setAdminClient(newAdminClient()) - .build() - .instantiate(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteSink.java deleted file mode 100644 index dbf312ae5a44..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteSink.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.internal; - -import static java.util.concurrent.TimeUnit.MINUTES; - -import com.google.api.core.ApiFuture; -import com.google.api.core.ApiFutures; -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.MessageMetadata; -import com.google.cloud.pubsublite.internal.CheckedApiException; -import com.google.cloud.pubsublite.internal.Publisher; -import com.google.cloud.pubsublite.proto.PubSubMessage; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import java.util.ArrayDeque; -import java.util.Deque; -import org.apache.beam.sdk.io.gcp.pubsublite.PublisherOptions; -import org.apache.beam.sdk.transforms.DoFn; - -/** A sink which publishes messages to Pub/Sub Lite. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class PubsubLiteSink extends DoFn { - private final PublisherOptions options; - - @GuardedBy("this") - private transient RunState runState; - - public PubsubLiteSink(PublisherOptions options) { - this.options = options; - } - - private static class RunState { - private final Deque> futures = new ArrayDeque<>(); - - private final Publisher publisher; - - RunState(PublisherOptions options) { - publisher = - PerServerPublisherCache.PUBLISHER_CACHE.get( - options, () -> new PublisherAssembler(options).newPublisher()); - } - - void publish(PubSubMessage message) { - futures.add(publisher.publish(message)); - } - - void waitForDone() throws Exception { - ApiFutures.allAsList(futures).get(1, MINUTES); - } - } - - @StartBundle - public synchronized void startBundle() throws ApiException { - runState = new RunState(options); - } - - @ProcessElement - public synchronized void processElement(@Element PubSubMessage message) - throws CheckedApiException { - runState.publish(message); - } - - // Intentionally don't flush on bundle finish to allow multi-sink client reuse. - @FinishBundle - public synchronized void finishBundle() throws Exception { - runState.waitForDone(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ServiceCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ServiceCache.java deleted file mode 100644 index fa25933e2b2b..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ServiceCache.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.internal; - -import com.google.api.core.ApiService; -import com.google.api.core.ApiService.Listener; -import com.google.api.core.ApiService.State; -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.internal.wire.ApiServiceUtils; -import com.google.cloud.pubsublite.internal.wire.SystemExecutors; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import java.util.HashMap; -import java.util.function.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** A map of working ApiServices by identifying key. The key must be hashable. */ -class ServiceCache implements AutoCloseable { - private final Logger logger = LoggerFactory.getLogger(ServiceCache.class); - - @GuardedBy("this") - private final HashMap liveMap = new HashMap<>(); - - private synchronized void evict(K key, V service) { - liveMap.remove(key, service); - } - - synchronized V get(K key, Supplier factory) throws ApiException { - V service = liveMap.get(key); - if (service != null) { - return service; - } - V newService = factory.get(); - liveMap.put(key, newService); - newService.addListener( - new Listener() { - @Override - public void failed(State s, Throwable t) { - logger.warn(newService.getClass().getSimpleName() + " failed.", t); - evict(key, newService); - } - - @Override - public void terminated(State from) { - evict(key, newService); - } - }, - SystemExecutors.getFuturesExecutor()); - newService.startAsync().awaitRunning(); - return newService; - } - - @VisibleForTesting - synchronized void set(K key, V service) { - liveMap.put(key, service); - } - - @Override - public synchronized void close() { - ApiServiceUtils.blockingShutdown(liveMap.values()); - liveMap.clear(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java deleted file mode 100644 index b9c187fe0272..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static com.google.cloud.pubsublite.internal.UncheckedApiPreconditions.checkArgument; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.AdminClient; -import com.google.cloud.pubsublite.AdminClientSettings; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.internal.wire.Subscriber; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import java.util.List; -import java.util.Map; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.runners.PTransformOverride; -import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.util.construction.PTransformMatchers; -import org.apache.beam.sdk.util.construction.ReplacementOutputs; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SubscribeTransform extends PTransform> { - - private static final Logger LOG = LoggerFactory.getLogger(SubscribeTransform.class); - - private static final long MEBIBYTE = 1L << 20; - private static final long SOFT_MEMORY_LIMIT = 512 * MEBIBYTE; - private static final long MIN_PER_PARTITION_MEMORY = 10 * MEBIBYTE; - private static final long MAX_PER_PARTITION_MEMORY = 100 * MEBIBYTE; - - private static final MemoryLimiter LIMITER = - new MemoryLimiterImpl(MIN_PER_PARTITION_MEMORY, MAX_PER_PARTITION_MEMORY, SOFT_MEMORY_LIMIT); - - private final SubscriberOptions options; - - public SubscribeTransform(SubscriberOptions options) { - this.options = options; - } - - private void checkSubscription(SubscriptionPartition subscriptionPartition) throws ApiException { - checkArgument(subscriptionPartition.subscription().equals(options.subscriptionPath())); - } - - private Subscriber newSubscriber( - Partition partition, Offset initialOffset, Consumer> consumer) { - try { - return new SubscriberAssembler(options, partition) - .getSubscriberFactory(initialOffset) - .newSubscriber( - messages -> consumer.accept(messages.stream().collect(Collectors.toList()))); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - private MemoryBufferedSubscriber newBufferedSubscriber( - SubscriptionPartition subscriptionPartition, Offset startOffset) throws ApiException { - checkSubscription(subscriptionPartition); - return new MemoryBufferedSubscriberImpl( - subscriptionPartition.partition(), - startOffset, - LIMITER, - consumer -> newSubscriber(subscriptionPartition.partition(), startOffset, consumer)); - } - - private MemoryBufferedSubscriber getCachedSubscriber( - SubscriptionPartition subscriptionPartition, Offset startOffset) { - Supplier getOrCreate = - () -> - PerServerSubscriberCache.CACHE.get( - subscriptionPartition, - () -> newBufferedSubscriber(subscriptionPartition, startOffset)); - while (true) { - MemoryBufferedSubscriber subscriber = getOrCreate.get(); - Offset fetchOffset = subscriber.fetchOffset(); - if (startOffset.equals(fetchOffset)) { - return subscriber; - } - LOG.info( - "Discarding subscriber due to mismatch, this should be rare. {}, start: {} fetch: {}", - subscriptionPartition, - startOffset, - fetchOffset); - try { - subscriber.stopAsync().awaitTerminated(); - } catch (Exception ignored) { - } - } - } - - private SubscriptionPartitionProcessor newPartitionProcessor( - SubscriptionPartition subscriptionPartition, - RestrictionTracker tracker, - OutputReceiver receiver) { - return new SubscriptionPartitionProcessorImpl( - tracker, - receiver, - getCachedSubscriber( - subscriptionPartition, Offset.of(tracker.currentRestriction().getRange().getFrom()))); - } - - private TopicBacklogReader newBacklogReader(SubscriptionPartition subscriptionPartition) { - checkSubscription(subscriptionPartition); - return new SubscriberAssembler(options, subscriptionPartition.partition()).getBacklogReader(); - } - - private TrackerWithProgress newRestrictionTracker( - TopicBacklogReader backlogReader, OffsetByteRange initial) { - return new OffsetByteRangeTracker(initial, backlogReader); - } - - private InitialOffsetReader newInitialOffsetReader(SubscriptionPartition subscriptionPartition) { - checkSubscription(subscriptionPartition); - return new SubscriberAssembler(options, subscriptionPartition.partition()) - .getInitialOffsetReader(); - } - - private BlockingCommitter newCommitter(SubscriptionPartition subscriptionPartition) { - checkSubscription(subscriptionPartition); - return new SubscriberAssembler(options, subscriptionPartition.partition()).newCommitter(); - } - - private TopicPath getTopicPath() { - try (AdminClient admin = - AdminClient.create( - AdminClientSettings.newBuilder() - .setRegion(options.subscriptionPath().location().extractRegion()) - .build())) { - return TopicPath.parse(admin.getSubscription(options.subscriptionPath()).get().getTopic()); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - @SuppressWarnings("unused") - private PCollection expandSdf(PBegin input) { - PCollection subscriptionPartitions = - input.apply(new SubscriptionPartitionLoader(getTopicPath(), options.subscriptionPath())); - return subscriptionPartitions.apply( - ParDo.of( - new PerSubscriptionPartitionSdf( - new ManagedFactoryImpl<>(this::newBacklogReader), - new ManagedFactoryImpl<>(this::newCommitter), - this::newInitialOffsetReader, - this::newRestrictionTracker, - this::newPartitionProcessor))); - } - - private PCollection expandSource(PBegin input) { - return input.apply( - Read.from( - new UnboundedSourceImpl(options, this::newBufferedSubscriber, this::newBacklogReader))); - } - - private static final class SourceTransform - extends PTransform> { - - private final SubscribeTransform impl; - - private SourceTransform(SubscribeTransform impl) { - this.impl = impl; - } - - @Override - public PCollection expand(PBegin input) { - return impl.expandSource(input); - } - } - - public static final PTransformOverride V1_READ_OVERRIDE = - PTransformOverride.of( - PTransformMatchers.classEqualTo(SubscribeTransform.class), new ReadOverrideFactory()); - - private static class ReadOverrideFactory - implements PTransformOverrideFactory< - PBegin, PCollection, SubscribeTransform> { - - @Override - public PTransformReplacement> getReplacementTransform( - AppliedPTransform, SubscribeTransform> transform) { - return PTransformReplacement.of( - transform.getPipeline().begin(), new SourceTransform(transform.getTransform())); - } - - @Override - public Map, ReplacementOutput> mapOutputs( - Map, PCollection> outputs, PCollection newOutput) { - return ReplacementOutputs.singleton(outputs, newOutput); - } - } - - @Override - public PCollection expand(PBegin input) { - return expandSdf(input); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriberAssembler.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriberAssembler.java deleted file mode 100644 index 79efe1ca1bc1..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriberAssembler.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static com.google.cloud.pubsublite.internal.wire.ServiceClients.addDefaultSettings; -import static com.google.cloud.pubsublite.internal.wire.ServiceClients.getCallContext; -import static java.util.concurrent.TimeUnit.MINUTES; - -import com.google.api.gax.rpc.ApiCallContext; -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.AdminClient; -import com.google.cloud.pubsublite.AdminClientSettings; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.internal.CursorClient; -import com.google.cloud.pubsublite.internal.CursorClientSettings; -import com.google.cloud.pubsublite.internal.ExtractStatus; -import com.google.cloud.pubsublite.internal.wire.Committer; -import com.google.cloud.pubsublite.internal.wire.CommitterSettings; -import com.google.cloud.pubsublite.internal.wire.PubsubContext; -import com.google.cloud.pubsublite.internal.wire.PubsubContext.Framework; -import com.google.cloud.pubsublite.internal.wire.RoutingMetadata; -import com.google.cloud.pubsublite.internal.wire.SubscriberBuilder; -import com.google.cloud.pubsublite.internal.wire.SubscriberFactory; -import com.google.cloud.pubsublite.proto.Cursor; -import com.google.cloud.pubsublite.proto.SeekRequest; -import com.google.cloud.pubsublite.v1.CursorServiceClient; -import com.google.cloud.pubsublite.v1.CursorServiceSettings; -import com.google.cloud.pubsublite.v1.SubscriberServiceClient; -import com.google.cloud.pubsublite.v1.SubscriberServiceSettings; -import java.util.concurrent.ConcurrentHashMap; -import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; - -class SubscriberAssembler { - private static final Framework FRAMEWORK = Framework.of("BEAM"); - private static final ConcurrentHashMap KNOWN_PATHS = - new ConcurrentHashMap<>(); - private static final ConcurrentHashMap SUB_CLIENTS = - new ConcurrentHashMap<>(); - private static final ConcurrentHashMap CURSOR_CLIENTS = - new ConcurrentHashMap<>(); - - private final SubscriberOptions options; - private final Partition partition; - - private static TopicPath lookupTopicPath(SubscriptionPath subscriptionPath) { - try (AdminClient adminClient = - AdminClient.create( - AdminClientSettings.newBuilder() - .setRegion(subscriptionPath.location().extractRegion()) - .build())) { - return TopicPath.parse( - adminClient.getSubscription(subscriptionPath).get(1, MINUTES).getTopic()); - } catch (Throwable t) { - throw ExtractStatus.toCanonical(t).underlying; - } - } - - private TopicPath getTopicPath() { - return KNOWN_PATHS.computeIfAbsent( - options.subscriptionPath(), SubscriberAssembler::lookupTopicPath); - } - - private SubscriberServiceClient newSubscriberServiceClient() throws ApiException { - try { - SubscriberServiceSettings.Builder settingsBuilder = SubscriberServiceSettings.newBuilder(); - return SubscriberServiceClient.create( - addDefaultSettings( - options.subscriptionPath().location().extractRegion(), settingsBuilder)); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - private SubscriberServiceClient getSubscriberServiceClient() { - return SUB_CLIENTS.computeIfAbsent( - options.subscriptionPath(), path -> newSubscriberServiceClient()); - } - - private CursorServiceClient newCursorClient() throws ApiException { - try { - CursorServiceSettings.Builder settingsBuilder = CursorServiceSettings.newBuilder(); - return CursorServiceClient.create( - addDefaultSettings( - options.subscriptionPath().location().extractRegion(), settingsBuilder)); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - private CursorServiceClient getCursorClient() { - return CURSOR_CLIENTS.computeIfAbsent(options.subscriptionPath(), path -> newCursorClient()); - } - - SubscriberAssembler(SubscriberOptions options, Partition partition) { - this.options = options; - this.partition = partition; - } - - SubscriberFactory getSubscriberFactory(Offset initialOffset) { - SubscriberServiceClient client = getSubscriberServiceClient(); - return consumer -> - SubscriberBuilder.newBuilder() - .setMessageConsumer(consumer) - .setSubscriptionPath(options.subscriptionPath()) - .setPartition(partition) - .setRetryStreamRaces(false) - .setStreamFactory( - responseStream -> { - ApiCallContext context = - getCallContext( - PubsubContext.of(FRAMEWORK), - RoutingMetadata.of(options.subscriptionPath(), partition)); - return client.subscribeCallable().splitCall(responseStream, context); - }) - .setInitialLocation( - SeekRequest.newBuilder() - .setCursor(Cursor.newBuilder().setOffset(initialOffset.value())) - .build()) - .build(); - } - - BlockingCommitter newCommitter() { - CursorServiceClient client = getCursorClient(); - Committer committer = - CommitterSettings.newBuilder() - .setPartition(partition) - .setSubscriptionPath(options.subscriptionPath()) - .setStreamFactory( - responseStream -> { - ApiCallContext context = - getCallContext( - PubsubContext.of(FRAMEWORK), - RoutingMetadata.of(options.subscriptionPath(), partition)); - return client.streamingCommitCursorCallable().splitCall(responseStream, context); - }) - .build() - .instantiate(); - committer.startAsync().awaitRunning(); - return new BlockingCommitterImpl(committer); - } - - TopicBacklogReader getBacklogReader() { - return TopicBacklogReaderSettings.newBuilder() - .setTopicPath(getTopicPath()) - .setPartition(partition) - .build() - .instantiate(); - } - - InitialOffsetReader getInitialOffsetReader() { - return new InitialOffsetReaderImpl( - CursorClient.create( - CursorClientSettings.newBuilder() - .setServiceClient(getCursorClient()) - .setRegion(options.subscriptionPath().location().extractRegion()) - .build()), - options.subscriptionPath(), - partition); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartition.java deleted file mode 100644 index 4d2f5a11c79c..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartition.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.value.AutoValue; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.SubscriptionPath; -import org.apache.beam.sdk.coders.DefaultCoder; - -@AutoValue -@DefaultCoder(SubscriptionPartitionCoder.class) -public abstract class SubscriptionPartition { - static SubscriptionPartition of(SubscriptionPath subscription, Partition partition) { - return new AutoValue_SubscriptionPartition(subscription, partition); - } - - abstract SubscriptionPath subscription(); - - abstract Partition partition(); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionCoder.java deleted file mode 100644 index feb2d8717705..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionCoder.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.SubscriptionPath; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderProviders; -import org.apache.beam.sdk.coders.DelegateCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TypeDescriptor; - -public class SubscriptionPartitionCoder extends AtomicCoder { - private static final Coder CODER = - DelegateCoder.of( - KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()), - SubscriptionPartitionCoder::toKv, - SubscriptionPartitionCoder::fromKv); - - private static KV toKv(SubscriptionPartition value) { - return KV.of(value.subscription().toString(), value.partition().value()); - } - - private static SubscriptionPartition fromKv(KV kv) { - return SubscriptionPartition.of( - SubscriptionPath.parse(kv.getKey()), Partition.of(kv.getValue())); - } - - @Override - public void encode(SubscriptionPartition value, OutputStream outStream) throws IOException { - CODER.encode(value, outStream); - } - - @Override - public SubscriptionPartition decode(InputStream inStream) throws IOException { - return CODER.decode(inStream); - } - - public static CoderProvider getCoderProvider() { - return CoderProviders.forCoder( - TypeDescriptor.of(SubscriptionPartition.class), new SubscriptionPartitionCoder()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionLoader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionLoader.java deleted file mode 100644 index 60ba6e0c65c0..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionLoader.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.PartitionLookupUtils; -import com.google.cloud.pubsublite.SubscriptionPath; -import com.google.cloud.pubsublite.TopicPath; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Impulse; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; -import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators; -import org.apache.beam.sdk.util.SerializableSupplier; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; - -class SubscriptionPartitionLoader extends PTransform> { - private final TopicPath topic; - private final SubscriptionPath subscription; - private final SerializableFunction getPartitionCount; - private final Duration pollDuration; - private final SerializableSupplier terminate; - - private class GeneratorFn extends DoFn { - @ProcessElement - public ProcessContinuation processElement( - RestrictionTracker restrictionTracker, - OutputReceiver output, - ManualWatermarkEstimator estimator) { - int previousCount = restrictionTracker.currentRestriction(); - int newCount = getPartitionCount.apply(topic); - if (!restrictionTracker.tryClaim(newCount)) { - return ProcessContinuation.stop(); - } - if (newCount > previousCount) { - for (int i = previousCount; i < newCount; ++i) { - output.outputWithTimestamp( - SubscriptionPartition.of(subscription, Partition.of(i)), - estimator.currentWatermark()); - } - } - estimator.setWatermark(getWatermark()); - return ProcessContinuation.resume().withResumeDelay(pollDuration); - } - - @GetInitialWatermarkEstimatorState - public Instant getInitialWatermarkEstimatorState(@Timestamp Instant initial) { - // TODO: Add back when dataflow is fixed. - // checkArgument(initial.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)); - // return initial; - return Instant.EPOCH; - } - - @GetInitialRestriction - public Integer getInitialRestriction() { - return 0; - } - - @NewTracker - public RestrictionTracker newTracker(@Restriction Integer input) { - return new RestrictionTracker() { - private boolean terminated = false; - private int position = input; - - @Override - public boolean tryClaim(Integer newPosition) { - checkArgument(newPosition >= position); - if (terminated) { - return false; - } - if (terminate.get()) { - terminated = true; - return false; - } - position = newPosition; - return true; - } - - @Override - public Integer currentRestriction() { - return position; - } - - @Override - public @Nullable SplitResult trySplit(double fractionOfRemainder) { - if (fractionOfRemainder != 0) { - return null; - } - if (terminated) { - return null; - } - terminated = true; - return SplitResult.of(position, position); - } - - @Override - public void checkDone() throws IllegalStateException { - checkState(terminated); - } - - @Override - public IsBounded isBounded() { - return IsBounded.UNBOUNDED; - } - }; - } - - @NewWatermarkEstimator - public ManualWatermarkEstimator newWatermarkEstimator( - @WatermarkEstimatorState Instant state) { - return new WatermarkEstimators.Manual(state); - } - - private Instant getWatermark() { - return Instant.now().minus(watermarkDelay()); - } - - private Duration watermarkDelay() { - return pollDuration.multipliedBy(3).dividedBy(2); - } - } - - SubscriptionPartitionLoader(TopicPath topic, SubscriptionPath subscription) { - this( - topic, - subscription, - PartitionLookupUtils::numPartitions, - Duration.standardMinutes(1), - () -> false); - } - - @VisibleForTesting - SubscriptionPartitionLoader( - TopicPath topic, - SubscriptionPath subscription, - SerializableFunction getPartitionCount, - Duration pollDuration, - SerializableSupplier terminate) { - this.topic = topic; - this.subscription = subscription; - this.getPartitionCount = getPartitionCount; - this.pollDuration = pollDuration; - this.terminate = terminate; - } - - @Override - public PCollection expand(PBegin input) { - return input - .apply("Impulse", Impulse.create()) - .apply("Watch Partition Count", ParDo.of(new GeneratorFn())); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessor.java deleted file mode 100644 index 768f6284c7d7..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessor.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.Offset; -import java.util.Optional; -import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; - -interface SubscriptionPartitionProcessor { - ProcessContinuation run(); - - Optional lastClaimed(); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorFactory.java deleted file mode 100644 index 62119678930d..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorFactory.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.proto.SequencedMessage; -import java.io.Serializable; -import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; - -interface SubscriptionPartitionProcessorFactory extends Serializable { - long serialVersionUID = 765145146544654L; - - SubscriptionPartitionProcessor newProcessor( - SubscriptionPartition subscriptionPartition, - RestrictionTracker tracker, - OutputReceiver receiver); -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorImpl.java deleted file mode 100644 index 00b615edd3e0..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionProcessorImpl.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import com.google.protobuf.util.Timestamps; -import java.util.Optional; -import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; -import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.joda.time.Instant; - -class SubscriptionPartitionProcessorImpl implements SubscriptionPartitionProcessor { - private final RestrictionTracker tracker; - private final OutputReceiver receiver; - private final MemoryBufferedSubscriber subscriber; - private Optional lastClaimedOffset = Optional.empty(); - - SubscriptionPartitionProcessorImpl( - RestrictionTracker tracker, - OutputReceiver receiver, - MemoryBufferedSubscriber subscriber) { - checkArgument( - Offset.of(tracker.currentRestriction().getRange().getFrom()) - .equals(subscriber.fetchOffset())); - this.tracker = tracker; - this.receiver = receiver; - this.subscriber = subscriber; - subscriber.rebuffer(); - } - - @Override - public ProcessContinuation run() { - // Read any available data. - for (Optional next = subscriber.peek(); - next.isPresent(); - next = subscriber.peek()) { - SequencedMessage message = next.get(); - Offset messageOffset = Offset.of(message.getCursor().getOffset()); - if (tracker.tryClaim(OffsetByteProgress.of(messageOffset, message.getSizeBytes()))) { - subscriber.pop(); - lastClaimedOffset = Optional.of(messageOffset); - receiver.outputWithTimestamp( - message, new Instant(Timestamps.toMillis(message.getPublishTime()))); - } else { - // Our claim failed, return stop() - return ProcessContinuation.stop(); - } - } - // There is no more data available, yield to the runtime. - return ProcessContinuation.resume(); - } - - @Override - public Optional lastClaimed() { - return lastClaimedOffset; - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReader.java deleted file mode 100644 index 28beaf2010a4..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReader.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.internal; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.proto.ComputeMessageStatsResponse; - -/** - * The TopicBacklogReader uses the TopicStats API to aggregate the backlog, or the distance between - * the current cursor and HEAD for a single {subscription, partition} pair. - */ -interface TopicBacklogReader extends AutoCloseable { - - /** - * Compute and aggregate message statistics for message between the provided start offset and - * HEAD. This method is blocking. - * - * @param offset The current offset of the subscriber. - * @return A ComputeMessageStatsResponse with the aggregated statistics for messages in the - * backlog. - */ - ComputeMessageStatsResponse computeMessageStats(Offset offset) throws ApiException; -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderImpl.java deleted file mode 100644 index b11daa43a020..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderImpl.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.internal; - -import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical; -import static java.util.concurrent.TimeUnit.MINUTES; - -import com.google.api.gax.rpc.ApiException; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.internal.TopicStatsClient; -import com.google.cloud.pubsublite.proto.ComputeMessageStatsResponse; - -final class TopicBacklogReaderImpl implements TopicBacklogReader { - private final TopicStatsClient client; - private final TopicPath topicPath; - private final Partition partition; - - public TopicBacklogReaderImpl(TopicStatsClient client, TopicPath topicPath, Partition partition) { - this.client = client; - this.topicPath = topicPath; - this.partition = partition; - } - - @Override - @SuppressWarnings("assignment") - public ComputeMessageStatsResponse computeMessageStats(Offset offset) throws ApiException { - try { - return client - .computeMessageStats(topicPath, partition, offset, Offset.of(Long.MAX_VALUE)) - .get(1, MINUTES); - } catch (Throwable t) { - throw toCanonical(t).underlying; - } - } - - @Override - public void close() { - client.close(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderSettings.java deleted file mode 100644 index bf6e4f519a34..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TopicBacklogReaderSettings.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.internal; - -import com.google.api.gax.rpc.ApiException; -import com.google.auto.value.AutoValue; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.TopicPath; -import com.google.cloud.pubsublite.internal.TopicStatsClient; -import com.google.cloud.pubsublite.internal.TopicStatsClientSettings; -import java.io.Serializable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Ticker; - -@AutoValue -abstract class TopicBacklogReaderSettings implements Serializable { - private static final long serialVersionUID = -4001752066450248673L; - - /** The topic path for this backlog reader. */ - abstract TopicPath topicPath(); - - abstract Partition partition(); - - static Builder newBuilder() { - return new AutoValue_TopicBacklogReaderSettings.Builder(); - } - - @AutoValue.Builder - abstract static class Builder { - - // Required parameters. - abstract Builder setTopicPath(TopicPath topicPath); - - abstract Builder setPartition(Partition partition); - - abstract TopicBacklogReaderSettings build(); - } - - private TopicStatsClient newClient() { - return TopicStatsClient.create( - TopicStatsClientSettings.newBuilder() - .setRegion(topicPath().location().extractRegion()) - .build()); - } - - TopicBacklogReader instantiate() throws ApiException { - TopicBacklogReader impl = new TopicBacklogReaderImpl(newClient(), topicPath(), partition()); - return new LimitingTopicBacklogReader(impl, Ticker.systemTicker()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TrackerWithProgress.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TrackerWithProgress.java deleted file mode 100644 index ae8bdf1832cb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/TrackerWithProgress.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.internal; - -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.HasProgress; - -public abstract class TrackerWithProgress - extends RestrictionTracker implements HasProgress {} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java deleted file mode 100644 index 1fa3ffcc0a2b..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.sdk.io.gcp.pubsublite.internal.ApiServices.asCloseable; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import com.google.api.core.ApiService.State; -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.internal.ExtractStatus; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import com.google.protobuf.util.Timestamps; -import java.io.IOException; -import java.util.NoSuchElementException; -import java.util.Optional; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.joda.time.Instant; - -public class UnboundedReaderImpl extends UnboundedReader { - private final UnboundedSource source; - private final MemoryBufferedSubscriber subscriber; - private final TopicBacklogReader backlogReader; - private final Supplier committer; - - private Offset fetchOffset; - private Optional lastMessageTimestamp = Optional.empty(); - private boolean advanced = false; - - UnboundedReaderImpl( - UnboundedSource source, - MemoryBufferedSubscriber subscriber, - TopicBacklogReader backlogReader, - Supplier committer, - Offset initialOffset) { - checkArgument(initialOffset.equals(subscriber.fetchOffset())); - this.source = source; - this.subscriber = subscriber; - this.backlogReader = backlogReader; - this.committer = committer; - this.fetchOffset = initialOffset; - } - - @Override - public SequencedMessage getCurrent() throws NoSuchElementException { - if (!advanced) { - throw new NoSuchElementException(); - } - return subscriber.peek().get(); - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - return getTimestamp(getCurrent()); - } - - private static Instant getTimestamp(SequencedMessage message) { - return Instant.ofEpochMilli(Timestamps.toMillis(message.getPublishTime())); - } - - @Override - public void close() throws IOException { - try (AutoCloseable c1 = backlogReader; - AutoCloseable c3 = asCloseable(subscriber)) { - } catch (Exception e) { - throw new IOException("Failed when closing reader.", e); - } - } - - @Override - public boolean start() throws IOException { - try { - subscriber.startAsync().awaitRunning(1, TimeUnit.MINUTES); - } catch (Exception e) { - throw new IOException(e); - } - return advance(); - } - - @Override - public boolean advance() throws IOException { - if (!subscriber.state().equals(State.RUNNING)) { - Throwable t = subscriber.failureCause(); - if ("DUPLICATE_SUBSCRIBER_CONNECTIONS" - .equals(ExtractStatus.getErrorInfoReason(ExtractStatus.toCanonical(t)))) { - throw new IOException( - "Partition reassigned to a different worker- this is expected and can be ignored.", t); - } - throw new IOException("Subscriber failed when trying to advance.", t); - } - if (advanced) { - subscriber.pop(); - } - Optional next = subscriber.peek(); - advanced = next.isPresent(); - if (!advanced) { - return false; - } - Offset nextOffset = Offset.of(next.get().getCursor().getOffset() + 1); - checkState(nextOffset.value() > fetchOffset.value()); - fetchOffset = nextOffset; - lastMessageTimestamp = Optional.of(getTimestamp(next.get())); - return true; - } - - @Override - public Instant getWatermark() { - return lastMessageTimestamp.orElse(BoundedWindow.TIMESTAMP_MIN_VALUE); - } - - @Override - public CheckpointMarkImpl getCheckpointMark() { - // By checkpointing, the runtime indicates it has finished processing all data it has already - // pulled. This means we can ask Pub/Sub Lite to refill our in-memory buffer without causing - // unbounded memory usage. - subscriber.rebuffer(); - return new CheckpointMarkImpl(fetchOffset, committer); - } - - @Override - public UnboundedSource getCurrentSource() { - return source; - } - - @Override - public long getSplitBacklogBytes() { - return backlogReader.computeMessageStats(fetchOffset).getMessageBytes(); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedSourceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedSourceImpl.java deleted file mode 100644 index fab104bd32ca..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedSourceImpl.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.internal; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - -import com.google.cloud.pubsublite.Offset; -import com.google.cloud.pubsublite.Partition; -import com.google.cloud.pubsublite.PartitionLookupUtils; -import com.google.cloud.pubsublite.proto.SequencedMessage; -import java.io.IOException; -import java.io.Serializable; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class UnboundedSourceImpl extends UnboundedSource { - - private final SubscriberOptions subscriberOptions; - private final SubscriberFactory subscriberFactory; - private final BacklogReaderFactory readerFactory; - private final Optional partition; - - interface SubscriberFactory extends Serializable { - - MemoryBufferedSubscriber create(SubscriptionPartition subscriptionPartition, Offset offset); - } - - interface BacklogReaderFactory extends Serializable { - - TopicBacklogReader create(SubscriptionPartition subscriptionPartition); - } - - UnboundedSourceImpl( - SubscriberOptions subscriberOptions, - SubscriberFactory subscriberFactory, - BacklogReaderFactory readerFactory) { - this.subscriberOptions = subscriberOptions; - this.subscriberFactory = subscriberFactory; - this.readerFactory = readerFactory; - this.partition = Optional.absent(); - } - - private UnboundedSourceImpl( - SubscriberOptions subscriberOptions, - SubscriberFactory subscriberFactory, - BacklogReaderFactory readerFactory, - Partition partition) { - this.subscriberOptions = subscriberOptions; - this.subscriberFactory = subscriberFactory; - this.readerFactory = readerFactory; - this.partition = Optional.of(partition); - } - - @Override - public List> split( - int desiredNumSplits, PipelineOptions options) throws Exception { - checkState(!partition.isPresent()); - int numPartitions = PartitionLookupUtils.numPartitions(subscriberOptions.subscriptionPath()); - return IntStream.range(0, numPartitions) - .mapToObj( - val -> - new UnboundedSourceImpl( - subscriberOptions, subscriberFactory, readerFactory, Partition.of(val))) - .collect(Collectors.toList()); - } - - @Override - public UnboundedReader createReader( - PipelineOptions options, @Nullable CheckpointMarkImpl checkpointMark) throws IOException { - checkState(partition.isPresent()); - SubscriberAssembler assembler = new SubscriberAssembler(subscriberOptions, partition.get()); - Offset initialOffset; - if (checkpointMark == null) { - initialOffset = assembler.getInitialOffsetReader().read(); - } else { - initialOffset = checkpointMark.offset; - } - SubscriptionPartition subscription = - SubscriptionPartition.of(subscriberOptions.subscriptionPath(), partition.get()); - MemoryBufferedSubscriber subscriber = subscriberFactory.create(subscription, initialOffset); - return new UnboundedReaderImpl( - this, - subscriber, - readerFactory.create(subscription), - CloserReference.of(assembler.newCommitter()), - initialOffset); - } - - @Override - public Coder getCheckpointMarkCoder() { - return CheckpointMarkImpl.coder(); - } - - @Override - public Coder getOutputCoder() { - return ProtoCoder.of(SequencedMessage.class); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/Uuid.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/Uuid.java deleted file mode 100644 index 16e5b116432a..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/Uuid.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.internal; - -import com.google.auto.value.AutoValue; -import com.google.protobuf.ByteString; -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.Base64; -import java.util.UUID; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.util.ByteStringOutputStream; - -/** A Uuid storable in a Pub/Sub Lite attribute. */ -@DefaultCoder(UuidCoder.class) -@AutoValue -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public abstract class Uuid { - public static final String DEFAULT_ATTRIBUTE = "x-goog-pubsublite-dataflow-uuid"; - - public abstract ByteString value(); - - public static Uuid of(ByteString value) { - return new AutoValue_Uuid(value); - } - - public static Uuid random() { - UUID uuid = UUID.randomUUID(); - ByteStringOutputStream output = new ByteStringOutputStream(16); - DataOutputStream stream = new DataOutputStream(output); - try { - stream.writeLong(uuid.getMostSignificantBits()); - stream.writeLong(uuid.getLeastSignificantBits()); - } catch (IOException e) { - throw new RuntimeException("Should never have an IOException since there is no io.", e); - } - // Encode to Base64 so the random UUIDs are valid if consumed from the Cloud Pub/Sub client. - return Uuid.of( - ByteString.copyFrom( - Base64.getEncoder().encode(output.toByteString().asReadOnlyByteBuffer()))); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidCoder.java deleted file mode 100644 index d3769b5482e2..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidCoder.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.internal; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderProviders; -import org.apache.beam.sdk.coders.DelegateCoder; -import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** A coder for a Uuid. */ -public class UuidCoder extends AtomicCoder { - private static final Coder CODER = - DelegateCoder.of(ByteStringCoder.of(), Uuid::value, Uuid::of); - - @Override - public void encode(Uuid value, OutputStream outStream) throws IOException { - CODER.encode(value, outStream); - } - - @Override - public Uuid decode(InputStream inStream) throws IOException { - return CODER.decode(inStream); - } - - public static CoderProvider getCoderProvider() { - return CoderProviders.forCoder(TypeDescriptor.of(Uuid.class), new UuidCoder()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidDeduplicationTransform.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidDeduplicationTransform.java deleted file mode 100644 index 4429187a26d5..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UuidDeduplicationTransform.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.internal; - -import com.google.cloud.pubsublite.proto.SequencedMessage; -import org.apache.beam.sdk.io.gcp.pubsublite.UuidDeduplicationOptions; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ProcessFunction; -import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** - * A transform for deduplicating Pub/Sub Lite messages based on the UUID they were published with. - */ -public class UuidDeduplicationTransform - extends PTransform, PCollection> { - private final UuidDeduplicationOptions options; - - public UuidDeduplicationTransform(UuidDeduplicationOptions options) { - this.options = options; - } - - @Override - public PCollection expand(PCollection input) { - ProcessFunction> mapWithKeys = - message -> KV.of(options.uuidExtractor().apply(message), message); - PCollection> uuidMapped = - input.apply( - "MapUuids", - MapElements.into(new TypeDescriptor>() {}).via(mapWithKeys)); - PCollection> unique = - uuidMapped.apply("Deduplicate", options.deduplicate()); - return unique.apply("StripUuids", Values.create()); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/package-info.java deleted file mode 100644 index ada7c9cd0210..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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. - */ -/** Defines transforms for reading and writing from Google Cloud Pub/Sub Lite. */ -@CheckReturnValue -@ParametersAreNonnullByDefault -package org.apache.beam.sdk.io.gcp.pubsublite.internal; - -import com.google.errorprone.annotations.CheckReturnValue; -import javax.annotation.ParametersAreNonnullByDefault; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/package-info.java deleted file mode 100644 index 044c4c8c40eb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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. - */ -/** Defines transforms for reading and writing from Google Cloud Pub/Sub Lite. */ -@CheckReturnValue -@ParametersAreNonnullByDefault -package org.apache.beam.sdk.io.gcp.pubsublite; - -import com.google.errorprone.annotations.CheckReturnValue; -import javax.annotation.ParametersAreNonnullByDefault; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java index df47964b9c5e..d9f63b5177ac 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java @@ -86,7 +86,6 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.iam.v1"), classesInPackage("com.google.spanner.v1"), classesInPackage("com.google.pubsub.v1"), - classesInPackage("com.google.cloud.pubsublite"), Matchers.equalTo(com.google.api.gax.rpc.ApiException.class), Matchers.equalTo(com.google.errorprone.annotations.CheckReturnValue.class), Matchers.>equalTo(com.google.api.gax.rpc.StatusCode.class),