TheNeuralBit commented on a change in pull request #12611: URL: https://github.com/apache/beam/pull/12611#discussion_r483316706
########## File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java ########## @@ -0,0 +1,287 @@ +/* + * 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.spanner; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.TimestampBound; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaTranslation; +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; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; + +/** + * Exposes {@link SpannerIO.WriteRows} and {@link SpannerIO.ReadRows} as an external transform for + * cross-language usage. + */ +@Experimental(Kind.PORTABILITY) +@AutoService(ExternalTransformRegistrar.class) +public class SpannerTransformRegistrar implements ExternalTransformRegistrar { + public static final String WRITE_URN = "beam:external:java:spanner:write:v1"; + public static final String READ_URN = "beam:external:java:spanner:read:v1"; + + @Override + public Map<String, ExternalTransformBuilder<?, ?, ?>> knownBuilderInstances() { + return ImmutableMap.of(WRITE_URN, new WriteBuilder(), READ_URN, new ReadBuilder()); + } + + public abstract static class CrossLanguageConfiguration { + String instanceId; + String databaseId; + String projectId; + @Nullable String host; + @Nullable String emulatorHost; + + public void setInstanceId(String instanceId) { + this.instanceId = instanceId; + } + + public void setDatabaseId(String databaseId) { + this.databaseId = databaseId; + } + + public void setProjectId(String projectId) { + this.projectId = projectId; + } + + public void setHost(@Nullable String host) { + this.host = host; + } + + public void setEmulatorHost(@Nullable String emulatorHost) { + this.emulatorHost = emulatorHost; + } + } + + @Experimental(Kind.PORTABILITY) + public static class ReadBuilder + implements ExternalTransformBuilder<ReadBuilder.Configuration, PBegin, PCollection<Row>> { + + public static class Configuration extends CrossLanguageConfiguration { + // TODO: BEAM-10851 Come up with something to determine schema without this explicit parameter + private Schema schema; + private @Nullable String sql; + private @Nullable String table; + private @Nullable Boolean batching; + private @Nullable String timestampBoundMode; + private @Nullable String readTimestamp; + private @Nullable String timeUnit; + private @Nullable Long exactStaleness; + + public void setSql(@Nullable String sql) { + this.sql = sql; + } + + public void setTable(@Nullable String table) { + this.table = table; + } + + public void setBatching(@Nullable Boolean batching) { + this.batching = batching; + } + + public void setTimestampBoundMode(@Nullable String timestampBoundMode) { + this.timestampBoundMode = timestampBoundMode; + } + + public void setSchema(byte[] schema) throws InvalidProtocolBufferException { + this.schema = SchemaTranslation.schemaFromProto(SchemaApi.Schema.parseFrom(schema)); + } + + public void setReadTimestamp(@Nullable String readTimestamp) { + this.readTimestamp = readTimestamp; + } + + public void setTimeUnit(@Nullable String timeUnit) { + this.timeUnit = timeUnit; + } + + public void setExactStaleness(@Nullable Long exactStaleness) { + this.exactStaleness = exactStaleness; + } + + private TimestampBound getTimestampBound() { + if (timestampBoundMode == null) { + return null; + } + + TimestampBound.Mode mode = TimestampBound.Mode.valueOf(timestampBoundMode); + if (mode == TimestampBound.Mode.MAX_STALENESS + || mode == TimestampBound.Mode.EXACT_STALENESS) { + checkArgument( + exactStaleness != null, + "Staleness value cannot be null when MAX_STALENESS or EXACT_STALENESS mode is selected"); + checkArgument( + timeUnit != null, + "Time unit cannot be null when MAX_STALENESS or EXACT_STALENESS mode is selected"); + } + if (mode == TimestampBound.Mode.READ_TIMESTAMP + || mode == TimestampBound.Mode.MIN_READ_TIMESTAMP) { + checkArgument( + readTimestamp != null, + "Timestamp cannot be null when READ_TIMESTAMP or MIN_READ_TIMESTAMP mode is selected"); + } + switch (mode) { + case STRONG: + return TimestampBound.strong(); + case MAX_STALENESS: + return TimestampBound.ofMaxStaleness(exactStaleness, TimeUnit.valueOf(timeUnit)); + case EXACT_STALENESS: + return TimestampBound.ofExactStaleness(exactStaleness, TimeUnit.valueOf(timeUnit)); + case READ_TIMESTAMP: + return TimestampBound.ofReadTimestamp(Timestamp.parseTimestamp(readTimestamp)); + case MIN_READ_TIMESTAMP: + return TimestampBound.ofMinReadTimestamp(Timestamp.parseTimestamp(readTimestamp)); + default: + throw new RuntimeException("Unknown timestamp bound mode: " + mode); + } + } + + public ReadOperation getReadOperation() { + checkArgument( + sql == null || table == null, + "Query and table params are mutually exclusive. Set just one of them."); + if (sql != null) { + return ReadOperation.create().withQuery(sql); + } + return ReadOperation.create().withTable(table).withColumns(schema.getFieldNames()); + } + } + + @Override + public PTransform<PBegin, PCollection<Row>> buildExternal(Configuration configuration) { + SpannerIO.Read readTransform = + SpannerIO.read() + .withProjectId(configuration.projectId) + .withDatabaseId(configuration.databaseId) + .withInstanceId(configuration.instanceId) + .withReadOperation(configuration.getReadOperation()); + + if (configuration.host != null) { + readTransform = readTransform.withHost(configuration.host); + } + if (configuration.emulatorHost != null) { + readTransform = readTransform.withEmulatorHost(configuration.emulatorHost); + } + if (configuration.getTimestampBound() != null) { + readTransform = readTransform.withTimestampBound(configuration.getTimestampBound()); + } + if (configuration.batching != null) { + readTransform = readTransform.withBatching(configuration.batching); + } + + return new SpannerIO.ReadRows(readTransform, configuration.schema); + } + } + + @Experimental(Kind.PORTABILITY) + public static class WriteBuilder + implements ExternalTransformBuilder<WriteBuilder.Configuration, PCollection<Row>, PDone> { + + public static class Configuration extends CrossLanguageConfiguration { Review comment: FYI now with https://github.com/apache/beam/pull/12481 it's possible to use schema inference for these configuration objects, so you should be able to use `@AutoValue` and `AutoValueSchema` which could save a lot of boiler plate There's an example here: https://github.com/apache/beam/blob/89a2d17624a5f2f445b7199fe7a61ec0eca8205a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java#L305-L315 (it's fine to leave it as-is, just letting you know) ########## File path: sdks/python/apache_beam/io/gcp/spanner.py ########## @@ -0,0 +1,503 @@ +# +# 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. +# + +"""PTransforms for supporting Spanner in Python pipelines. + + These transforms are currently supported by Beam portable + Flink and Spark runners. + + **Setup** + + Transforms provided in this module are cross-language transforms + implemented in the Beam Java SDK. During the pipeline construction, Python SDK + will connect to a Java expansion service to expand these transforms. + To facilitate this, a small amount of setup is needed before using these + transforms in a Beam Python pipeline. + + There are several ways to setup cross-language Spanner transforms. + + * Option 1: use the default expansion service + * Option 2: specify a custom expansion service + + See below for details regarding each of these options. + + *Option 1: Use the default expansion service* + + This is the recommended and easiest setup option for using Python Spanner + transforms. This option is only available for Beam 2.25.0 and later. + + This option requires following pre-requisites before running the Beam + pipeline. + + * Install Java runtime in the computer from where the pipeline is constructed + and make sure that 'java' command is available. + + In this option, Python SDK will either download (for released Beam version) or + build (when running from a Beam Git clone) a expansion service jar and use + that to expand transforms. Currently Spanner transforms use the + 'beam-sdks-java-io-google-cloud-platform-expansion-service' jar for this + purpose. + + *Option 2: specify a custom expansion service* + + In this option, you startup your own expansion service and provide that as + a parameter when using the transforms provided in this module. + + This option requires following pre-requisites before running the Beam + pipeline. + + * Startup your own expansion service. + * Update your pipeline to provide the expansion service address when + initiating Spanner transforms provided in this module. + + Flink Users can use the built-in Expansion Service of the Flink Runner's + Job Server. If you start Flink's Job Server, the expansion service will be + started on port 8097. For a different address, please set the + expansion_service parameter. + + **More information** + + For more information regarding cross-language transforms see: + - https://beam.apache.org/roadmap/portability/ + + For more information specific to Flink runner see: + - https://beam.apache.org/documentation/runners/flink/ +""" + +# pytype: skip-file + +from __future__ import absolute_import + +import typing +import uuid +from typing import List +from typing import NamedTuple +from typing import Optional + +from past.builtins import unicode + +from apache_beam import coders +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import ExternalTransform +from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder +from apache_beam.typehints.schemas import named_tuple_to_schema + +__all__ = [ + 'WriteToSpanner', + 'ReadFromSpanner', + 'MutationCreator', + 'TimestampBoundMode', + 'TimeUnit', +] + + +def default_io_expansion_service(): + return BeamJarExpansionService( + 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar') + + +WriteToSpannerSchema = typing.NamedTuple( + 'WriteToSpannerSchema', + [ + ('instance_id', unicode), + ('database_id', unicode), + ('project_id', Optional[unicode]), + ('max_batch_size_bytes', Optional[int]), + ('max_number_mutations', Optional[int]), + ('max_number_rows', Optional[int]), + ('grouping_factor', Optional[int]), + ('host', Optional[unicode]), + ('emulator_host', Optional[unicode]), + ('commit_deadline', Optional[int]), + ('max_cumulative_backoff', Optional[int]), + ], +) + + +class WriteToSpanner(ExternalTransform): + """ + A PTransform which writes mutations to the specified instance's database + via Spanner. + + This transform receives Mutations defined as NamedTuple which are created + via utility class MutationCreator. Mutation needs to know what row type does + it wrap. Example:: + + ExampleRow = typing.NamedTuple('ExampleRow', + [('id', int), ('name', unicode)]) + coders.registry.register_coder(ExampleRow, coders.RowCoder) + + mutation_creator = MutationCreator('table', ExampleRow, 'ExampleMutation') Review comment: Overall I think it makes a lot of sense to use Rows for the Mutations, with a nested Row for the data, but this API is pretty tricky. Could you look into adding a separate PTransform (or multiple PTransforms) for converting the Rows to mutations? I think an API like this should be possible: ```py pc = ... #some PCollection with a schema pc | RowToMutation.insert('table') | WriteToSpanner(...) OR pc | RowToMutation.insertOrUpdate('table') | WriteToSpanner(...) OR pc | RowToMutation.delete('table') | WriteToSpanner(...) ``` The PTransform would be able to look at the `element_type` of the input PCollection and create a mutation type that wraps it in the `expand` method. There's not a lot of examples of logic like this in the Python SDK (yet) the only one I know of is here: https://github.com/apache/beam/blob/cfa448d121297398312d09c531258a72b413488b/sdks/python/apache_beam/dataframe/schemas.py#L50-L55 That way the user wouldn't need to pass the type they're planning on using to MutationCreator. What do you think of that? ########## File path: sdks/python/apache_beam/io/gcp/spanner.py ########## @@ -0,0 +1,504 @@ +# +# 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. +# + +"""PTransforms for supporting Spanner in Python pipelines. + + These transforms are currently supported by Beam portable + Flink and Spark runners. + + **Setup** + + Transforms provided in this module are cross-language transforms + implemented in the Beam Java SDK. During the pipeline construction, Python SDK + will connect to a Java expansion service to expand these transforms. + To facilitate this, a small amount of setup is needed before using these + transforms in a Beam Python pipeline. + + There are several ways to setup cross-language Spanner transforms. + + * Option 1: use the default expansion service + * Option 2: specify a custom expansion service + + See below for details regarding each of these options. + + *Option 1: Use the default expansion service* + + This is the recommended and easiest setup option for using Python Spanner + transforms. This option is only available for Beam 2.25.0 and later. + + This option requires following pre-requisites before running the Beam + pipeline. + + * Install Java runtime in the computer from where the pipeline is constructed + and make sure that 'java' command is available. + + In this option, Python SDK will either download (for released Beam version) or + build (when running from a Beam Git clone) a expansion service jar and use + that to expand transforms. Currently Spanner transforms use the + 'beam-sdks-java-io-google-cloud-platform-expansion-service' jar for this + purpose. + + *Option 2: specify a custom expansion service* + + In this option, you startup your own expansion service and provide that as + a parameter when using the transforms provided in this module. + + This option requires following pre-requisites before running the Beam + pipeline. + + * Startup your own expansion service. + * Update your pipeline to provide the expansion service address when + initiating Spanner transforms provided in this module. + + Flink Users can use the built-in Expansion Service of the Flink Runner's + Job Server. If you start Flink's Job Server, the expansion service will be + started on port 8097. For a different address, please set the + expansion_service parameter. + + **More information** + + For more information regarding cross-language transforms see: + - https://beam.apache.org/roadmap/portability/ + + For more information specific to Flink runner see: + - https://beam.apache.org/documentation/runners/flink/ +""" + +# pytype: skip-file + +from __future__ import absolute_import + +import typing +import uuid +from typing import List +from typing import NamedTuple +from typing import Optional + +from past.builtins import unicode + +from apache_beam import coders +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import ExternalTransform +from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder +from apache_beam.typehints.schemas import named_tuple_to_schema + +__all__ = [ + 'WriteToSpanner', + 'ReadFromSpanner', + 'MutationCreator', + 'TimestampBoundMode', + 'TimeUnit', +] + + +def default_io_expansion_service(): + return BeamJarExpansionService( + 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar') + + +WriteToSpannerSchema = typing.NamedTuple( + 'WriteToSpannerSchema', + [ + ('instance_id', unicode), + ('database_id', unicode), + ('project_id', Optional[unicode]), + ('batch_size_bytes', Optional[int]), + ('max_num_mutations', Optional[int]), + ('max_num_rows', Optional[int]), + ('grouping_factor', Optional[int]), + ('host', Optional[unicode]), + ('emulator_host', Optional[unicode]), + ('commit_deadline', Optional[int]), + ('max_cumulative_backoff', Optional[int]), + ], +) + + +class WriteToSpanner(ExternalTransform): Review comment: Yeah that makes sense. There's definitely still value in adding this even if we end up preferring the native Python one, since we can use it from the Go SDK in the future. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
