From 8db6693c34712c899a1655d5305f2602a9120e9b Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 4 May 2017 09:21:23 -0700 Subject: [PATCH 01/23] Initial implementation of SpannerIO.Write This closes #2166. --- .../examples/spanner/SpannerCSVLoader.java | 143 +++++++++ pom.xml | 10 +- sdks/java/core/pom.xml | 4 +- sdks/java/io/google-cloud-platform/pom.xml | 33 ++- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 272 ++++++++++++++++++ .../beam/sdk/io/gcp/spanner/package-info.java | 23 ++ .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 2 + 7 files changed, 474 insertions(+), 13 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java new file mode 100644 index 000000000000..eee581da2148 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java @@ -0,0 +1,143 @@ +/* + * 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.examples.spanner; + +import com.google.cloud.spanner.Database; +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerException; +import com.google.cloud.spanner.SpannerOptions; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + + + +/** + * Generalized bulk loader for importing CSV files into Spanner. + * + */ +public class SpannerCSVLoader { + + /** + * Command options specification. + */ + private interface Options extends PipelineOptions { + @Description("Create a sample database") + @Default.Boolean(false) + boolean isCreateDatabase(); + void setCreateDatabase(boolean createDatabase); + + @Description("File to read from ") + @Validation.Required + String getInput(); + void setInput(String value); + + @Description("Instance ID to write to in Spanner") + @Validation.Required + String getInstanceId(); + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Validation.Required + String getDatabaseId(); + void setDatabaseId(String value); + + @Description("Table name") + @Validation.Required + String getTable(); + void setTable(String value); + } + + + /** + * Constructs and executes the processing pipeline based upon command options. + */ + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline p = Pipeline.create(options); + PCollection lines = p.apply(TextIO.Read.from(options.getInput())); + PCollection mutations = lines + .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); + mutations + .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); + p.run().waitUntilFinish(); + } + + public static void createDatabase(Options options) { + Spanner client = SpannerOptions.getDefaultInstance().getService(); + + DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); + try { + databaseAdminClient.dropDatabase(options.getInstanceId(), options + .getDatabaseId()); + } catch (SpannerException e) { + // Does not exist, ignore. + } + Operation op = databaseAdminClient.createDatabase( + options.getInstanceId(), options + .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" + + " Key INT64," + + " Name STRING," + + " Email STRING," + + " Age INT," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + + /** + * A DoFn that creates a Spanner Mutation for each CSV line. + */ + static class NaiveParseCsvFn extends DoFn { + private final String table; + + NaiveParseCsvFn(String table) { + this.table = table; + } + + @ProcessElement + public void processElement(ProcessContext c) { + String line = c.element(); + String[] elements = line.split(","); + if (elements.length != 4) { + return; + } + Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) + .set("Key").to(Long.valueOf(elements[0])) + .set("Name").to(elements[1]) + .set("Email").to(elements[2]) + .set("Age").to(Integer.valueOf(elements[3])) + .build(); + c.output(mutation); + } + } +} diff --git a/pom.xml b/pom.xml index 15428913d9fa..51051eb02692 100644 --- a/pom.xml +++ b/pom.xml @@ -141,13 +141,15 @@ 4.4.1 4.3.5.RELEASE 2.0 + 1.0.0-rc2 2.20 2.20 3.0.2 - + -Werror -Xpkginfo:always nothing + 0.16.0-beta pom @@ -854,6 +856,12 @@ ${google-cloud-bigdataoss.version} + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigdataoss util diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index b27fea880efe..e00d5e3a91f2 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -203,8 +203,8 @@ joda-time - + org.tukaani xz diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 9c1286c4119e..4312d9b3e257 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -77,11 +77,28 @@ jackson-databind + + io.grpc + grpc-core + + + + com.google.api.grpc + grpc-google-common-protos + ${grpc-google-common-protos.version} + + com.google.apis google-api-services-bigquery + + com.google.api + api-common + ${api-common.version} + + com.google.apis google-api-services-pubsub @@ -112,11 +129,6 @@ grpc-auth - - io.grpc - grpc-core - - io.grpc grpc-netty @@ -151,6 +163,12 @@ joda-time + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigtable bigtable-protos @@ -186,11 +204,6 @@ google-auth-library-oauth2-http - - com.google.api.grpc - grpc-google-common-protos - - org.slf4j slf4j-api diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java new file mode 100644 index 000000000000..172ed8f02687 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -0,0 +1,272 @@ +/* + * 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 com.google.common.base.Preconditions.checkNotNull; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; + +import com.google.cloud.spanner.AbortedException; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + + +/** + * Google Cloud Spanner connectors. + * + *

Reading from Cloud Spanner

+ * Status: Not implemented. + * + *

Writing to Cloud Spanner

+ * Status: Experimental. + * + *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. + * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to + * {@link PCollection} and specify instance and database identifiers. + * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud + * Spanner. + * + *

{@code
+ *
+ * Pipeline p = ...;
+ * // Read the CSV file.
+ * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput()));
+ * // Parse the line and convert to mutation.
+ * PCollection mutations = lines.apply("Parse CSV", parseFromCsv());
+ * // Write mutations.
+ * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
+ * p.run();
+ *
+ * }
+ + */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class SpannerIO { + + private SpannerIO() { + } + + @VisibleForTesting + static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + + /** + * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch + * size. + */ + public static Writer writeTo(String instanceId, String databaseId) { + return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + } + + /** + * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * + * @see SpannerIO + */ + public static class Writer extends PTransform, PDone> { + + private final String instanceId; + private final String databaseId; + private int batchSize; + + Writer(String instanceId, String databaseId, int batchSize) { + this.instanceId = instanceId; + this.databaseId = databaseId; + this.batchSize = batchSize; + } + + /** + * Returns a new {@link Writer} with a limit on the number of mutations per batch. + * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + */ + public Writer withBatchSize(Integer batchSize) { + return new Writer(instanceId, databaseId, batchSize); + } + + @Override + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", ParDo.of( + new SpannerWriterFn(instanceId, databaseId, batchSize))); + + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + checkNotNull(instanceId, "instanceId"); + checkNotNull(databaseId, "databaseId"); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("instanceId", instanceId) + .add("databaseId", databaseId) + .toString(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Output Database")); + } + + } + + + /** + * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * + *

See + * + *

Commits are non-transactional. If a commit fails, it will be retried (up to + * {@link SpannerIO#MAX_RETRIES}. times). This means that the + * mutation operation should be idempotent. + */ + @VisibleForTesting + static class SpannerWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private transient Spanner spanner; + private final String instanceId; + private final String databaseId; + private final int batchSize; + private transient DatabaseClient dbClient; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); + + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + + @VisibleForTesting + SpannerWriterFn(String instanceId, String databaseId, int batchSize) { + this.instanceId = checkNotNull(instanceId, "instanceId"); + this.databaseId = checkNotNull(databaseId, "databaseId"); + this.batchSize = batchSize; + } + + @Setup + public void setup() throws Exception { + SpannerOptions options = SpannerOptions.newBuilder().build(); + spanner = options.getService(); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + Mutation m = c.element(); + mutations.add(m); + int columnCount = m.asMap().size(); + if ((mutations.size() + 1) * columnCount >= batchSize) { + flushBatch(); + } + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (!mutations.isEmpty()) { + flushBatch(); + } + } + + @Teardown + public void teardown() throws Exception { + if (spanner == null) { + return; + } + spanner.closeAsync().get(); + } + + /** + * Writes a batch of mutations to Cloud Spanner. + * + *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. + * If the retry limit is exceeded, the last exception from Cloud Spanner will be + * thrown. + * + * @throws AbortedException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws AbortedException, IOException, InterruptedException { + LOG.debug("Writing batch of {} mutations", mutations.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); + + while (true) { + // Batch upsert rows. + try { + dbClient.writeAtLeastOnce(mutations); + + // Break if the commit threw no exception. + break; + } catch (AbortedException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); + } + + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Database")); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java new file mode 100644 index 000000000000..19e468cce041 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 an API for reading from and writing to + * Google Cloud Spanner. + */ +package org.apache.beam.sdk.io.gcp.spanner; 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 7025004cc953..8950452b5db2 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 @@ -75,6 +75,8 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.beam"), classesInPackage("org.apache.commons.logging"), + classesInPackage("com.google.cloud"), + classesInPackage("com.google.cloud.spanner"), // via Bigtable classesInPackage("org.joda.time")); From 491f047726ec2146a634a870efa4a43b9d4d643d Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:42:07 -0700 Subject: [PATCH 02/23] Minor style, compilation, javadoc fixups --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 172ed8f02687..ca6b3246175d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -22,7 +22,6 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; - import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -35,6 +34,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -47,11 +47,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - - - /** - * Google Cloud Spanner connectors. + * {@link PTransform Transforms} for reading from and writing to + * Google Cloud Spanner. * *

Reading from Cloud Spanner

* Status: Not implemented. @@ -61,9 +59,8 @@ * *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. - * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud - * Spanner. + * {@link PCollection} and specify instance and database identifiers. For example, following code + * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. * *

{@code
  *
@@ -77,14 +74,10 @@
  * p.run();
  *
  * }
- */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - private SpannerIO() { - } - @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; @@ -97,7 +90,7 @@ public static Writer writeTo(String instanceId, String databaseId) { } /** - * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * A {@link PTransform} that writes {@link Mutation} objects to Google Cloud Spanner. * * @see SpannerIO */ @@ -130,7 +123,7 @@ public PDone expand(PCollection input) { } @Override - public void validate(PCollection input) { + public void validate(PipelineOptions options) { checkNotNull(instanceId, "instanceId"); checkNotNull(databaseId, "databaseId"); } @@ -152,19 +145,17 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Output Database")); } - } - /** - * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. * - *

See - * *

Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerIO#MAX_RETRIES}. times). This means that the - * mutation operation should be idempotent. + * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be + * idempotent. + * + *

See Google Cloud Spanner documentation. */ @VisibleForTesting static class SpannerWriterFn extends DoFn { @@ -269,4 +260,7 @@ public void populateDisplayData(Builder builder) { .withLabel("Database")); } } + + private SpannerIO() {} // Prevent construction. + } From 7a2d09a0d2783f074348d9b8a1f5bbab7e0a2871 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:24 -0700 Subject: [PATCH 03/23] SpannerCSVLoader: update for TextIO API Change --- .../java/org/apache/beam/examples/spanner/SpannerCSVLoader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java index eee581da2148..9aa8672b87dd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java @@ -84,7 +84,7 @@ public static void main(String[] args) throws Exception { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); Pipeline p = Pipeline.create(options); - PCollection lines = p.apply(TextIO.Read.from(options.getInput())); + PCollection lines = p.apply(TextIO.read().from(options.getInput())); PCollection mutations = lines .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); mutations From 77be2762825def0c9a62c30b02b9d0a8ff2a15f2 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:54 -0700 Subject: [PATCH 04/23] Fix spanner dependency management Also minor cleanup alphabetization in root pom.xml --- pom.xml | 15 +++++++++++---- sdks/java/io/google-cloud-platform/pom.xml | 3 --- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 51051eb02692..51ea5b56e00d 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,7 @@ 0.5.160222 1.4.0 1.3.0 + 1.0.0-rc2 1.0-rc2 1.4.1 0.6.1 @@ -136,15 +137,15 @@ v1-rev10-1.22.0 1.7.14 1.6.2 + 4.3.5.RELEASE 3.1.4 v1-rev71-1.22.0 4.4.1 - 4.3.5.RELEASE - 2.0 - 1.0.0-rc2 - 2.20 + 2.20 3.0.2 + 2.0 + 2.20 -Werror -Xpkginfo:always @@ -599,6 +600,12 @@ ${grpc.version} + + com.google.api + api-common + ${google-api-common.version} + + com.google.api-client google-api-client diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 4312d9b3e257..076020f327a8 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -85,7 +85,6 @@ com.google.api.grpc grpc-google-common-protos - ${grpc-google-common-protos.version} @@ -96,7 +95,6 @@ com.google.api api-common - ${api-common.version} @@ -166,7 +164,6 @@ com.google.cloud google-cloud-spanner - ${spanner.version} From 4049047a38bd5dfa6facee8f3c68086c81873e91 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 12 May 2017 12:59:27 -0700 Subject: [PATCH 05/23] Introduced MutationSizeEstimator. --- .../io/gcp/spanner/MutationSizeEstimator.java | 87 ++++++++++++++ .../beam/sdk/io/gcp/spanner/SpannerIO.java | 55 +++++---- .../spanner/MutationSizeEstimatorTest.java | 109 ++++++++++++++++++ 3 files changed, 223 insertions(+), 28 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java new file mode 100644 index 000000000000..a8c43bfd015a --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java @@ -0,0 +1,87 @@ +package org.apache.beam.sdk.io.gcp.spanner; + +import com.google.cloud.ByteArray; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Value; + +/** + * Estimates the logical size of {@link com.google.cloud.spanner.Mutation}. + */ +class MutationSizeEstimator { + + // Prevent construction. + private MutationSizeEstimator() { + } + + /** + * Estimates a size of mutation in bytes. + */ + static long sizeOf(Mutation m) { + long result = 0; + for (Value v : m.getValues()) { + switch (v.getType().getCode()) { + case ARRAY: + result += estimateArrayValue(v); + break; + case STRUCT: + throw new IllegalArgumentException("Structs are not supported in mutation."); + default: + result += estimatePrimitiveValue(v); + } + } + return result; + } + + private static long estimatePrimitiveValue(Value v) { + switch (v.getType().getCode()) { + case BOOL: + return 1; + case INT64: + case FLOAT64: + return 8; + case DATE: + case TIMESTAMP: + return 12; + case STRING: + return v.isNull()? 0 : v.getString().length(); + case BYTES: + return v.isNull()? 0 : v.getBytes().length(); + } + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } + + private static long estimateArrayValue(Value v) { + switch (v.getType().getArrayElementType().getCode()) { + case BOOL: + return v.getBoolArray().size(); + case INT64: + return 8 * v.getInt64Array().size(); + case FLOAT64: + return 8 * v.getFloat64Array().size(); + case STRING: + long totalLength = 0; + for (String s : v.getStringArray()) { + if (s == null) { + continue; + } + totalLength += s.length(); + } + return totalLength; + case BYTES: + totalLength = 0; + for (ByteArray bytes : v.getBytesArray()) { + if (bytes == null) { + continue; + } + totalLength += bytes.length(); + } + return totalLength; + case DATE: + return 12 * v.getDateArray().size(); + case TIMESTAMP: + return 12 * v.getTimestampArray().size(); + } + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } + +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index ca6b3246175d..8a595c8732f7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -17,22 +17,12 @@ */ package org.apache.beam.sdk.io.gcp.spanner; -import static com.google.common.base.Preconditions.checkNotNull; - import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; -import com.google.cloud.spanner.AbortedException; -import com.google.cloud.spanner.DatabaseClient; -import com.google.cloud.spanner.DatabaseId; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerOptions; +import com.google.cloud.spanner.*; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -47,6 +37,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkNotNull; + /** * {@link PTransform Transforms} for reading from and writing to * Google Cloud Spanner. @@ -78,15 +74,14 @@ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - @VisibleForTesting - static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + private static final long DEFAULT_BATCH_SIZE = 1024 * 1024; // 1 MB - /** - * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch - * size. - */ + /** + * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch + * size. + */ public static Writer writeTo(String instanceId, String databaseId) { - return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + return new Writer(instanceId, databaseId, DEFAULT_BATCH_SIZE); } /** @@ -98,9 +93,9 @@ public static class Writer extends PTransform, PDone> { private final String instanceId; private final String databaseId; - private int batchSize; + private long batchSize; - Writer(String instanceId, String databaseId, int batchSize) { + Writer(String instanceId, String databaseId, long batchSize) { this.instanceId = instanceId; this.databaseId = databaseId; this.batchSize = batchSize; @@ -108,7 +103,7 @@ public static class Writer extends PTransform, PDone> { /** * Returns a new {@link Writer} with a limit on the number of mutations per batch. - * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * Defaults to {@link SpannerIO#DEFAULT_BATCH_SIZE}. */ public Writer withBatchSize(Integer batchSize) { return new Writer(instanceId, databaseId, batchSize); @@ -149,7 +144,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in - * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * batches, where the maximum batch size is {@link SpannerIO#DEFAULT_BATCH_SIZE}. * *

Commits are non-transactional. If a commit fails, it will be retried (up to * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be @@ -163,10 +158,11 @@ static class SpannerWriterFn extends DoFn { private transient Spanner spanner; private final String instanceId; private final String databaseId; - private final int batchSize; + private final long maxBatchSize; private transient DatabaseClient dbClient; // Current batch of mutations to be written. - private final List mutations = new ArrayList<>(); + private List mutations; + private long batchSize = 0; private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -174,10 +170,10 @@ static class SpannerWriterFn extends DoFn { .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(String instanceId, String databaseId, int batchSize) { + SpannerWriterFn(String instanceId, String databaseId, long maxBatchSize) { this.instanceId = checkNotNull(instanceId, "instanceId"); this.databaseId = checkNotNull(databaseId, "databaseId"); - this.batchSize = batchSize; + this.maxBatchSize = maxBatchSize; } @Setup @@ -186,14 +182,16 @@ public void setup() throws Exception { spanner = options.getService(); dbClient = spanner.getDatabaseClient( DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + mutations = new ArrayList<>(); + batchSize = 0; } @ProcessElement public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); mutations.add(m); - int columnCount = m.asMap().size(); - if ((mutations.size() + 1) * columnCount >= batchSize) { + batchSize += MutationSizeEstimator.sizeOf(m); + if (batchSize >= maxBatchSize) { flushBatch(); } } @@ -248,6 +246,7 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } LOG.debug("Successfully wrote {} mutations", mutations.size()); mutations.clear(); + batchSize = 0; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java new file mode 100644 index 000000000000..e9f419894b09 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -0,0 +1,109 @@ +package org.apache.beam.sdk.io.gcp.spanner; + +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import org.junit.Test; + +import java.util.Arrays; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +public class MutationSizeEstimatorTest { + + @Test + public void primitives() throws Exception { + Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2.9).build(); + Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").to(false).build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(1L)); + } + + @Test + public void primitiveArrays() throws Exception { + Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").toInt64Array(new + long[]{1L, 2L, 3L}) + .build(); + Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").toFloat64Array(new + double[]{1., 2.}).build(); + Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").toBoolArray(new + boolean[] {true, true, false, true}) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(16L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(4L)); + } + + @Test + public void strings() throws Exception { + Mutation emptyString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("").build(); + Mutation nullString = Mutation.newInsertOrUpdateBuilder("test").set("one").to((String)null) + .build(); + Mutation sampleString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("abc") + .build(); + Mutation sampleArray = Mutation.newInsertOrUpdateBuilder("test").set("one") + .toStringArray(Arrays.asList("one", "two", null)) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(emptyString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sampleString), is(3L)); + assertThat(MutationSizeEstimator.sizeOf(sampleArray), is(6L)); + } + + @Test + public void bytes() throws Exception { + Mutation empty = Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray + .fromBase64("")) + .build(); + Mutation nullValue = Mutation.newInsertOrUpdateBuilder("test").set("one").to((ByteArray)null) + .build(); + Mutation sample = Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray + .fromBase64("abcdabcd")) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(empty), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullValue), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sample), is(6L)); + } + + @Test + public void dates() throws Exception { + Mutation timestamp = Mutation.newInsertOrUpdateBuilder("test").set("one").to(Timestamp + .now()) + .build(); + Mutation nullTimestamp = Mutation.newInsertOrUpdateBuilder("test").set("one").to( + (Timestamp)null) + .build(); + Mutation date = Mutation.newInsertOrUpdateBuilder("test").set("one").to(Date + .fromYearMonthDay(2017, 10, 10)) + .build(); + Mutation nullDate = Mutation.newInsertOrUpdateBuilder("test").set("one").to( + (Date)null) + .build(); + Mutation timestampArray = Mutation.newInsertOrUpdateBuilder("test").set("one") + .toTimestampArray(Arrays.asList(Timestamp + .now(), null)) + .build(); + Mutation dateArray = Mutation.newInsertOrUpdateBuilder("test").set("one") + .toDateArray(Arrays.asList(null, Date.fromYearMonthDay(2017, 1, 1), null, Date + .fromYearMonthDay(2017, 1, 2))) + .build(); + + + assertThat(MutationSizeEstimator.sizeOf(timestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(date), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullTimestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullDate), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(timestampArray), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(dateArray), is(48L)); + } + + +} From 4392de9be272c045a9854340f887e420f25f0f4a Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 4 May 2017 09:21:23 -0700 Subject: [PATCH 06/23] Initial implementation of SpannerIO.Write This closes #2166. --- .../examples/spanner/SpannerCSVLoader.java | 143 +++++++++ pom.xml | 10 +- sdks/java/core/pom.xml | 4 +- sdks/java/io/google-cloud-platform/pom.xml | 33 ++- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 272 ++++++++++++++++++ .../beam/sdk/io/gcp/spanner/package-info.java | 23 ++ .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 2 + 7 files changed, 474 insertions(+), 13 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java new file mode 100644 index 000000000000..eee581da2148 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java @@ -0,0 +1,143 @@ +/* + * 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.examples.spanner; + +import com.google.cloud.spanner.Database; +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerException; +import com.google.cloud.spanner.SpannerOptions; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + + + +/** + * Generalized bulk loader for importing CSV files into Spanner. + * + */ +public class SpannerCSVLoader { + + /** + * Command options specification. + */ + private interface Options extends PipelineOptions { + @Description("Create a sample database") + @Default.Boolean(false) + boolean isCreateDatabase(); + void setCreateDatabase(boolean createDatabase); + + @Description("File to read from ") + @Validation.Required + String getInput(); + void setInput(String value); + + @Description("Instance ID to write to in Spanner") + @Validation.Required + String getInstanceId(); + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Validation.Required + String getDatabaseId(); + void setDatabaseId(String value); + + @Description("Table name") + @Validation.Required + String getTable(); + void setTable(String value); + } + + + /** + * Constructs and executes the processing pipeline based upon command options. + */ + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline p = Pipeline.create(options); + PCollection lines = p.apply(TextIO.Read.from(options.getInput())); + PCollection mutations = lines + .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); + mutations + .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); + p.run().waitUntilFinish(); + } + + public static void createDatabase(Options options) { + Spanner client = SpannerOptions.getDefaultInstance().getService(); + + DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); + try { + databaseAdminClient.dropDatabase(options.getInstanceId(), options + .getDatabaseId()); + } catch (SpannerException e) { + // Does not exist, ignore. + } + Operation op = databaseAdminClient.createDatabase( + options.getInstanceId(), options + .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" + + " Key INT64," + + " Name STRING," + + " Email STRING," + + " Age INT," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + + /** + * A DoFn that creates a Spanner Mutation for each CSV line. + */ + static class NaiveParseCsvFn extends DoFn { + private final String table; + + NaiveParseCsvFn(String table) { + this.table = table; + } + + @ProcessElement + public void processElement(ProcessContext c) { + String line = c.element(); + String[] elements = line.split(","); + if (elements.length != 4) { + return; + } + Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) + .set("Key").to(Long.valueOf(elements[0])) + .set("Name").to(elements[1]) + .set("Email").to(elements[2]) + .set("Age").to(Integer.valueOf(elements[3])) + .build(); + c.output(mutation); + } + } +} diff --git a/pom.xml b/pom.xml index a978f58fe3b1..c957778e87e2 100644 --- a/pom.xml +++ b/pom.xml @@ -141,13 +141,15 @@ 4.4.1 4.3.5.RELEASE 2.0 + 1.0.0-rc2 2.20 2.20 3.0.2 - + -Werror -Xpkginfo:always nothing + 0.16.0-beta pom @@ -860,6 +862,12 @@ ${google-cloud-bigdataoss.version} + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigdataoss util diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 882657b1db4e..9ad4d9966c52 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -284,8 +284,8 @@ joda-time - + org.tukaani xz diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 7594365bc11c..c5af213273f5 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -84,11 +84,28 @@ jackson-databind + + io.grpc + grpc-core + + + + com.google.api.grpc + grpc-google-common-protos + ${grpc-google-common-protos.version} + + com.google.apis google-api-services-bigquery + + com.google.api + api-common + ${api-common.version} + + com.google.apis google-api-services-pubsub @@ -119,11 +136,6 @@ grpc-auth - - io.grpc - grpc-core - - io.grpc grpc-netty @@ -152,6 +164,12 @@ joda-time + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigtable bigtable-protos @@ -187,11 +205,6 @@ google-auth-library-oauth2-http - - com.google.api.grpc - grpc-google-common-protos - - org.slf4j slf4j-api diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java new file mode 100644 index 000000000000..172ed8f02687 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -0,0 +1,272 @@ +/* + * 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 com.google.common.base.Preconditions.checkNotNull; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; + +import com.google.cloud.spanner.AbortedException; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + + +/** + * Google Cloud Spanner connectors. + * + *

Reading from Cloud Spanner

+ * Status: Not implemented. + * + *

Writing to Cloud Spanner

+ * Status: Experimental. + * + *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. + * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to + * {@link PCollection} and specify instance and database identifiers. + * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud + * Spanner. + * + *

{@code
+ *
+ * Pipeline p = ...;
+ * // Read the CSV file.
+ * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput()));
+ * // Parse the line and convert to mutation.
+ * PCollection mutations = lines.apply("Parse CSV", parseFromCsv());
+ * // Write mutations.
+ * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
+ * p.run();
+ *
+ * }
+ + */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class SpannerIO { + + private SpannerIO() { + } + + @VisibleForTesting + static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + + /** + * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch + * size. + */ + public static Writer writeTo(String instanceId, String databaseId) { + return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + } + + /** + * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * + * @see SpannerIO + */ + public static class Writer extends PTransform, PDone> { + + private final String instanceId; + private final String databaseId; + private int batchSize; + + Writer(String instanceId, String databaseId, int batchSize) { + this.instanceId = instanceId; + this.databaseId = databaseId; + this.batchSize = batchSize; + } + + /** + * Returns a new {@link Writer} with a limit on the number of mutations per batch. + * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + */ + public Writer withBatchSize(Integer batchSize) { + return new Writer(instanceId, databaseId, batchSize); + } + + @Override + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", ParDo.of( + new SpannerWriterFn(instanceId, databaseId, batchSize))); + + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + checkNotNull(instanceId, "instanceId"); + checkNotNull(databaseId, "databaseId"); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("instanceId", instanceId) + .add("databaseId", databaseId) + .toString(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Output Database")); + } + + } + + + /** + * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * + *

See + * + *

Commits are non-transactional. If a commit fails, it will be retried (up to + * {@link SpannerIO#MAX_RETRIES}. times). This means that the + * mutation operation should be idempotent. + */ + @VisibleForTesting + static class SpannerWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private transient Spanner spanner; + private final String instanceId; + private final String databaseId; + private final int batchSize; + private transient DatabaseClient dbClient; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); + + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + + @VisibleForTesting + SpannerWriterFn(String instanceId, String databaseId, int batchSize) { + this.instanceId = checkNotNull(instanceId, "instanceId"); + this.databaseId = checkNotNull(databaseId, "databaseId"); + this.batchSize = batchSize; + } + + @Setup + public void setup() throws Exception { + SpannerOptions options = SpannerOptions.newBuilder().build(); + spanner = options.getService(); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + Mutation m = c.element(); + mutations.add(m); + int columnCount = m.asMap().size(); + if ((mutations.size() + 1) * columnCount >= batchSize) { + flushBatch(); + } + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (!mutations.isEmpty()) { + flushBatch(); + } + } + + @Teardown + public void teardown() throws Exception { + if (spanner == null) { + return; + } + spanner.closeAsync().get(); + } + + /** + * Writes a batch of mutations to Cloud Spanner. + * + *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. + * If the retry limit is exceeded, the last exception from Cloud Spanner will be + * thrown. + * + * @throws AbortedException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws AbortedException, IOException, InterruptedException { + LOG.debug("Writing batch of {} mutations", mutations.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); + + while (true) { + // Batch upsert rows. + try { + dbClient.writeAtLeastOnce(mutations); + + // Break if the commit threw no exception. + break; + } catch (AbortedException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); + } + + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Database")); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java new file mode 100644 index 000000000000..19e468cce041 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 an API for reading from and writing to + * Google Cloud Spanner. + */ +package org.apache.beam.sdk.io.gcp.spanner; 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 7025004cc953..8950452b5db2 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 @@ -75,6 +75,8 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.beam"), classesInPackage("org.apache.commons.logging"), + classesInPackage("com.google.cloud"), + classesInPackage("com.google.cloud.spanner"), // via Bigtable classesInPackage("org.joda.time")); From b092be8df5cd96012c4d67b3defbd3554707edfe Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:42:07 -0700 Subject: [PATCH 07/23] Minor style, compilation, javadoc fixups --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 45 +++++++++---------- 1 file changed, 20 insertions(+), 25 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 172ed8f02687..c9c81a51a384 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,10 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; - import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -35,23 +31,25 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - - - /** - * Google Cloud Spanner connectors. + * {@link PTransform Transforms} for reading from and writing to + * Google Cloud Spanner. * *

Reading from Cloud Spanner

* Status: Not implemented. @@ -61,9 +59,8 @@ * *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. - * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud - * Spanner. + * {@link PCollection} and specify instance and database identifiers. For example, following code + * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. * *

{@code
  *
@@ -77,14 +74,10 @@
  * p.run();
  *
  * }
- */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - private SpannerIO() { - } - @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; @@ -97,10 +90,11 @@ public static Writer writeTo(String instanceId, String databaseId) { } /** - * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * A {@link PTransform} that writes {@link Mutation} objects to Google Cloud Spanner. * * @see SpannerIO */ + @Experimental(Experimental.Kind.SOURCE_SINK) public static class Writer extends PTransform, PDone> { private final String instanceId; @@ -130,7 +124,7 @@ public PDone expand(PCollection input) { } @Override - public void validate(PCollection input) { + public void validate(PipelineOptions options) { checkNotNull(instanceId, "instanceId"); checkNotNull(databaseId, "databaseId"); } @@ -152,19 +146,17 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Output Database")); } - } - /** - * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. * - *

See - * *

Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerIO#MAX_RETRIES}. times). This means that the - * mutation operation should be idempotent. + * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be + * idempotent. + * + *

See Google Cloud Spanner documentation. */ @VisibleForTesting static class SpannerWriterFn extends DoFn { @@ -208,7 +200,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { if (!mutations.isEmpty()) { flushBatch(); } @@ -269,4 +261,7 @@ public void populateDisplayData(Builder builder) { .withLabel("Database")); } } + + private SpannerIO() {} // Prevent construction. + } From e16339a3e699d7bea30f3f3a703b4c7c3ead0f59 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:54 -0700 Subject: [PATCH 08/23] Fix spanner dependency management Also minor cleanup alphabetization in root pom.xml --- pom.xml | 15 +++++++++++---- sdks/java/io/google-cloud-platform/pom.xml | 3 --- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index c957778e87e2..52b7656a2acc 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,7 @@ 0.5.160222 1.4.0 1.3.0 + 1.0.0-rc2 1.0-rc2 1.4.1 0.6.1 @@ -136,15 +137,15 @@ v1-rev10-1.22.0 1.7.14 1.6.2 + 4.3.5.RELEASE 3.1.4 v1-rev71-1.22.0 4.4.1 - 4.3.5.RELEASE - 2.0 - 1.0.0-rc2 - 2.20 + 2.20 3.0.2 + 2.0 + 2.20 -Werror -Xpkginfo:always @@ -605,6 +606,12 @@ ${grpc.version} + + com.google.api + api-common + ${google-api-common.version} + + com.google.api-client google-api-client diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index c5af213273f5..7938694c85a7 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -92,7 +92,6 @@ com.google.api.grpc grpc-google-common-protos - ${grpc-google-common-protos.version} @@ -103,7 +102,6 @@ com.google.api api-common - ${api-common.version} @@ -167,7 +165,6 @@ com.google.cloud google-cloud-spanner - ${spanner.version} From 9798534793286aeb3deacdc0e30f63032b6077fe Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:24 -0700 Subject: [PATCH 09/23] Delete SpannerCSVLoader This is not appropriate for examples. SpannerIO should be well-javadoced and integration tested. --- .../examples/spanner/SpannerCSVLoader.java | 143 ------------------ 1 file changed, 143 deletions(-) delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java deleted file mode 100644 index eee581da2148..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java +++ /dev/null @@ -1,143 +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.examples.spanner; - -import com.google.cloud.spanner.Database; -import com.google.cloud.spanner.DatabaseAdminClient; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Operation; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerException; -import com.google.cloud.spanner.SpannerOptions; -import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; -import java.util.Collections; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; - - - -/** - * Generalized bulk loader for importing CSV files into Spanner. - * - */ -public class SpannerCSVLoader { - - /** - * Command options specification. - */ - private interface Options extends PipelineOptions { - @Description("Create a sample database") - @Default.Boolean(false) - boolean isCreateDatabase(); - void setCreateDatabase(boolean createDatabase); - - @Description("File to read from ") - @Validation.Required - String getInput(); - void setInput(String value); - - @Description("Instance ID to write to in Spanner") - @Validation.Required - String getInstanceId(); - void setInstanceId(String value); - - @Description("Database ID to write to in Spanner") - @Validation.Required - String getDatabaseId(); - void setDatabaseId(String value); - - @Description("Table name") - @Validation.Required - String getTable(); - void setTable(String value); - } - - - /** - * Constructs and executes the processing pipeline based upon command options. - */ - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - Pipeline p = Pipeline.create(options); - PCollection lines = p.apply(TextIO.Read.from(options.getInput())); - PCollection mutations = lines - .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); - mutations - .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); - p.run().waitUntilFinish(); - } - - public static void createDatabase(Options options) { - Spanner client = SpannerOptions.getDefaultInstance().getService(); - - DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); - try { - databaseAdminClient.dropDatabase(options.getInstanceId(), options - .getDatabaseId()); - } catch (SpannerException e) { - // Does not exist, ignore. - } - Operation op = databaseAdminClient.createDatabase( - options.getInstanceId(), options - .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" - + " Key INT64," - + " Name STRING," - + " Email STRING," - + " Age INT," - + ") PRIMARY KEY (Key)")); - op.waitFor(); - } - - - /** - * A DoFn that creates a Spanner Mutation for each CSV line. - */ - static class NaiveParseCsvFn extends DoFn { - private final String table; - - NaiveParseCsvFn(String table) { - this.table = table; - } - - @ProcessElement - public void processElement(ProcessContext c) { - String line = c.element(); - String[] elements = line.split(","); - if (elements.length != 4) { - return; - } - Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) - .set("Key").to(Long.valueOf(elements[0])) - .set("Name").to(elements[1]) - .set("Email").to(elements[2]) - .set("Age").to(Integer.valueOf(elements[3])) - .build(); - c.output(mutation); - } - } -} From f495a414d6d6ec1a3098fb751f499680d070f7c5 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 15 May 2017 10:16:18 -0700 Subject: [PATCH 10/23] Refine Spanner API tests And remove outdated Bigtable comment --- .../org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 8950452b5db2..91caded1ad35 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 @@ -63,7 +63,10 @@ public void testGcpApiSurface() throws Exception { Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableClusterName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class), - // via Bigtable, PR above out to fix. + Matchers.>equalTo(com.google.cloud.ByteArray.class), + Matchers.>equalTo(com.google.cloud.Date.class), + Matchers.>equalTo(com.google.cloud.Timestamp.class), + classesInPackage("com.google.cloud.spanner"), classesInPackage("com.google.datastore.v1"), classesInPackage("com.google.protobuf"), classesInPackage("com.google.type"), @@ -75,9 +78,6 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.beam"), classesInPackage("org.apache.commons.logging"), - classesInPackage("com.google.cloud"), - classesInPackage("com.google.cloud.spanner"), - // via Bigtable classesInPackage("org.joda.time")); assertThat(apiSurface, containsOnlyClassesMatching(allowedClasses)); From ce5d04cc354c3d1bd3938eff94aa3474adac1e1b Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 15 May 2017 10:42:57 -0700 Subject: [PATCH 11/23] SpannerIO.Write cleanup and style fixes * Rename to Write to match the rest of the SDK. * Convert to AutoValue, delete toString. * Drop .writeTo(), instead use .write() as default constructor. * Temporarily drop withBatchSize, as its existence is not clearly justified. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 114 +++++++++--------- 1 file changed, 60 insertions(+), 54 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index c9c81a51a384..cc922ec9d0e5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.value.AutoValue; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -26,12 +27,11 @@ import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -48,31 +48,27 @@ import org.slf4j.LoggerFactory; /** - * {@link PTransform Transforms} for reading from and writing to + * Experimental {@link PTransform Transforms} for reading from and writing to * Google Cloud Spanner. * *

Reading from Cloud Spanner

- * Status: Not implemented. + *

This functionality is not yet implemented. * *

Writing to Cloud Spanner

- * Status: Experimental. + *

The Cloud Spanner {@link SpannerIO.Write} transform writes to Cloud Spanner by executing a + * collection of input row {@link Mutation Mutations}. The mutations grouped into batches for + * efficiency. * - *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. - * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. For example, following code - * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. + *

To configure the write transform, create an instance using {@link #write()} and then specify + * the destination Cloud Spanner instance ({@link Write#withInstanceId(String)} and destination + * database ({@link Write#withDatabaseId(String)}). For example: * *

{@code
- *
- * Pipeline p = ...;
- * // Read the CSV file.
- * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput()));
- * // Parse the line and convert to mutation.
- * PCollection mutations = lines.apply("Parse CSV", parseFromCsv());
+ * // Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
+ * PCollection mutations = ...;
  * // Write mutations.
- * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
- * p.run();
- *
+ * mutations.apply(
+ *     "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
  * }
*/ @Experimental(Experimental.Kind.SOURCE_SINK) @@ -81,12 +77,14 @@ public class SpannerIO { @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; - /** - * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch - * size. - */ - public static Writer writeTo(String instanceId, String databaseId) { - return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + /** + * Creates an unitialized instance of {@link Write}. Before use, the {@link Write} must be + * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify + * the Cloud Spanner database being written. + */ + @Experimental + public static Write write() { + return new AutoValue_SpannerIO_Write.Builder().build(); } /** @@ -95,55 +93,63 @@ public static Writer writeTo(String instanceId, String databaseId) { * @see SpannerIO */ @Experimental(Experimental.Kind.SOURCE_SINK) - public static class Writer extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { - private final String instanceId; - private final String databaseId; - private int batchSize; + @Nullable + abstract String getInstanceId(); - Writer(String instanceId, String databaseId, int batchSize) { - this.instanceId = instanceId; - this.databaseId = databaseId; - this.batchSize = batchSize; + @Nullable + abstract String getDatabaseId(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setInstanceId(String instanceId); + + abstract Builder setDatabaseId(String databaseId); + + abstract Write build(); } /** - * Returns a new {@link Writer} with a limit on the number of mutations per batch. - * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * instance. + * + *

Does not modify this object. */ - public Writer withBatchSize(Integer batchSize) { - return new Writer(instanceId, databaseId, batchSize); + public Write withInstanceId(String instanceId) { + return toBuilder().setInstanceId(instanceId).build(); } - @Override - public PDone expand(PCollection input) { - input.apply("Write mutations to Spanner", ParDo.of( - new SpannerWriterFn(instanceId, databaseId, batchSize))); - - return PDone.in(input.getPipeline()); + /** + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * database. + * + *

Does not modify this object. + */ + public Write withDatabaseId(String databaseId) { + return toBuilder().setDatabaseId(databaseId).build(); } @Override - public void validate(PipelineOptions options) { - checkNotNull(instanceId, "instanceId"); - checkNotNull(databaseId, "databaseId"); - } + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", + ParDo.of(new SpannerWriterFn( + getInstanceId(), getDatabaseId(), SPANNER_MUTATIONS_PER_COMMIT_LIMIT))); - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("instanceId", instanceId) - .add("databaseId", databaseId) - .toString(); + return PDone.in(input.getPipeline()); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", instanceId) + .addIfNotNull(DisplayData.item("instanceId", getInstanceId()) .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) + .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()) .withLabel("Output Database")); } } From 16e4b832628205b1f560e5f37fb105d17f2b56ec Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 11:38:44 -0700 Subject: [PATCH 12/23] Use a new batch size limit. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 66 +++++++++++-------- 1 file changed, 39 insertions(+), 27 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index cc922ec9d0e5..1dbea6cc368a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.spanner; -import static com.google.common.base.Preconditions.checkNotNull; - import com.google.auto.value.AutoValue; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; @@ -74,17 +72,16 @@ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - @VisibleForTesting - static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + private static final long DEFAULT_BATCH_SIZE = 1024 * 1024; // 1 MB /** - * Creates an unitialized instance of {@link Write}. Before use, the {@link Write} must be + * Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify * the Cloud Spanner database being written. */ @Experimental public static Write write() { - return new AutoValue_SpannerIO_Write.Builder().build(); + return new AutoValue_SpannerIO_Write.Builder().setBatchSize(DEFAULT_BATCH_SIZE).build(); } /** @@ -104,6 +101,8 @@ public abstract static class Write extends PTransform, PDo abstract Builder toBuilder(); + public abstract long batchSize(); + @AutoValue.Builder abstract static class Builder { @@ -111,9 +110,17 @@ abstract static class Builder { abstract Builder setDatabaseId(String databaseId); + abstract Builder setBatchSize(long batchSize); + abstract Write build(); } + // TODO(mairbek): Once SpannerOptions is serializable, make it Write parameter. + public SpannerOptions spannerOptions() { + SpannerOptions.Builder builder = SpannerOptions.newBuilder(); + return builder.build(); + } + /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner * instance. @@ -124,6 +131,15 @@ public Write withInstanceId(String instanceId) { return toBuilder().setInstanceId(instanceId).build(); } + /** + * Returns a new {@link SpannerIO.Write} with a new batch size limit. + * + *

Does not modify this object. + */ + public Write withBatchSize(long batchSize) { + return toBuilder().setBatchSize(batchSize).build(); + } + /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner * database. @@ -136,10 +152,8 @@ public Write withDatabaseId(String databaseId) { @Override public PDone expand(PCollection input) { - input.apply("Write mutations to Spanner", - ParDo.of(new SpannerWriterFn( - getInstanceId(), getDatabaseId(), SPANNER_MUTATIONS_PER_COMMIT_LIMIT))); - + input.apply("Write mutations to Cloud Spanner", ParDo.of( + new SpannerWriterFn(this))); return PDone.in(input.getPipeline()); } @@ -156,7 +170,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in - * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * batches, where the maximum batch size is {@link SpannerIO#DEFAULT_BATCH_SIZE}. * *

Commits are non-transactional. If a commit fails, it will be retried (up to * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be @@ -167,13 +181,12 @@ public void populateDisplayData(DisplayData.Builder builder) { @VisibleForTesting static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private final Write spec; private transient Spanner spanner; - private final String instanceId; - private final String databaseId; - private final int batchSize; private transient DatabaseClient dbClient; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); + private long batchSize = 0; private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -181,26 +194,25 @@ static class SpannerWriterFn extends DoFn { .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(String instanceId, String databaseId, int batchSize) { - this.instanceId = checkNotNull(instanceId, "instanceId"); - this.databaseId = checkNotNull(databaseId, "databaseId"); - this.batchSize = batchSize; + SpannerWriterFn(Write spec) { + this.spec = spec; } @Setup public void setup() throws Exception { - SpannerOptions options = SpannerOptions.newBuilder().build(); - spanner = options.getService(); - dbClient = spanner.getDatabaseClient( - DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + spanner = spec.spannerOptions().getService(); + String projectId = spec.spannerOptions().getProjectId(); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(projectId, spec.getInstanceId(), spec.getDatabaseId())); + mutations.clear(); + batchSize = 0; } @ProcessElement public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); mutations.add(m); - int columnCount = m.asMap().size(); - if ((mutations.size() + 1) * columnCount >= batchSize) { + if (batchSize >= spec.batchSize()) { flushBatch(); } } @@ -215,7 +227,7 @@ public void finishBundle() throws Exception { @Teardown public void teardown() throws Exception { if (spanner == null) { - return; + return; } spanner.closeAsync().get(); } @@ -261,9 +273,9 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", instanceId) + .addIfNotNull(DisplayData.item("instanceId", spec.getInstanceId()) .withLabel("Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) + .addIfNotNull(DisplayData.item("databaseId", spec.getDatabaseId()) .withLabel("Database")); } } From 5a2214a2623fa6d8d4a7b50852b59dc7789837af Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 15 May 2017 10:42:57 -0700 Subject: [PATCH 13/23] SpannerIO.Write cleanup and style fixes * Rename to Write to match the rest of the SDK. * Convert to AutoValue, delete toString. * Drop .writeTo(), instead use .write() as default constructor. * Temporarily drop withBatchSize, as its existence is not clearly justified. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 116 ++++++++++-------- 1 file changed, 62 insertions(+), 54 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index c9c81a51a384..ec119311c106 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.value.AutoValue; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -26,12 +27,11 @@ import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -48,31 +48,29 @@ import org.slf4j.LoggerFactory; /** - * {@link PTransform Transforms} for reading from and writing to + * Experimental {@link PTransform Transforms} for reading from and writing to * Google Cloud Spanner. * *

Reading from Cloud Spanner

- * Status: Not implemented. + * + *

This functionality is not yet implemented. * *

Writing to Cloud Spanner

- * Status: Experimental. * - *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. - * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. For example, following code - * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. + *

The Cloud Spanner {@link SpannerIO.Write} transform writes to Cloud Spanner by executing a + * collection of input row {@link Mutation Mutations}. The mutations grouped into batches for + * efficiency. * - *

{@code
+ * 

To configure the write transform, create an instance using {@link #write()} and then specify + * the destination Cloud Spanner instance ({@link Write#withInstanceId(String)} and destination + * database ({@link Write#withDatabaseId(String)}). For example: * - * Pipeline p = ...; - * // Read the CSV file. - * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput())); - * // Parse the line and convert to mutation. - * PCollection mutations = lines.apply("Parse CSV", parseFromCsv()); + *

{@code
+ * // Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
+ * PCollection mutations = ...;
  * // Write mutations.
- * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
- * p.run();
- *
+ * mutations.apply(
+ *     "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
  * }
*/ @Experimental(Experimental.Kind.SOURCE_SINK) @@ -81,12 +79,14 @@ public class SpannerIO { @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; - /** - * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch - * size. - */ - public static Writer writeTo(String instanceId, String databaseId) { - return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + /** + * Creates an unitialized instance of {@link Write}. Before use, the {@link Write} must be + * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify + * the Cloud Spanner database being written. + */ + @Experimental + public static Write write() { + return new AutoValue_SpannerIO_Write.Builder().build(); } /** @@ -95,55 +95,63 @@ public static Writer writeTo(String instanceId, String databaseId) { * @see SpannerIO */ @Experimental(Experimental.Kind.SOURCE_SINK) - public static class Writer extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { - private final String instanceId; - private final String databaseId; - private int batchSize; + @Nullable + abstract String getInstanceId(); - Writer(String instanceId, String databaseId, int batchSize) { - this.instanceId = instanceId; - this.databaseId = databaseId; - this.batchSize = batchSize; + @Nullable + abstract String getDatabaseId(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setInstanceId(String instanceId); + + abstract Builder setDatabaseId(String databaseId); + + abstract Write build(); } /** - * Returns a new {@link Writer} with a limit on the number of mutations per batch. - * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * instance. + * + *

Does not modify this object. */ - public Writer withBatchSize(Integer batchSize) { - return new Writer(instanceId, databaseId, batchSize); + public Write withInstanceId(String instanceId) { + return toBuilder().setInstanceId(instanceId).build(); } - @Override - public PDone expand(PCollection input) { - input.apply("Write mutations to Spanner", ParDo.of( - new SpannerWriterFn(instanceId, databaseId, batchSize))); - - return PDone.in(input.getPipeline()); + /** + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * database. + * + *

Does not modify this object. + */ + public Write withDatabaseId(String databaseId) { + return toBuilder().setDatabaseId(databaseId).build(); } @Override - public void validate(PipelineOptions options) { - checkNotNull(instanceId, "instanceId"); - checkNotNull(databaseId, "databaseId"); - } + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", + ParDo.of(new SpannerWriterFn( + getInstanceId(), getDatabaseId(), SPANNER_MUTATIONS_PER_COMMIT_LIMIT))); - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("instanceId", instanceId) - .add("databaseId", databaseId) - .toString(); + return PDone.in(input.getPipeline()); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", instanceId) + .addIfNotNull(DisplayData.item("instanceId", getInstanceId()) .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) + .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()) .withLabel("Output Database")); } } From 5fc07fe297f6cf5fdd00bb3385d6a23baaf67104 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 14:59:22 -0700 Subject: [PATCH 14/23] Added a SpannerIO unit test. --- .../examples/spanner/SpannerCSVLoader.java | 143 ---------------- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 70 ++++++-- .../sdk/io/gcp/spanner/SpannerIOTest.java | 154 ++++++++++++++++++ 3 files changed, 213 insertions(+), 154 deletions(-) delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java deleted file mode 100644 index 9aa8672b87dd..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java +++ /dev/null @@ -1,143 +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.examples.spanner; - -import com.google.cloud.spanner.Database; -import com.google.cloud.spanner.DatabaseAdminClient; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Operation; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerException; -import com.google.cloud.spanner.SpannerOptions; -import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; -import java.util.Collections; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; - - - -/** - * Generalized bulk loader for importing CSV files into Spanner. - * - */ -public class SpannerCSVLoader { - - /** - * Command options specification. - */ - private interface Options extends PipelineOptions { - @Description("Create a sample database") - @Default.Boolean(false) - boolean isCreateDatabase(); - void setCreateDatabase(boolean createDatabase); - - @Description("File to read from ") - @Validation.Required - String getInput(); - void setInput(String value); - - @Description("Instance ID to write to in Spanner") - @Validation.Required - String getInstanceId(); - void setInstanceId(String value); - - @Description("Database ID to write to in Spanner") - @Validation.Required - String getDatabaseId(); - void setDatabaseId(String value); - - @Description("Table name") - @Validation.Required - String getTable(); - void setTable(String value); - } - - - /** - * Constructs and executes the processing pipeline based upon command options. - */ - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - Pipeline p = Pipeline.create(options); - PCollection lines = p.apply(TextIO.read().from(options.getInput())); - PCollection mutations = lines - .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); - mutations - .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); - p.run().waitUntilFinish(); - } - - public static void createDatabase(Options options) { - Spanner client = SpannerOptions.getDefaultInstance().getService(); - - DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); - try { - databaseAdminClient.dropDatabase(options.getInstanceId(), options - .getDatabaseId()); - } catch (SpannerException e) { - // Does not exist, ignore. - } - Operation op = databaseAdminClient.createDatabase( - options.getInstanceId(), options - .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" - + " Key INT64," - + " Name STRING," - + " Email STRING," - + " Age INT," - + ") PRIMARY KEY (Key)")); - op.waitFor(); - } - - - /** - * A DoFn that creates a Spanner Mutation for each CSV line. - */ - static class NaiveParseCsvFn extends DoFn { - private final String table; - - NaiveParseCsvFn(String table) { - this.table = table; - } - - @ProcessElement - public void processElement(ProcessContext c) { - String line = c.element(); - String[] elements = line.split(","); - if (elements.length != 4) { - return; - } - Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) - .set("Key").to(Long.valueOf(elements[0])) - .set("Name").to(elements[1]) - .set("Email").to(elements[2]) - .set("Age").to(Integer.valueOf(elements[3])) - .build(); - c.output(mutation); - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 1dbea6cc368a..aa4ef582039a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -18,10 +18,12 @@ package org.apache.beam.sdk.io.gcp.spanner; import com.google.auto.value.AutoValue; +import com.google.cloud.ServiceOptions; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; import com.google.cloud.spanner.Mutation; +import com.google.cloud.ServiceFactory; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; @@ -30,6 +32,7 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -45,6 +48,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static com.google.common.base.Preconditions.checkNotNull; + /** * Experimental {@link PTransform Transforms} for reading from and writing to * Google Cloud Spanner. @@ -93,34 +98,57 @@ public static Write write() { @AutoValue public abstract static class Write extends PTransform, PDone> { + @Nullable + abstract String getProjectId(); + @Nullable abstract String getInstanceId(); @Nullable abstract String getDatabaseId(); - abstract Builder toBuilder(); + abstract long getBatchSize(); - public abstract long batchSize(); + @Nullable @VisibleForTesting + abstract ServiceFactory getServiceFactory(); + + abstract Builder toBuilder(); @AutoValue.Builder abstract static class Builder { + abstract Builder setProjectId(String projectId); + abstract Builder setInstanceId(String instanceId); abstract Builder setDatabaseId(String databaseId); abstract Builder setBatchSize(long batchSize); + @VisibleForTesting + abstract Builder setServiceFactory(ServiceFactory serviceFactory); + abstract Write build(); } - // TODO(mairbek): Once SpannerOptions is serializable, make it Write parameter. - public SpannerOptions spannerOptions() { + public SpannerOptions getSpannerOptions() { SpannerOptions.Builder builder = SpannerOptions.newBuilder(); + if (getServiceFactory() != null) { + builder.setServiceFactory(getServiceFactory()); + } return builder.build(); } + /** + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * project. + * + *

Does not modify this object. + */ + public Write withProjectId(String projectId) { + return toBuilder().setProjectId(projectId).build(); + } + /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner * instance. @@ -150,6 +178,20 @@ public Write withDatabaseId(String databaseId) { return toBuilder().setDatabaseId(databaseId).build(); } + @VisibleForTesting + Write withServiceFactory(ServiceFactory serviceFactory) { + return toBuilder().setServiceFactory(serviceFactory).build(); + } + + @Override + public void validate(PipelineOptions options) { + checkNotNull(getInstanceId(), "SpannerIO.write() requires instance id to be set with " + + "withInstanceId method"); + checkNotNull(getDatabaseId(), "SpannerIO.write() requires database id to be set with " + + "withDatabaseId method"); + } + + @Override public PDone expand(PCollection input) { input.apply("Write mutations to Cloud Spanner", ParDo.of( @@ -185,7 +227,7 @@ static class SpannerWriterFn extends DoFn { private transient Spanner spanner; private transient DatabaseClient dbClient; // Current batch of mutations to be written. - private final List mutations = new ArrayList<>(); + private List mutations; private long batchSize = 0; private static final int MAX_RETRIES = 5; @@ -200,11 +242,10 @@ static class SpannerWriterFn extends DoFn { @Setup public void setup() throws Exception { - spanner = spec.spannerOptions().getService(); - String projectId = spec.spannerOptions().getProjectId(); + spanner = spec.getSpannerOptions().getService(); dbClient = spanner.getDatabaseClient( - DatabaseId.of(projectId, spec.getInstanceId(), spec.getDatabaseId())); - mutations.clear(); + DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); + mutations = new ArrayList<>(); batchSize = 0; } @@ -212,11 +253,17 @@ public void setup() throws Exception { public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); mutations.add(m); - if (batchSize >= spec.batchSize()) { + batchSize += MutationSizeEstimator.sizeOf(m); + if (batchSize >= spec.getBatchSize()) { flushBatch(); } } + private String projectId() { + return spec.getProjectId() == null ? + ServiceOptions.getDefaultProjectId() : spec.getProjectId(); + } + @FinishBundle public void finishBundle() throws Exception { if (!mutations.isEmpty()) { @@ -266,7 +313,8 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } } LOG.debug("Successfully wrote {} mutations", mutations.size()); - mutations.clear(); + mutations = new ArrayList<>(); + batchSize = 0; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java new file mode 100644 index 000000000000..10b32ec70386 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -0,0 +1,154 @@ +package org.apache.beam.sdk.io.gcp.spanner; + +import avro.shaded.com.google.common.collect.Iterables; +import com.google.api.core.ApiFuture; +import com.google.cloud.ServiceFactory; +import com.google.cloud.spanner.*; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; + +import java.io.Serializable; +import java.util.Arrays; + +import static org.mockito.Mockito.*; + +@RunWith(JUnit4.class) +public class SpannerIOTest implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule + public transient ExpectedException thrown = ExpectedException.none(); + // Tests that use serviceFactory must be serializable, to avoid parallel execution. + private ServiceFactory serviceFactory; + // Marked as static so they could be returned by serviceFactory, which is serializable. + private static Spanner mockSpanner; + private static DatabaseClient mockDatabaseClient; + + @Before + public void setUp() throws Exception { + ApiFuture voidFuture = mock(ApiFuture.class, withSettings().serializable()); + mockSpanner = mock(Spanner.class, withSettings().serializable()); + mockDatabaseClient = mock(DatabaseClient.class, withSettings().serializable()); + serviceFactory = mock(ServiceFactory.class, withSettings().serializable()); + when(mockSpanner.getDatabaseClient(Matchers.any(DatabaseId.class))).thenReturn(mockDatabaseClient); + when(mockSpanner.closeAsync()).thenReturn(voidFuture); + serviceFactory = new FakeServiceFactory(); + } + + @Test + public void emptyTransform() throws Exception { + SpannerIO.Write write = SpannerIO.write(); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyInstanceId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withDatabaseId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyDatabaseId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withInstanceId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires database id to be set with"); + write.validate(null); + } + + @Test + @Category(NeedsRunner.class) + public synchronized void singleMutationPipeline() throws Exception { + Mutation mutation = Mutation + .newInsertOrUpdateBuilder("test").set("one").to(2).build(); + PCollection mutations = pipeline.apply(Create.of(mutation)); + + SpannerIO.Write write = SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance").withDatabaseId + ("test-database").withServiceFactory(serviceFactory); + mutations.apply(write); + pipeline.run(); + verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", + "test-instance", "test-database")); + verify(mockDatabaseClient, times(1)).writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + @Test + public synchronized void batching() throws Exception { + Mutation one = Mutation + .newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation + .newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSize(1000000000) + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriterFn writerFn = new SpannerIO.SpannerWriterFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", + "test-instance", "test-database")); + verify(mockDatabaseClient, times(1)).writeAtLeastOnce(argThat(new IterableOfSize(2))); + } + + @Test + public synchronized void noBatching() throws Exception { + Mutation one = Mutation + .newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation + .newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSize(0) // turn off batching. + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriterFn writerFn = new SpannerIO.SpannerWriterFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", + "test-instance", "test-database")); + verify(mockDatabaseClient, times(2)).writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + private static class FakeServiceFactory implements ServiceFactory, + Serializable { + @Override + public Spanner create(SpannerOptions serviceOptions) { + return mockSpanner; + } + } + + private static class IterableOfSize extends ArgumentMatcher> { + private final int size; + + private IterableOfSize(int size) { + this.size = size; + } + + @Override + public boolean matches(Object argument) { + return argument instanceof Iterable && Iterables.size((Iterable) argument) == size; + } + } +} From 0b7b84dfbe17cbc0747197f34c8512610d01d86e Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 16:25:33 -0700 Subject: [PATCH 15/23] Renaming and documenting --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 31 ++++++++++--------- .../sdk/io/gcp/spanner/SpannerIOTest.java | 4 +-- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index aa4ef582039a..1e0707407980 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -73,6 +73,17 @@ * mutations.apply( * "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database")); * }

+ * + * The default size of the batch is set to 1MB, to override this use + * {@link Write#withBatchSize(long)}. Setting batch size to a small value or zero + * practically disables batching. + * + * The transform does not provide same transactional guarantees as Cloud Spanner. In particular, + *
  • Mutations are not submitted atomically; + *
  • A mutation is applied at least once; + *
  • If the pipeline was unexpectedly stopped, mutations that were already applied will not get + * rolled back. + * */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { @@ -131,7 +142,7 @@ abstract static class Builder { abstract Write build(); } - public SpannerOptions getSpannerOptions() { + SpannerOptions getSpannerOptions() { SpannerOptions.Builder builder = SpannerOptions.newBuilder(); if (getServiceFactory() != null) { builder.setServiceFactory(getServiceFactory()); @@ -191,11 +202,10 @@ public void validate(PipelineOptions options) { + "withDatabaseId method"); } - @Override public PDone expand(PCollection input) { input.apply("Write mutations to Cloud Spanner", ParDo.of( - new SpannerWriterFn(this))); + new SpannerWriteFn(this))); return PDone.in(input.getPipeline()); } @@ -211,18 +221,11 @@ public void populateDisplayData(DisplayData.Builder builder) { } /** - * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in - * batches, where the maximum batch size is {@link SpannerIO#DEFAULT_BATCH_SIZE}. - * - *

    Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be - * idempotent. - * - *

    See Google Cloud Spanner documentation. + * Batches together and writes mutations to Google Cloud Spanner. */ @VisibleForTesting - static class SpannerWriterFn extends DoFn { - private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + static class SpannerWriteFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteFn.class); private final Write spec; private transient Spanner spanner; private transient DatabaseClient dbClient; @@ -236,7 +239,7 @@ static class SpannerWriterFn extends DoFn { .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(Write spec) { + SpannerWriteFn(Write spec) { this.spec = spec; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java index 10b32ec70386..e593c187a80f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -101,7 +101,7 @@ public synchronized void batching() throws Exception { .withDatabaseId("test-database") .withBatchSize(1000000000) .withServiceFactory(serviceFactory); - SpannerIO.SpannerWriterFn writerFn = new SpannerIO.SpannerWriterFn(write); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); fnTester.processBundle(Arrays.asList(one, two)); @@ -122,7 +122,7 @@ public synchronized void noBatching() throws Exception { .withDatabaseId("test-database") .withBatchSize(0) // turn off batching. .withServiceFactory(serviceFactory); - SpannerIO.SpannerWriterFn writerFn = new SpannerIO.SpannerWriterFn(write); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); fnTester.processBundle(Arrays.asList(one, two)); From 2b5a10e7b64148db2bbcfadbef81dee7c0e10347 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 16:56:23 -0700 Subject: [PATCH 16/23] Reorganized test. Better formatting --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 91 +++--- .../spanner/MutationSizeEstimatorTest.java | 207 +++++++------- .../sdk/io/gcp/spanner/SpannerIOTest.java | 260 ++++++++++-------- 3 files changed, 300 insertions(+), 258 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 1e0707407980..7e1a4aa251ff 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -17,13 +17,15 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import static com.google.common.base.Preconditions.checkNotNull; + import com.google.auto.value.AutoValue; +import com.google.cloud.ServiceFactory; import com.google.cloud.ServiceOptions; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; import com.google.cloud.spanner.Mutation; -import com.google.cloud.ServiceFactory; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; @@ -48,16 +50,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static com.google.common.base.Preconditions.checkNotNull; - /** - * Experimental {@link PTransform Transforms} for reading from and writing to - * Google Cloud Spanner. + * Experimental {@link PTransform Transforms} for reading from and writing to Google Cloud Spanner. * *

    Reading from Cloud Spanner

    + * *

    This functionality is not yet implemented. * *

    Writing to Cloud Spanner

    + * *

    The Cloud Spanner {@link SpannerIO.Write} transform writes to Cloud Spanner by executing a * collection of input row {@link Mutation Mutations}. The mutations grouped into batches for * efficiency. @@ -74,21 +76,20 @@ * "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database")); * } * - * The default size of the batch is set to 1MB, to override this use - * {@link Write#withBatchSize(long)}. Setting batch size to a small value or zero - * practically disables batching. + * The default size of the batch is set to 1MB, to override this use {@link + * Write#withBatchSize(long)}. Setting batch size to a small value or zero practically disables + * batching. * - * The transform does not provide same transactional guarantees as Cloud Spanner. In particular, + *

    The transform does not provide same transactional guarantees as Cloud Spanner. In particular, *

  • Mutations are not submitted atomically; *
  • A mutation is applied at least once; *
  • If the pipeline was unexpectedly stopped, mutations that were already applied will not get - * rolled back. - * + * rolled back. */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - private static final long DEFAULT_BATCH_SIZE = 1024 * 1024; // 1 MB + private static final long DEFAULT_BATCH_SIZE = 1024 * 1024; // 1 MB /** * Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be @@ -120,7 +121,8 @@ public abstract static class Write extends PTransform, PDo abstract long getBatchSize(); - @Nullable @VisibleForTesting + @Nullable + @VisibleForTesting abstract ServiceFactory getServiceFactory(); abstract Builder toBuilder(); @@ -151,8 +153,7 @@ SpannerOptions getSpannerOptions() { } /** - * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner - * project. + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner project. * *

    Does not modify this object. */ @@ -170,7 +171,7 @@ public Write withInstanceId(String instanceId) { return toBuilder().setInstanceId(instanceId).build(); } - /** + /** * Returns a new {@link SpannerIO.Write} with a new batch size limit. * *

    Does not modify this object. @@ -196,16 +197,17 @@ Write withServiceFactory(ServiceFactory serviceFactory) @Override public void validate(PipelineOptions options) { - checkNotNull(getInstanceId(), "SpannerIO.write() requires instance id to be set with " - + "withInstanceId method"); - checkNotNull(getDatabaseId(), "SpannerIO.write() requires database id to be set with " - + "withDatabaseId method"); + checkNotNull( + getInstanceId(), + "SpannerIO.write() requires instance id to be set with " + "withInstanceId method"); + checkNotNull( + getDatabaseId(), + "SpannerIO.write() requires database id to be set with " + "withDatabaseId method"); } @Override public PDone expand(PCollection input) { - input.apply("Write mutations to Cloud Spanner", ParDo.of( - new SpannerWriteFn(this))); + input.apply("Write mutations to Cloud Spanner", ParDo.of(new SpannerWriteFn(this))); return PDone.in(input.getPipeline()); } @@ -213,16 +215,14 @@ public PDone expand(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", getInstanceId()) - .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()) - .withLabel("Output Database")); + .addIfNotNull( + DisplayData.item("instanceId", getInstanceId()).withLabel("Output Instance")) + .addIfNotNull( + DisplayData.item("databaseId", getDatabaseId()).withLabel("Output Database")); } } - /** - * Batches together and writes mutations to Google Cloud Spanner. - */ + /** Batches together and writes mutations to Google Cloud Spanner. */ @VisibleForTesting static class SpannerWriteFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteFn.class); @@ -236,18 +236,20 @@ static class SpannerWriteFn extends DoFn { private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = FluentBackoff.DEFAULT - .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + .withMaxRetries(MAX_RETRIES) + .withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting SpannerWriteFn(Write spec) { - this.spec = spec; + this.spec = spec; } @Setup public void setup() throws Exception { spanner = spec.getSpannerOptions().getService(); - dbClient = spanner.getDatabaseClient( - DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); + dbClient = + spanner.getDatabaseClient( + DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); mutations = new ArrayList<>(); batchSize = 0; } @@ -263,8 +265,9 @@ public void processElement(ProcessContext c) throws Exception { } private String projectId() { - return spec.getProjectId() == null ? - ServiceOptions.getDefaultProjectId() : spec.getProjectId(); + return spec.getProjectId() == null + ? ServiceOptions.getDefaultProjectId() + : spec.getProjectId(); } @FinishBundle @@ -285,12 +288,11 @@ public void teardown() throws Exception { /** * Writes a batch of mutations to Cloud Spanner. * - *

    If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. - * If the retry limit is exceeded, the last exception from Cloud Spanner will be - * thrown. + *

    If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. If the retry limit + * is exceeded, the last exception from Cloud Spanner will be thrown. * * @throws AbortedException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. + * backing off between retries fails. */ private void flushBatch() throws AbortedException, IOException, InterruptedException { LOG.debug("Writing batch of {} mutations", mutations.size()); @@ -307,8 +309,8 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } catch (AbortedException exception) { // Only log the code and message for potentially-transient errors. The entire exception // will be propagated upon the last retry. - LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), - exception.getMessage()); + LOG.error( + "Error writing to Spanner ({}): {}", exception.getCode(), exception.getMessage()); if (!BackOffUtils.next(sleeper, backoff)) { LOG.error("Aborting after {} retries.", MAX_RETRIES); throw exception; @@ -324,13 +326,10 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", spec.getInstanceId()) - .withLabel("Instance")) - .addIfNotNull(DisplayData.item("databaseId", spec.getDatabaseId()) - .withLabel("Database")); + .addIfNotNull(DisplayData.item("instanceId", spec.getInstanceId()).withLabel("Instance")) + .addIfNotNull(DisplayData.item("databaseId", spec.getDatabaseId()).withLabel("Database")); } } private SpannerIO() {} // Prevent construction. - } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java index e9f419894b09..d1c2c273cf1a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -1,109 +1,120 @@ package org.apache.beam.sdk.io.gcp.spanner; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; -import org.junit.Test; - import java.util.Arrays; - -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; +import org.junit.Test; public class MutationSizeEstimatorTest { - @Test - public void primitives() throws Exception { - Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); - Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2.9).build(); - Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").to(false).build(); - - assertThat(MutationSizeEstimator.sizeOf(int64), is(8L)); - assertThat(MutationSizeEstimator.sizeOf(float64), is(8L)); - assertThat(MutationSizeEstimator.sizeOf(bool), is(1L)); - } - - @Test - public void primitiveArrays() throws Exception { - Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").toInt64Array(new - long[]{1L, 2L, 3L}) - .build(); - Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").toFloat64Array(new - double[]{1., 2.}).build(); - Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").toBoolArray(new - boolean[] {true, true, false, true}) - .build(); - - assertThat(MutationSizeEstimator.sizeOf(int64), is(24L)); - assertThat(MutationSizeEstimator.sizeOf(float64), is(16L)); - assertThat(MutationSizeEstimator.sizeOf(bool), is(4L)); - } - - @Test - public void strings() throws Exception { - Mutation emptyString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("").build(); - Mutation nullString = Mutation.newInsertOrUpdateBuilder("test").set("one").to((String)null) - .build(); - Mutation sampleString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("abc") - .build(); - Mutation sampleArray = Mutation.newInsertOrUpdateBuilder("test").set("one") - .toStringArray(Arrays.asList("one", "two", null)) - .build(); - - assertThat(MutationSizeEstimator.sizeOf(emptyString), is(0L)); - assertThat(MutationSizeEstimator.sizeOf(nullString), is(0L)); - assertThat(MutationSizeEstimator.sizeOf(sampleString), is(3L)); - assertThat(MutationSizeEstimator.sizeOf(sampleArray), is(6L)); - } - - @Test - public void bytes() throws Exception { - Mutation empty = Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray - .fromBase64("")) - .build(); - Mutation nullValue = Mutation.newInsertOrUpdateBuilder("test").set("one").to((ByteArray)null) - .build(); - Mutation sample = Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray - .fromBase64("abcdabcd")) - .build(); - - assertThat(MutationSizeEstimator.sizeOf(empty), is(0L)); - assertThat(MutationSizeEstimator.sizeOf(nullValue), is(0L)); - assertThat(MutationSizeEstimator.sizeOf(sample), is(6L)); - } - - @Test - public void dates() throws Exception { - Mutation timestamp = Mutation.newInsertOrUpdateBuilder("test").set("one").to(Timestamp - .now()) - .build(); - Mutation nullTimestamp = Mutation.newInsertOrUpdateBuilder("test").set("one").to( - (Timestamp)null) - .build(); - Mutation date = Mutation.newInsertOrUpdateBuilder("test").set("one").to(Date - .fromYearMonthDay(2017, 10, 10)) - .build(); - Mutation nullDate = Mutation.newInsertOrUpdateBuilder("test").set("one").to( - (Date)null) - .build(); - Mutation timestampArray = Mutation.newInsertOrUpdateBuilder("test").set("one") - .toTimestampArray(Arrays.asList(Timestamp - .now(), null)) - .build(); - Mutation dateArray = Mutation.newInsertOrUpdateBuilder("test").set("one") - .toDateArray(Arrays.asList(null, Date.fromYearMonthDay(2017, 1, 1), null, Date - .fromYearMonthDay(2017, 1, 2))) - .build(); - - - assertThat(MutationSizeEstimator.sizeOf(timestamp), is(12L)); - assertThat(MutationSizeEstimator.sizeOf(date), is(12L)); - assertThat(MutationSizeEstimator.sizeOf(nullTimestamp), is(12L)); - assertThat(MutationSizeEstimator.sizeOf(nullDate), is(12L)); - assertThat(MutationSizeEstimator.sizeOf(timestampArray), is(24L)); - assertThat(MutationSizeEstimator.sizeOf(dateArray), is(48L)); - } - - + @Test + public void primitives() throws Exception { + Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2.9).build(); + Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").to(false).build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(1L)); + } + + @Test + public void primitiveArrays() throws Exception { + Mutation int64 = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toInt64Array(new long[] {1L, 2L, 3L}) + .build(); + Mutation float64 = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toFloat64Array(new double[] {1., 2.}) + .build(); + Mutation bool = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toBoolArray(new boolean[] {true, true, false, true}) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(16L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(4L)); + } + + @Test + public void strings() throws Exception { + Mutation emptyString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("").build(); + Mutation nullString = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((String) null).build(); + Mutation sampleString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("abc").build(); + Mutation sampleArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toStringArray(Arrays.asList("one", "two", null)) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(emptyString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sampleString), is(3L)); + assertThat(MutationSizeEstimator.sizeOf(sampleArray), is(6L)); + } + + @Test + public void bytes() throws Exception { + Mutation empty = + Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray.fromBase64("")).build(); + Mutation nullValue = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((ByteArray) null).build(); + Mutation sample = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(ByteArray.fromBase64("abcdabcd")) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(empty), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullValue), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sample), is(6L)); + } + + @Test + public void dates() throws Exception { + Mutation timestamp = + Mutation.newInsertOrUpdateBuilder("test").set("one").to(Timestamp.now()).build(); + Mutation nullTimestamp = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((Timestamp) null).build(); + Mutation date = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(Date.fromYearMonthDay(2017, 10, 10)) + .build(); + Mutation nullDate = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((Date) null).build(); + Mutation timestampArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toTimestampArray(Arrays.asList(Timestamp.now(), null)) + .build(); + Mutation dateArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toDateArray( + Arrays.asList( + null, + Date.fromYearMonthDay(2017, 1, 1), + null, + Date.fromYearMonthDay(2017, 1, 2))) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(timestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(date), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullTimestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullDate), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(timestampArray), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(dateArray), is(48L)); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java index e593c187a80f..05a28d7d1b58 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -1,9 +1,14 @@ package org.apache.beam.sdk.io.gcp.spanner; +import static org.mockito.Mockito.*; + import avro.shaded.com.google.common.collect.Iterables; import com.google.api.core.ApiFuture; import com.google.cloud.ServiceFactory; import com.google.cloud.spanner.*; +import java.io.Serializable; +import java.util.*; +import javax.annotation.concurrent.GuardedBy; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -19,136 +24,163 @@ import org.mockito.ArgumentMatcher; import org.mockito.Matchers; -import java.io.Serializable; -import java.util.Arrays; - -import static org.mockito.Mockito.*; - @RunWith(JUnit4.class) public class SpannerIOTest implements Serializable { - @Rule - public final transient TestPipeline pipeline = TestPipeline.create(); - @Rule - public transient ExpectedException thrown = ExpectedException.none(); - // Tests that use serviceFactory must be serializable, to avoid parallel execution. - private ServiceFactory serviceFactory; + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + private FakeServiceFactory serviceFactory; + + @Before + @SuppressWarnings("unchecked") + public void setUp() throws Exception { + serviceFactory = new FakeServiceFactory(); + } + + @Test + public void emptyTransform() throws Exception { + SpannerIO.Write write = SpannerIO.write(); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyInstanceId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withDatabaseId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyDatabaseId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withInstanceId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires database id to be set with"); + write.validate(null); + } + + @Test + @Category(NeedsRunner.class) + public void singleMutationPipeline() throws Exception { + Mutation mutation = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2).build(); + PCollection mutations = pipeline.apply(Create.of(mutation)); + + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withServiceFactory(serviceFactory); + mutations.apply(write); + pipeline.run(); + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + @Test + public void batching() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSize(1000000000) + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(2))); + } + + @Test + public void noBatching() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSize(0) // turn off batching. + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(2)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + private static class FakeServiceFactory + implements ServiceFactory, Serializable { // Marked as static so they could be returned by serviceFactory, which is serializable. - private static Spanner mockSpanner; - private static DatabaseClient mockDatabaseClient; - - @Before - public void setUp() throws Exception { - ApiFuture voidFuture = mock(ApiFuture.class, withSettings().serializable()); - mockSpanner = mock(Spanner.class, withSettings().serializable()); - mockDatabaseClient = mock(DatabaseClient.class, withSettings().serializable()); - serviceFactory = mock(ServiceFactory.class, withSettings().serializable()); - when(mockSpanner.getDatabaseClient(Matchers.any(DatabaseId.class))).thenReturn(mockDatabaseClient); - when(mockSpanner.closeAsync()).thenReturn(voidFuture); - serviceFactory = new FakeServiceFactory(); - } + private static final Object lock = new Object(); - @Test - public void emptyTransform() throws Exception { - SpannerIO.Write write = SpannerIO.write(); - thrown.expect(NullPointerException.class); - thrown.expectMessage("requires instance id to be set with"); - write.validate(null); - } + @GuardedBy("lock") + private static final List mockSpanners = new ArrayList<>(); - @Test - public void emptyInstanceId() throws Exception { - SpannerIO.Write write = SpannerIO.write().withDatabaseId("123"); - thrown.expect(NullPointerException.class); - thrown.expectMessage("requires instance id to be set with"); - write.validate(null); - } + @GuardedBy("lock") + private static final List mockDatabaseClients = new ArrayList<>(); - @Test - public void emptyDatabaseId() throws Exception { - SpannerIO.Write write = SpannerIO.write().withInstanceId("123"); - thrown.expect(NullPointerException.class); - thrown.expectMessage("requires database id to be set with"); - write.validate(null); - } + @GuardedBy("lock") + private static int count = 0; + + private final int index; - @Test - @Category(NeedsRunner.class) - public synchronized void singleMutationPipeline() throws Exception { - Mutation mutation = Mutation - .newInsertOrUpdateBuilder("test").set("one").to(2).build(); - PCollection mutations = pipeline.apply(Create.of(mutation)); - - SpannerIO.Write write = SpannerIO.write() - .withProjectId("test-project") - .withInstanceId("test-instance").withDatabaseId - ("test-database").withServiceFactory(serviceFactory); - mutations.apply(write); - pipeline.run(); - verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", - "test-instance", "test-database")); - verify(mockDatabaseClient, times(1)).writeAtLeastOnce(argThat(new IterableOfSize(1))); + public FakeServiceFactory() { + synchronized (lock) { + index = count++; + mockSpanners.add(mock(Spanner.class, withSettings().serializable())); + mockDatabaseClients.add(mock(DatabaseClient.class, withSettings().serializable())); + } + ApiFuture voidFuture = mock(ApiFuture.class, withSettings().serializable()); + when(mockSpanner().getDatabaseClient(Matchers.any(DatabaseId.class))) + .thenReturn(mockDatabaseClient()); + when(mockSpanner().closeAsync()).thenReturn(voidFuture); } - @Test - public synchronized void batching() throws Exception { - Mutation one = Mutation - .newInsertOrUpdateBuilder("test").set("one").to(1).build(); - Mutation two = Mutation - .newInsertOrUpdateBuilder("test").set("two").to(2).build(); - SpannerIO.Write write = SpannerIO.write() - .withProjectId("test-project") - .withInstanceId("test-instance") - .withDatabaseId("test-database") - .withBatchSize(1000000000) - .withServiceFactory(serviceFactory); - SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); - DoFnTester fnTester = DoFnTester.of(writerFn); - fnTester.processBundle(Arrays.asList(one, two)); - - verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", - "test-instance", "test-database")); - verify(mockDatabaseClient, times(1)).writeAtLeastOnce(argThat(new IterableOfSize(2))); + DatabaseClient mockDatabaseClient() { + synchronized (lock) { + return mockDatabaseClients.get(index); + } } - @Test - public synchronized void noBatching() throws Exception { - Mutation one = Mutation - .newInsertOrUpdateBuilder("test").set("one").to(1).build(); - Mutation two = Mutation - .newInsertOrUpdateBuilder("test").set("two").to(2).build(); - SpannerIO.Write write = SpannerIO.write() - .withProjectId("test-project") - .withInstanceId("test-instance") - .withDatabaseId("test-database") - .withBatchSize(0) // turn off batching. - .withServiceFactory(serviceFactory); - SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); - DoFnTester fnTester = DoFnTester.of(writerFn); - fnTester.processBundle(Arrays.asList(one, two)); - - verify(mockSpanner).getDatabaseClient(DatabaseId.of("test-project", - "test-instance", "test-database")); - verify(mockDatabaseClient, times(2)).writeAtLeastOnce(argThat(new IterableOfSize(1))); + Spanner mockSpanner() { + synchronized (lock) { + return mockSpanners.get(index); + } } - private static class FakeServiceFactory implements ServiceFactory, - Serializable { - @Override - public Spanner create(SpannerOptions serviceOptions) { - return mockSpanner; - } + @Override + public Spanner create(SpannerOptions serviceOptions) { + synchronized (lock) { + return mockSpanner(); + } } + } - private static class IterableOfSize extends ArgumentMatcher> { - private final int size; + private static class IterableOfSize extends ArgumentMatcher> { + private final int size; - private IterableOfSize(int size) { - this.size = size; - } + private IterableOfSize(int size) { + this.size = size; + } - @Override - public boolean matches(Object argument) { - return argument instanceof Iterable && Iterables.size((Iterable) argument) == size; - } + @Override + public boolean matches(Object argument) { + return argument instanceof Iterable && Iterables.size((Iterable) argument) == size; } + } } From fd2a76cf8b4fe31be3f7e665e11affe2602cf6a9 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 17:09:37 -0700 Subject: [PATCH 17/23] Pre PR cleanup --- sdks/java/io/google-cloud-platform/pom.xml | 5 - .../io/gcp/spanner/MutationSizeEstimator.java | 136 +++++++++--------- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 4 +- .../sdk/io/gcp/spanner/SpannerIOTest.java | 42 ++++-- 4 files changed, 100 insertions(+), 87 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 9efecbc7dd1b..7938694c85a7 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -167,11 +167,6 @@ google-cloud-spanner - - com.google.cloud - google-cloud-spanner - - com.google.cloud.bigtable bigtable-protos diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java index a8c43bfd015a..887db618ea3f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java @@ -4,84 +4,78 @@ import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Value; -/** - * Estimates the logical size of {@link com.google.cloud.spanner.Mutation}. - */ +/** Estimates the logical size of {@link com.google.cloud.spanner.Mutation}. */ class MutationSizeEstimator { - // Prevent construction. - private MutationSizeEstimator() { - } + // Prevent construction. + private MutationSizeEstimator() {} - /** - * Estimates a size of mutation in bytes. - */ - static long sizeOf(Mutation m) { - long result = 0; - for (Value v : m.getValues()) { - switch (v.getType().getCode()) { - case ARRAY: - result += estimateArrayValue(v); - break; - case STRUCT: - throw new IllegalArgumentException("Structs are not supported in mutation."); - default: - result += estimatePrimitiveValue(v); - } - } - return result; + /** Estimates a size of mutation in bytes. */ + static long sizeOf(Mutation m) { + long result = 0; + for (Value v : m.getValues()) { + switch (v.getType().getCode()) { + case ARRAY: + result += estimateArrayValue(v); + break; + case STRUCT: + throw new IllegalArgumentException("Structs are not supported in mutation."); + default: + result += estimatePrimitiveValue(v); + } } + return result; + } - private static long estimatePrimitiveValue(Value v) { - switch (v.getType().getCode()) { - case BOOL: - return 1; - case INT64: - case FLOAT64: - return 8; - case DATE: - case TIMESTAMP: - return 12; - case STRING: - return v.isNull()? 0 : v.getString().length(); - case BYTES: - return v.isNull()? 0 : v.getBytes().length(); - } - throw new IllegalArgumentException("Unsupported type " + v.getType()); + private static long estimatePrimitiveValue(Value v) { + switch (v.getType().getCode()) { + case BOOL: + return 1; + case INT64: + case FLOAT64: + return 8; + case DATE: + case TIMESTAMP: + return 12; + case STRING: + return v.isNull() ? 0 : v.getString().length(); + case BYTES: + return v.isNull() ? 0 : v.getBytes().length(); } + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } - private static long estimateArrayValue(Value v) { - switch (v.getType().getArrayElementType().getCode()) { - case BOOL: - return v.getBoolArray().size(); - case INT64: - return 8 * v.getInt64Array().size(); - case FLOAT64: - return 8 * v.getFloat64Array().size(); - case STRING: - long totalLength = 0; - for (String s : v.getStringArray()) { - if (s == null) { - continue; - } - totalLength += s.length(); - } - return totalLength; - case BYTES: - totalLength = 0; - for (ByteArray bytes : v.getBytesArray()) { - if (bytes == null) { - continue; - } - totalLength += bytes.length(); - } - return totalLength; - case DATE: - return 12 * v.getDateArray().size(); - case TIMESTAMP: - return 12 * v.getTimestampArray().size(); + private static long estimateArrayValue(Value v) { + switch (v.getType().getArrayElementType().getCode()) { + case BOOL: + return v.getBoolArray().size(); + case INT64: + return 8 * v.getInt64Array().size(); + case FLOAT64: + return 8 * v.getFloat64Array().size(); + case STRING: + long totalLength = 0; + for (String s : v.getStringArray()) { + if (s == null) { + continue; + } + totalLength += s.length(); } - throw new IllegalArgumentException("Unsupported type " + v.getType()); + return totalLength; + case BYTES: + totalLength = 0; + for (ByteArray bytes : v.getBytesArray()) { + if (bytes == null) { + continue; + } + totalLength += bytes.length(); + } + return totalLength; + case DATE: + return 12 * v.getDateArray().size(); + case TIMESTAMP: + return 12 * v.getTimestampArray().size(); } - + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 7e1a4aa251ff..8aa61663c2f5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -199,10 +199,10 @@ Write withServiceFactory(ServiceFactory serviceFactory) public void validate(PipelineOptions options) { checkNotNull( getInstanceId(), - "SpannerIO.write() requires instance id to be set with " + "withInstanceId method"); + "SpannerIO.write() requires instance id to be set with withInstanceId method"); checkNotNull( getDatabaseId(), - "SpannerIO.write() requires database id to be set with " + "withDatabaseId method"); + "SpannerIO.write() requires database id to be set with withDatabaseId method"); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java index 05a28d7d1b58..f820dfb004d9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -67,39 +67,65 @@ public void singleMutationPipeline() throws Exception { Mutation mutation = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2).build(); PCollection mutations = pipeline.apply(Create.of(mutation)); + mutations.apply(SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withServiceFactory(serviceFactory)); + pipeline.run(); + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + @Test + public void batching() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); SpannerIO.Write write = SpannerIO.write() .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") + .withBatchSize(1000000000) .withServiceFactory(serviceFactory); - mutations.apply(write); - pipeline.run(); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + verify(serviceFactory.mockSpanner()) .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); verify(serviceFactory.mockDatabaseClient(), times(1)) - .writeAtLeastOnce(argThat(new IterableOfSize(1))); + .writeAtLeastOnce(argThat(new IterableOfSize(2))); } @Test - public void batching() throws Exception { + public void batchingGroups() throws Exception { Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + Mutation three = Mutation.newInsertOrUpdateBuilder("test").set("three").to(3).build(); + + // Have a room to accumulate one more item. + long batchSize = MutationSizeEstimator.sizeOf(one) + 1; + SpannerIO.Write write = SpannerIO.write() .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") - .withBatchSize(1000000000) + .withBatchSize(batchSize) .withServiceFactory(serviceFactory); SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); - fnTester.processBundle(Arrays.asList(one, two)); + fnTester.processBundle(Arrays.asList(one, two, three)); verify(serviceFactory.mockSpanner()) .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); verify(serviceFactory.mockDatabaseClient(), times(1)) .writeAtLeastOnce(argThat(new IterableOfSize(2))); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); } @Test @@ -165,9 +191,7 @@ Spanner mockSpanner() { @Override public Spanner create(SpannerOptions serviceOptions) { - synchronized (lock) { - return mockSpanner(); - } + return mockSpanner(); } } From e24254c2d1d82638b76347a473a46acfef768845 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 18:09:44 -0700 Subject: [PATCH 18/23] Sketch out integration test --- .../sdk/io/gcp/spanner/SpannerWriteIT.java | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java new file mode 100644 index 000000000000..57e73d42dd64 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java @@ -0,0 +1,81 @@ +package org.apache.beam.sdk.io.gcp.spanner; + +import com.google.cloud.spanner.*; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableTestOptions; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class SpannerWriteIT { + + public interface SpannerTestPipelineOptions extends TestPipelineOptions { + @Description("Project ID for Spanner") + @Default.String("apache-beam-testing") + String getProjectId(); + + void setProjectId(String value); + + @Description("Instance ID to write to in Spanner") + @Default.String("beam-test") + String getInstanceId(); + + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Default.String("beam-testdb") + String getDatabaseId(); + + void setDatabaseId(String value); + + @Description("Table name") + @Default.String("users") + String getTable(); + + void setTable(String value); + } + + private Spanner spanner; + private DatabaseAdminClient databaseAdminClient; + private SpannerTestPipelineOptions options; + + @Before + public void setUp() throws Exception { + PipelineOptionsFactory.register(SpannerTestPipelineOptions.class); + options = TestPipeline.testingPipelineOptions().as + (SpannerTestPipelineOptions.class); + + spanner = SpannerOptions.newBuilder().setProjectId(options.getProjectId()) + .build().getService(); + + databaseAdminClient = spanner.getDatabaseAdminClient(); + Operation op = databaseAdminClient.createDatabase( + options.getInstanceId(), options + .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" + + " Key INT64," + + " Name STRING," + + " Email STRING," + + " Age INT," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + @Test + public void testWrite() throws Exception { + + } + + @After + public void tearDown() throws Exception { + databaseAdminClient.dropDatabase(options.getInstanceId(), options + .getDatabaseId()); + + } +} From 34c7c6f378d19a2c528954c3d20780921e9461dc Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 18:40:21 -0700 Subject: [PATCH 19/23] End to end test for Cloud Spanner Sink --- pom.xml | 2 +- .../sdk/io/gcp/spanner/SpannerWriteIT.java | 177 +++++++++++------- 2 files changed, 115 insertions(+), 64 deletions(-) diff --git a/pom.xml b/pom.xml index 52b7656a2acc..7395ca68ff6e 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,7 @@ 0.5.160304 20.0 1.2.0 - 0.1.0 + 0.1.9 1.3 2.8.8 3.0.1 diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java index 57e73d42dd64..3abfd97783cd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java @@ -1,81 +1,132 @@ package org.apache.beam.sdk.io.gcp.spanner; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + import com.google.cloud.spanner.*; import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableTestOptions; +import java.util.Collections; +import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.commons.lang3.RandomStringUtils; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; -import java.util.Collections; - +/** End-to-end test of Cloud Spanner Sink. */ +@RunWith(JUnit4.class) public class SpannerWriteIT { - - public interface SpannerTestPipelineOptions extends TestPipelineOptions { - @Description("Project ID for Spanner") - @Default.String("apache-beam-testing") - String getProjectId(); - - void setProjectId(String value); - - @Description("Instance ID to write to in Spanner") - @Default.String("beam-test") - String getInstanceId(); - - void setInstanceId(String value); - - @Description("Database ID to write to in Spanner") - @Default.String("beam-testdb") - String getDatabaseId(); - - void setDatabaseId(String value); - - @Description("Table name") - @Default.String("users") - String getTable(); - - void setTable(String value); - } - - private Spanner spanner; - private DatabaseAdminClient databaseAdminClient; - private SpannerTestPipelineOptions options; - - @Before - public void setUp() throws Exception { - PipelineOptionsFactory.register(SpannerTestPipelineOptions.class); - options = TestPipeline.testingPipelineOptions().as - (SpannerTestPipelineOptions.class); - - spanner = SpannerOptions.newBuilder().setProjectId(options.getProjectId()) - .build().getService(); - - databaseAdminClient = spanner.getDatabaseAdminClient(); - Operation op = databaseAdminClient.createDatabase( - options.getInstanceId(), options - .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" - + " Key INT64," - + " Name STRING," - + " Email STRING," - + " Age INT," - + ") PRIMARY KEY (Key)")); - op.waitFor(); - } - - @Test - public void testWrite() throws Exception { - + @Rule public final transient TestPipeline p = TestPipeline.create(); + + public interface SpannerTestPipelineOptions extends TestPipelineOptions { + @Description("Project ID for Spanner") + @Default.String("apache-beam-testing") + String getProjectId(); + + void setProjectId(String value); + + @Description("Instance ID to write to in Spanner") + @Default.String("beam-test") + String getInstanceId(); + + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Default.String("beam-testdb") + String getDatabaseId(); + + void setDatabaseId(String value); + + @Description("Table name") + @Default.String("users") + String getTable(); + + void setTable(String value); + } + + private Spanner spanner; + private DatabaseAdminClient databaseAdminClient; + private SpannerTestPipelineOptions options; + + @Before + public void setUp() throws Exception { + PipelineOptionsFactory.register(SpannerTestPipelineOptions.class); + options = TestPipeline.testingPipelineOptions().as(SpannerTestPipelineOptions.class); + + spanner = SpannerOptions.newBuilder().setProjectId(options.getProjectId()).build().getService(); + + databaseAdminClient = spanner.getDatabaseAdminClient(); + Operation op = + databaseAdminClient.createDatabase( + options.getInstanceId(), + options.getDatabaseId(), + Collections.singleton( + "CREATE TABLE " + + options.getTable() + + " (" + + " Key INT64," + + " Value STRING(MAX)," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + @Test + public void testWrite() throws Exception { + p.apply(GenerateSequence.from(0).to(100)) + .apply(ParDo.of(new GenerateMutations(options.getTable()))) + .apply( + SpannerIO.write() + .withProjectId(options.getProjectId()) + .withInstanceId(options.getInstanceId()) + .withDatabaseId(options.getDatabaseId())); + + p.run(); + DatabaseClient databaseClient = + spanner.getDatabaseClient( + DatabaseId.of( + options.getProjectId(), options.getInstanceId(), options.getDatabaseId())); + + ResultSet resultSet = + databaseClient + .singleUse() + .executeQuery(Statement.of("SELECT COUNT(*) FROM " + options.getTable())); + assertThat(resultSet.next(), is(true)); + assertThat(resultSet.getLong(0), equalTo(100L)); + assertThat(resultSet.next(), is(false)); + } + + @After + public void tearDown() throws Exception { + databaseAdminClient.dropDatabase(options.getInstanceId(), options.getDatabaseId()); + spanner.closeAsync().get(); + } + + private static class GenerateMutations extends DoFn { + private final String table; + private final int valueSize = 100; + + public GenerateMutations(String table) { + this.table = table; } - @After - public void tearDown() throws Exception { - databaseAdminClient.dropDatabase(options.getInstanceId(), options - .getDatabaseId()); - + @ProcessElement + public void processElement(ProcessContext c) { + Mutation.WriteBuilder builder = Mutation.newInsertOrUpdateBuilder(table); + Long key = c.element(); + builder.set("Key").to(key); + builder.set("Value").to(RandomStringUtils.random(valueSize, true, true)); + Mutation mutation = builder.build(); + c.output(mutation); } + } } From f30b520159df45592f98ef7371e9a099c37472bd Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 15 May 2017 23:27:13 -0700 Subject: [PATCH 20/23] Make checkstyle happy --- pom.xml | 19 ++++++++++ sdks/java/io/google-cloud-platform/pom.xml | 25 ++++++++++--- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 2 +- .../spanner/MutationSizeEstimatorTest.java | 1 + .../sdk/io/gcp/spanner/SpannerIOTest.java | 35 ++++++++++++++----- .../sdk/io/gcp/spanner/SpannerWriteIT.java | 12 ++++++- 6 files changed, 78 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index 7395ca68ff6e..57ae07d5810f 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,7 @@ -Xpkginfo:always nothing 0.16.0-beta + 1.0.0-rc3 pom @@ -814,6 +815,12 @@ + + com.google.cloud + google-cloud-core + ${google-cloud-core.version} + + com.google.cloud.bigtable bigtable-protos @@ -915,6 +922,18 @@ ${protobuf.version} + + com.google.api.grpc + proto-google-common-protos + ${grpc-google-common-protos.version} + + + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + ${grpc-google-common-protos.version} + + com.google.api.grpc grpc-google-common-protos diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 7938694c85a7..f7a99439f944 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -89,11 +89,6 @@ grpc-core - - com.google.api.grpc - grpc-google-common-protos - - com.google.apis google-api-services-bigquery @@ -109,6 +104,11 @@ google-api-services-pubsub + + com.google.cloud + google-cloud-core + + com.google.api.grpc grpc-google-pubsub-v1 @@ -246,6 +246,21 @@ true + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + + + + com.google.api.grpc + proto-google-common-protos + + + + org.apache.commons + commons-lang3 + + org.apache.beam diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 8aa61663c2f5..8444ac8bce96 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -76,7 +76,7 @@ * "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database")); * } * - * The default size of the batch is set to 1MB, to override this use {@link + *

    The default size of the batch is set to 1MB, to override this use {@link * Write#withBatchSize(long)}. Setting batch size to a small value or zero practically disables * batching. * diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java index d1c2c273cf1a..da7c42a3e47d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -10,6 +10,7 @@ import java.util.Arrays; import org.junit.Test; +/** A set of unit tests for {@link MutationSizeEstimator}. */ public class MutationSizeEstimatorTest { @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java index f820dfb004d9..6fbc5f12c78b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -1,14 +1,26 @@ package org.apache.beam.sdk.io.gcp.spanner; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; -import avro.shaded.com.google.common.collect.Iterables; import com.google.api.core.ApiFuture; import com.google.cloud.ServiceFactory; -import com.google.cloud.spanner.*; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.collect.Iterables; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import javax.annotation.concurrent.GuardedBy; + import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -24,6 +36,10 @@ import org.mockito.ArgumentMatcher; import org.mockito.Matchers; + +/** + * Unit tests for {@link SpannerIO}. + */ @RunWith(JUnit4.class) public class SpannerIOTest implements Serializable { @Rule public final transient TestPipeline pipeline = TestPipeline.create(); @@ -67,11 +83,12 @@ public void singleMutationPipeline() throws Exception { Mutation mutation = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2).build(); PCollection mutations = pipeline.apply(Create.of(mutation)); - mutations.apply(SpannerIO.write() - .withProjectId("test-project") - .withInstanceId("test-instance") - .withDatabaseId("test-database") - .withServiceFactory(serviceFactory)); + mutations.apply( + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withServiceFactory(serviceFactory)); pipeline.run(); verify(serviceFactory.mockSpanner()) .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java index 3abfd97783cd..248f64645768 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java @@ -4,7 +4,16 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -import com.google.cloud.spanner.*; +import com.google.cloud.spanner.Database; +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.cloud.spanner.Statement; import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; import java.util.Collections; import org.apache.beam.sdk.io.GenerateSequence; @@ -28,6 +37,7 @@ public class SpannerWriteIT { @Rule public final transient TestPipeline p = TestPipeline.create(); + /** Pipeline options for this test. */ public interface SpannerTestPipelineOptions extends TestPipelineOptions { @Description("Project ID for Spanner") @Default.String("apache-beam-testing") From 1f256efd78af37f9860cbc8fa32a7732adeb23bf Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Wed, 24 May 2017 14:57:35 -0700 Subject: [PATCH 21/23] Merge branch 'master' of https://github.com/apache/beam into mergespanner # Conflicts: # pom.xml # sdks/java/io/google-cloud-platform/pom.xml # sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java --- pom.xml | 10 ++++++++++ sdks/java/io/google-cloud-platform/pom.xml | 20 +++++++++++++++----- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index ce162e9fbd4e..c0d17b0ca820 100644 --- a/pom.xml +++ b/pom.xml @@ -928,7 +928,17 @@ protobuf-java ${protobuf.version} + + com.google.api.grpc + proto-google-common-protos + ${grpc-google-common-protos.version} + + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + ${grpc-google-common-protos.version} + com.google.api.grpc grpc-google-common-protos diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 9143ccf553cf..8b5382092ec6 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -86,11 +86,6 @@ grpc-core - - com.google.api.grpc - grpc-google-common-protos - - com.google.apis google-api-services-bigquery @@ -248,6 +243,21 @@ true + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + + + + com.google.api.grpc + proto-google-common-protos + + + + org.apache.commons + commons-lang3 + + org.apache.beam From dabda8d1fe8f8ca7b09c6f3a48262fae24c29dc9 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Wed, 24 May 2017 15:03:09 -0700 Subject: [PATCH 22/23] Revert style change --- sdks/java/core/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index c034e705c95b..3c68c96e15fe 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -276,8 +276,8 @@ joda-time - + org.tukaani xz From c07a32f7c7025360918119e6a2563e97ce2ab2ac Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Wed, 24 May 2017 15:12:24 -0700 Subject: [PATCH 23/23] Addressing code review comments. --- pom.xml | 3500 +++++++++-------- .../io/gcp/spanner/MutationSizeEstimator.java | 17 + .../beam/sdk/io/gcp/spanner/SpannerIO.java | 52 +- .../spanner/MutationSizeEstimatorTest.java | 17 + .../sdk/io/gcp/spanner/SpannerIOTest.java | 23 +- .../sdk/io/gcp/spanner/SpannerWriteIT.java | 25 +- 6 files changed, 1858 insertions(+), 1776 deletions(-) diff --git a/pom.xml b/pom.xml index c0d17b0ca820..da2da26b9b30 100644 --- a/pom.xml +++ b/pom.xml @@ -17,1766 +17,1768 @@ --> - 4.0.0 - - - org.apache - apache - 18 - - - org.apache.beam - beam-parent - Apache Beam :: Parent - Apache Beam provides a simple, Java-based interface - for processing virtually any size data. This artifact includes the parent POM - for other Beam artifacts. - http://beam.apache.org/ - 2016 - - 2.1.0-SNAPSHOT - - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - scm:git:https://git-wip-us.apache.org/repos/asf/beam.git - scm:git:https://git-wip-us.apache.org/repos/asf/beam.git - https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary - HEAD - - - - jira - https://issues.apache.org/jira/browse/BEAM - - - - - Beam Dev - dev-subscribe@beam.apache.org - dev-unsubscribe@beam.apache.org - dev@beam.apache.org - http://www.mail-archive.com/dev%beam.apache.org - - - Beam User - user-subscribe@beam.apache.org - user-unsubscribe@beam.apache.org - user@beam.apache.org - http://www.mail-archive.com/user%beam.apache.org - - - Beam Commits - commits-subscribe@beam.apache.org - commits-unsubscribe@beam.apache.org - commits@beam.apache.org - http://www.mail-archive.com/commits%beam.apache.org - - - - - - The Apache Beam Team - dev@beam.apache.org - http://beam.apache.org - Apache Software Foundation - http://www.apache.org - - - - - UTF-8 - - - - true - - - - - - 3.5 - 1.9 - 2.24.0 - 1.0.0-rc2 - 1.8.1 - v2-rev295-1.22.0 - 0.9.6.2 - v1-rev6-1.22.0 - 0.1.0 - v2-rev8-1.22.0 - v1b3-rev196-1.22.0 - 0.5.160222 - 1.4.0 - 1.3.0 - 1.0.0-rc2 - 1.0-rc2 - 1.4.1 - 0.6.1 - 1.22.0 - 1.4.5 - 1.0.2 - 0.5.160304 - 20.0 - 1.2.0 - 0.1.9 - 1.3 - 2.8.8 - 3.0.1 - 2.4 - 4.12 - 1.9.5 - 4.1.8.Final - 1.1.33.Fork26 - 3.2.0 - v1-rev10-1.22.0 - 1.7.14 - 0.16.0-beta - 1.6.2 - 4.3.5.RELEASE - 3.1.4 - v1-rev71-1.22.0 - 4.4.1 - 4.3.5.RELEASE - 1.1.4-M3 - - 1.5.0.Final - 2.0 - 2.20 - 2.20 - 3.6.1 - 1.4.0 - 3.0.2 - 3.0.2 - 3.0.0 - - -Werror - -Xpkginfo:always - nothing - - - pom - - - - sdks/java/build-tools - sdks - runners - examples - - sdks/java/javadoc - - - - - - release - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - - attach-javadocs - package - - jar - - - - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - package - - jar-no-fork - - - - attach-test-sources - package - - test-jar-no-fork - - - - - - - org.apache.rat - apache-rat-plugin - - - verify - - check - - - - - - org.apache.maven.plugins - maven-resources-plugin - ${maven-resources-plugin.version} - - - - - - - org.apache.rat - apache-rat-plugin - - - - org.apache.maven.plugins - maven-javadoc-plugin - - - - org.apache.maven.plugins - maven-source-plugin - - - - - - - doclint-java8-disable - - [1.8,) - - - -Xdoclint:all -Xdoclint:-missing - - - - - java7-packageinfo - - 1.7 - - - - -Xpkginfo:legacy - **/package-info.java - - - - - maven-compiler-plugin - - - - compile-package-info - - compile - - compile - - - -Xpkginfo:always - - - **/package-info.java - - - - - - - - - - - eclipse-jdt - - - -Xlint:all - - - - - - maven-compiler-plugin - - jdt - - - - org.eclipse.tycho - tycho-compiler-jdt - 0.26.0 - - - - - - - - - - - - - org.apache.beam - beam-sdks-common-fn-api - ${project.version} - - - - org.apache.beam - beam-sdks-common-fn-api - ${project.version} - test-jar - - - - org.apache.beam - beam-sdks-common-runner-api - ${project.version} - - - - org.apache.beam - beam-sdks-java-core - ${project.version} - - - - org.apache.beam - beam-sdks-java-extensions-join-library - ${project.version} - - - - org.apache.beam - beam-sdks-java-extensions-protobuf - ${project.version} - - - - org.apache.beam - beam-sdks-java-extensions-google-cloud-platform-core - ${project.version} - - - - org.apache.beam - beam-sdks-java-extensions-google-cloud-platform-core - tests - ${project.version} - - - - org.apache.beam - beam-sdks-java-extensions-sorter - ${project.version} - - - - org.apache.beam - beam-sdks-java-harness - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-common - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-common - tests - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-elasticsearch - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-google-cloud-platform - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-google-cloud-platform - tests - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-hadoop-common - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-hadoop-file-system - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-hbase - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-jdbc - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-jms - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-kafka - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-kinesis - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-mongodb - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-mqtt - ${project.version} - - - - org.apache.beam - beam-sdks-java-io-hadoop-input-format - ${project.version} - - - - org.apache.beam - beam-runners-core-construction-java - ${project.version} - - - - org.apache.beam - beam-runners-core-java - ${project.version} - - - - org.apache.beam - beam-runners-direct-java - ${project.version} - - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - ${project.version} - - - - org.apache.beam - beam-runners-spark - ${project.version} - - - - org.apache.beam - beam-runners-flink_2.10 - ${project.version} - - - - org.apache.beam - beam-runners-apex - ${project.version} - - - - org.apache.beam - beam-examples-java - ${project.version} - - - - org.apache.beam - beam-examples-java8 - ${project.version} - - - - org.apache.commons - commons-compress - ${apache.commons.compress.version} - - - - org.apache.commons - commons-lang3 - ${apache.commons.lang.version} - - - - io.grpc - grpc-all - ${grpc.version} - - - com.google.protobuf - protobuf-lite - - - - - - io.grpc - grpc-auth - ${grpc.version} - - - - io.grpc - grpc-core - ${grpc.version} - - - - io.grpc - grpc-protobuf - ${grpc.version} - - - - io.grpc - grpc-protobuf-lite - ${grpc.version} - - - com.google.protobuf - protobuf-lite - - - - - - io.grpc - grpc-netty - ${grpc.version} - - - - io.grpc - grpc-stub - ${grpc.version} - - - - com.google.api - api-common - ${google-api-common.version} - - - - com.google.api-client - google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.api-client - google-api-client-jackson2 - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.api-client - google-api-client-java6 - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.oauth-client - google-oauth-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.oauth-client - google-oauth-client-java6 - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.http-client - google-http-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.http-client - google-http-client-jackson - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - org.codehaus.jackson - jackson-core-asl - - - - - - com.google.http-client - google-http-client-jackson2 - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.http-client - google-http-client-protobuf - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.auth - google-auth-library-credentials - ${google-auth.version} - - - - com.google.auth - google-auth-library-oauth2-http - ${google-auth.version} - - - - com.google.guava - guava-jdk5 - - - - - - com.google.apis - google-api-services-bigquery - ${bigquery.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.apis - google-api-services-cloudresourcemanager - ${cloudresourcemanager.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.apis - google-api-services-pubsub - ${pubsub.version} - - - com.google.guava - guava-jdk5 - - - - - - com.google.api.grpc - grpc-google-pubsub-v1 - ${pubsubgrpc.version} - - - - com.google.guava - guava-jdk5 - - - - io.grpc - grpc-all - - - - - - com.google.apis - google-api-services-storage - ${storage.version} - - - - com.google.guava - guava-jdk5 - - - - - - com.google.cloud.bigtable - bigtable-protos - ${bigtable.version} - - - - com.google.cloud.bigtable - bigtable-client-core - ${bigtable.version} - - - - io.netty - netty-tcnative-boringssl-static - ${netty.tcnative.version} - runtime - - - - com.google.cloud.datastore - datastore-v1-proto-client - ${datastore.client.version} - - - - com.google.cloud.datastore - datastore-v1-protos - ${datastore.proto.version} - - - - com.google.auto.service - auto-service - ${google-auto-service.version} - - - - com.google.auto.value - auto-value - ${google-auto-value.version} - - - - com.google.code.findbugs - jsr305 - ${findbugs.version} - - - - com.google.cloud.bigdataoss - gcsio - ${google-cloud-bigdataoss.version} - - - - com.google.cloud - google-cloud-core - ${google-cloud-core.version} - - - - com.google.cloud - google-cloud-spanner - ${spanner.version} - - - - com.google.cloud.bigdataoss - util - ${google-cloud-bigdataoss.version} - - - - com.google.apis - google-api-services-dataflow - ${dataflow.version} - - - - com.google.guava - guava-jdk5 - - - - - - com.google.apis - google-api-services-clouddebugger - ${clouddebugger.version} - - - - com.google.guava - guava-jdk5 - - - - - - com.google.protobuf - protobuf-java - ${protobuf.version} - - - com.google.api.grpc - proto-google-common-protos - ${grpc-google-common-protos.version} - - - - com.google.api.grpc - proto-google-cloud-spanner-admin-database-v1 - ${grpc-google-common-protos.version} - - - com.google.api.grpc - grpc-google-common-protos - ${grpc-google-common-protos.version} - - - com.google.protobuf - protobuf-lite - - - - - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-annotations - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - - - com.fasterxml.jackson.dataformat - jackson-dataformat-yaml - ${jackson.version} - - - com.fasterxml.jackson.module - jackson-module-scala_2.10 - ${jackson.version} - - - - io.netty - netty-handler - ${netty.version} - - - - io.netty - netty-transport-native-epoll - ${netty.version} - linux-x86_64 - - - - org.apache.avro - avro - ${avro.version} - - - - com.google.guava - guava - ${guava.version} - - - - com.google.errorprone - error_prone_annotations - 2.0.15 - - - - joda-time - joda-time - ${joda.version} - - - - org.slf4j - slf4j-api - ${slf4j.version} - - - - net.bytebuddy - byte-buddy - 1.6.8 - - - - org.springframework - spring-expression - ${spring.version} - - - - org.xerial.snappy - snappy-java - ${snappy-java.version} - - - - - - org.apache.beam - beam-sdks-java-core - ${project.version} - tests - test - - - - org.apache.beam - beam-runners-core-java - ${project.version} - tests - test - - - - org.hamcrest - hamcrest-all - ${hamcrest.version} - test - - - - org.hamcrest - hamcrest-core - ${hamcrest.version} - test - - - - junit - junit - ${junit.version} - test - - - - org.slf4j - slf4j-jdk14 - ${slf4j.version} - test - - - - - com.google.guava - guava-testlib - ${guava.version} - test - - - - org.mockito - mockito-all - ${mockito.version} - test - - - - com.google.cloud.dataflow - google-cloud-dataflow-java-proto-library-all - ${google-cloud-dataflow-java-proto-library-all.version} - test - - - - - - - - - kr.motd.maven - os-maven-plugin - ${os-maven-plugin.version} - - - + 4.0.0 + + + org.apache + apache + 18 + + + org.apache.beam + beam-parent + Apache Beam :: Parent + Apache Beam provides a simple, Java-based interface + for processing virtually any size data. This artifact includes the parent POM + for other Beam artifacts. + http://beam.apache.org/ + 2016 + + 2.1.0-SNAPSHOT + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + scm:git:https://git-wip-us.apache.org/repos/asf/beam.git + scm:git:https://git-wip-us.apache.org/repos/asf/beam.git + https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary + HEAD + + + + jira + https://issues.apache.org/jira/browse/BEAM + + + + + Beam Dev + dev-subscribe@beam.apache.org + dev-unsubscribe@beam.apache.org + dev@beam.apache.org + http://www.mail-archive.com/dev%beam.apache.org + + + Beam User + user-subscribe@beam.apache.org + user-unsubscribe@beam.apache.org + user@beam.apache.org + http://www.mail-archive.com/user%beam.apache.org + + + Beam Commits + commits-subscribe@beam.apache.org + commits-unsubscribe@beam.apache.org + commits@beam.apache.org + http://www.mail-archive.com/commits%beam.apache.org + + + + + + The Apache Beam Team + dev@beam.apache.org + http://beam.apache.org + Apache Software Foundation + http://www.apache.org + + + + + UTF-8 + + + + true + + + + + + 3.5 + 1.9 + 2.24.0 + 1.0.0-rc2 + 1.8.1 + v2-rev295-1.22.0 + 0.9.6.2 + v1-rev6-1.22.0 + 0.1.0 + v2-rev8-1.22.0 + v1b3-rev196-1.22.0 + 0.5.160222 + 1.4.0 + 1.3.0 + 1.0.0-rc2 + 1.0-rc2 + 1.4.1 + 0.6.1 + 1.22.0 + 1.4.5 + 1.0.2 + 0.5.160304 + 20.0 + 1.2.0 + 0.1.9 + 1.3 + 2.8.8 + 3.0.1 + 2.4 + 4.12 + 1.9.5 + 4.1.8.Final + 1.1.33.Fork26 + 3.2.0 + v1-rev10-1.22.0 + 1.7.14 + 0.16.0-beta + 1.6.2 + 4.3.5.RELEASE + 3.1.4 + v1-rev71-1.22.0 + 4.4.1 + 4.3.5.RELEASE + 1.1.4-M3 + + 1.5.0.Final + 2.0 + 2.20 + 2.20 + 3.6.1 + 1.4.0 + 3.0.2 + 3.0.2 + 3.0.0 + + -Werror + -Xpkginfo:always + nothing + + + pom + + + + sdks/java/build-tools + sdks + runners + examples + + sdks/java/javadoc + + + + + + release + - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.5.0 - - - - org.apache.maven.plugins - maven-antrun-plugin - 1.8 - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.17 - - - com.puppycrawl.tools - checkstyle - 6.19 - - - org.apache.beam - beam-sdks-java-build-tools - ${project.version} - - - - beam/checkstyle.xml - beam/suppressions.xml - true - true - false - true - - - - - test-compile - - check - - - - - - - maven-compiler-plugin - ${maven-compiler-plugin.version} - - 1.7 - 1.7 - - -Xlint:all - ${compiler.error.flag} - - -Xlint:-options - - -Xlint:-cast - -Xlint:-deprecation - -Xlint:-processing - -Xlint:-rawtypes - -Xlint:-serial - -Xlint:-try - -Xlint:-unchecked - -Xlint:-varargs - - - - - true - - false - - - - - default-compile - - compile - - compile - - - ${compiler.default.pkginfo.flag} - - - ${compiler.default.exclude} - - - - - - - - org.apache.maven.plugins - maven-deploy-plugin - 2.8.2 - - - - org.apache.maven.plugins - maven-jar-plugin - ${maven-jar-plugin.version} - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - - - true - - - - - org.apache.maven.plugins - maven-source-plugin - 3.0.1 - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.4 - - ${beam.javadoc_opts} - Apache Beam SDK for Java, version ${project.version} API - Apache Beam SDK for Java, version ${project.version} - false - true - - - - - org.apache.rat - apache-rat-plugin - 0.12 - - ${project.build.directory}/${project.build.finalName}.rat - false - true - true - - - **/target/**/* - **/dependency-reduced-pom.xml - **/hs_err_pid*.log - .github/**/* - **/*.iml - **/.idea/**/* - **/*.egg-info/**/* - **/package-list - **/user.avsc - **/test/resources/**/*.txt - **/test/**/.placeholder - .repository/**/* - **/nose-*.egg/**/* - **/.eggs/**/* - **/.tox/**/* - - - **/.checkstyle - **/.classpath - **/.factorypath - **/.project - **/.settings/**/* - - - **/sdks/python/apache_beam/runners/api/*.py - - - - - - org.codehaus.mojo - versions-maven-plugin - 2.3 - - - - org.codehaus.mojo - exec-maven-plugin - 1.5.0 - - false - - - java.util.logging.config.file - logging.properties - - - - - - - - org.jacoco - jacoco-maven-plugin - 0.7.8 - - - - prepare-agent - - - file - true - - - - report - prepare-package - - report - - - - - - **/AutoValue_*.class - - - - - - - org.eluder.coveralls - coveralls-maven-plugin - 4.3.0 - - - - org.apache.maven.plugins - maven-surefire-plugin - ${surefire-plugin.version} - - false - false - true - ${beamSurefireArgline} - - - - org.apache.maven.surefire - surefire-junit47 - ${surefire-plugin.version} - - - - - - org.apache.maven.plugins - maven-failsafe-plugin - ${failsafe-plugin.version} - - false - true - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.avro - avro-maven-plugin - ${avro.version} - - schema - - - - - false - - - - - - org.apache.maven.plugins - maven-jar-plugin - [2.5,) - - jar - test-jar - - - - - - - - - org.jacoco - jacoco-maven-plugin - [0.7.5,) - - report - prepare-agent - - - - - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${maven-shade-plugin.version} - - - bundle-and-repackage - package - - shade - - - true - - - com.google.guava:guava - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - com.google.common - - - com.google.common.**.testing.* - - - - org.apache.${renderedArtifactId}.repackaged.com.google.common - - - - com.google.thirdparty - - - org.apache.${renderedArtifactId}.repackaged.com.google.thirdparty - - - - - - - - - - - - - org.apache.avro - avro-maven-plugin - ${avro.version} - - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - release - clean install - deploy - true - - - - - - org.apache.maven.plugins - maven-assembly-plugin - 3.0.0 - - - source-release-assembly - - apache-beam-${project.version} - - - - - - - org.codehaus.mojo - findbugs-maven-plugin - 3.0.4 - - - org.apache.beam - beam-sdks-java-build-tools - ${project.version} - - - - - com.google.code.findbugs - findbugs - ${findbugs.version} - - - com.apple - AppleJavaExtensions - - - - - - - beam/findbugs-filter.xml - - - - - test-compile - - check - - - - - - - org.apache.maven.plugins - maven-clean-plugin - 3.0.0 - - - - . - - **/*.pyc - **/*.egg-info/ - **/sdks/python/LICENSE - **/sdks/python/NOTICE - **/sdks/python/README.md - - false - - - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - render-artifact-id - - regex-properties - - prepare-package - - - - renderedArtifactId - [^A-Za-z0-9] - . - ${project.artifactId} - false - - - - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - 3.0.0 - - - analyze-only - - - true - true - - - - + - org.apache.maven.plugins - maven-enforcer-plugin - 1.4.1 - - - enforce - - enforce - - - - - 1.7 - - - jdk.tools:jdk.tools - - - - [1.7,) - - - - [3.2,) - - - - - - enforce-banned-dependencies - - enforce - - - - - - com.google.guava:guava-jdk5 - com.google.protobuf:protobuf-lite - - - - true - - - - - - org.codehaus.mojo - extra-enforcer-rules - 1.0-beta-6 - - + org.apache.maven.plugins + maven-javadoc-plugin + + + attach-javadocs + package + + jar + + + + - org.codehaus.mojo - build-helper-maven-plugin - - - - org.apache.maven.plugins - maven-jar-plugin + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + package + + jar-no-fork + + + + attach-test-sources + package + + test-jar-no-fork + + + + - org.apache.maven.plugins - maven-shade-plugin + org.apache.rat + apache-rat-plugin + + + verify + + check + + + - org.apache.maven.plugins - maven-resources-plugin - 3.0.2 - false - - - python-copy-resources - prepare-package - - copy-resources - - - ${basedir}/sdks/python - - - ${basedir} - - LICENSE - NOTICE - README.md - - - - - - + org.apache.maven.plugins + maven-resources-plugin + ${maven-resources-plugin.version} - - + + - + + org.apache.rat + apache-rat-plugin + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + org.apache.maven.plugins + maven-source-plugin + + + + + + + doclint-java8-disable + + [1.8,) + + + -Xdoclint:all -Xdoclint:-missing + + + + + java7-packageinfo + + 1.7 + + + + -Xpkginfo:legacy + **/package-info.java + + + + + maven-compiler-plugin + + + + compile-package-info + + compile + + compile + + + -Xpkginfo:always + + + **/package-info.java + + + + + + + + + + + eclipse-jdt + + + -Xlint:all + + + + - org.codehaus.mojo - versions-maven-plugin - 2.3 - - - - dependency-updates-report - plugin-updates-report - - - + maven-compiler-plugin + + jdt + + + + org.eclipse.tycho + tycho-compiler-jdt + 0.26.0 + + - - - - - - 3.2 - + + + + + + + + + + org.apache.beam + beam-sdks-common-fn-api + ${project.version} + + + + org.apache.beam + beam-sdks-common-fn-api + ${project.version} + test-jar + + + + org.apache.beam + beam-sdks-common-runner-api + ${project.version} + + + + org.apache.beam + beam-sdks-java-core + ${project.version} + + + + org.apache.beam + beam-sdks-java-extensions-join-library + ${project.version} + + + + org.apache.beam + beam-sdks-java-extensions-protobuf + ${project.version} + + + + org.apache.beam + beam-sdks-java-extensions-google-cloud-platform-core + ${project.version} + + + + org.apache.beam + beam-sdks-java-extensions-google-cloud-platform-core + tests + ${project.version} + + + + org.apache.beam + beam-sdks-java-extensions-sorter + ${project.version} + + + + org.apache.beam + beam-sdks-java-harness + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-common + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-common + tests + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-elasticsearch + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + tests + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-hadoop-common + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-hadoop-file-system + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-hbase + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-jdbc + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-jms + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-kafka + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-kinesis + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-mongodb + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-mqtt + ${project.version} + + + + org.apache.beam + beam-sdks-java-io-hadoop-input-format + ${project.version} + + + + org.apache.beam + beam-runners-core-construction-java + ${project.version} + + + + org.apache.beam + beam-runners-core-java + ${project.version} + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${project.version} + + + + org.apache.beam + beam-runners-spark + ${project.version} + + + + org.apache.beam + beam-runners-flink_2.10 + ${project.version} + + + + org.apache.beam + beam-runners-apex + ${project.version} + + + + org.apache.beam + beam-examples-java + ${project.version} + + + + org.apache.beam + beam-examples-java8 + ${project.version} + + + + org.apache.commons + commons-compress + ${apache.commons.compress.version} + + + + org.apache.commons + commons-lang3 + ${apache.commons.lang.version} + + + + io.grpc + grpc-all + ${grpc.version} + + + com.google.protobuf + protobuf-lite + + + + + + io.grpc + grpc-auth + ${grpc.version} + + + + io.grpc + grpc-core + ${grpc.version} + + + + io.grpc + grpc-protobuf + ${grpc.version} + + + + io.grpc + grpc-protobuf-lite + ${grpc.version} + + + com.google.protobuf + protobuf-lite + + + + + + io.grpc + grpc-netty + ${grpc.version} + + + + io.grpc + grpc-stub + ${grpc.version} + + + + com.google.api + api-common + ${google-api-common.version} + + + + com.google.api-client + google-api-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.api-client + google-api-client-jackson2 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.api-client + google-api-client-java6 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.oauth-client + google-oauth-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.oauth-client + google-oauth-client-java6 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client-jackson + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + org.codehaus.jackson + jackson-core-asl + + + + + + com.google.http-client + google-http-client-jackson2 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client-protobuf + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.auth + google-auth-library-credentials + ${google-auth.version} + + + + com.google.auth + google-auth-library-oauth2-http + ${google-auth.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-bigquery + ${bigquery.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-cloudresourcemanager + ${cloudresourcemanager.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-pubsub + ${pubsub.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.api.grpc + grpc-google-pubsub-v1 + ${pubsubgrpc.version} + + + + com.google.guava + guava-jdk5 + + + + io.grpc + grpc-all + + + + + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + ${grpc-google-common-protos.version} + + + + com.google.api.grpc + proto-google-common-protos + ${grpc-google-common-protos.version} + + + + com.google.apis + google-api-services-storage + ${storage.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.cloud.bigtable + bigtable-protos + ${bigtable.version} + + + + com.google.cloud.bigtable + bigtable-client-core + ${bigtable.version} + + + + io.netty + netty-tcnative-boringssl-static + ${netty.tcnative.version} + runtime + + + + com.google.cloud.datastore + datastore-v1-proto-client + ${datastore.client.version} + + + + com.google.cloud.datastore + datastore-v1-protos + ${datastore.proto.version} + + + + com.google.auto.service + auto-service + ${google-auto-service.version} + + + + com.google.auto.value + auto-value + ${google-auto-value.version} + + + + com.google.code.findbugs + jsr305 + ${findbugs.version} + + + + com.google.cloud.bigdataoss + gcsio + ${google-cloud-bigdataoss.version} + + + + com.google.cloud + google-cloud-core + ${google-cloud-core.version} + + + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + + + com.google.cloud.bigdataoss + util + ${google-cloud-bigdataoss.version} + + + + com.google.apis + google-api-services-dataflow + ${dataflow.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-clouddebugger + ${clouddebugger.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + + com.google.api.grpc + grpc-google-common-protos + ${grpc-google-common-protos.version} + + + com.google.protobuf + protobuf-lite + + + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${jackson.version} + + + com.fasterxml.jackson.module + jackson-module-scala_2.10 + ${jackson.version} + + + + io.netty + netty-handler + ${netty.version} + + + + io.netty + netty-transport-native-epoll + ${netty.version} + linux-x86_64 + + + + org.apache.avro + avro + ${avro.version} + + + + com.google.guava + guava + ${guava.version} + + + + com.google.errorprone + error_prone_annotations + 2.0.15 + + + + joda-time + joda-time + ${joda.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + net.bytebuddy + byte-buddy + 1.6.8 + + + + org.springframework + spring-expression + ${spring.version} + + + + org.xerial.snappy + snappy-java + ${snappy-java.version} + + + + + + org.apache.beam + beam-sdks-java-core + ${project.version} + tests + test + + + + org.apache.beam + beam-runners-core-java + ${project.version} + tests + test + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + test + + + + org.hamcrest + hamcrest-core + ${hamcrest.version} + test + + + + junit + junit + ${junit.version} + test + + + + org.slf4j + slf4j-jdk14 + ${slf4j.version} + test + + + + + com.google.guava + guava-testlib + ${guava.version} + test + + + + org.mockito + mockito-all + ${mockito.version} + test + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-proto-library-all + ${google-cloud-dataflow-java-proto-library-all.version} + test + + + + + + + + + kr.motd.maven + os-maven-plugin + ${os-maven-plugin.version} + + + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.5.0 + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.8 + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + 6.19 + + + org.apache.beam + beam-sdks-java-build-tools + ${project.version} + + + + beam/checkstyle.xml + beam/suppressions.xml + true + true + false + true + + + + + test-compile + + check + + + + + + + maven-compiler-plugin + ${maven-compiler-plugin.version} + + 1.7 + 1.7 + + -Xlint:all + ${compiler.error.flag} + + -Xlint:-options + + -Xlint:-cast + -Xlint:-deprecation + -Xlint:-processing + -Xlint:-rawtypes + -Xlint:-serial + -Xlint:-try + -Xlint:-unchecked + -Xlint:-varargs + + + + + true + + false + + + + + default-compile + + compile + + compile + + + ${compiler.default.pkginfo.flag} + + + ${compiler.default.exclude} + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + 2.8.2 + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + + default-jar + + jar + + + + default-test-jar + + test-jar + + + + + true + + + + + org.apache.maven.plugins + maven-source-plugin + 3.0.1 + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.4 + + ${beam.javadoc_opts} + Apache Beam SDK for Java, version ${project.version} API + Apache Beam SDK for Java, version ${project.version} + false + true + + + + + org.apache.rat + apache-rat-plugin + 0.12 + + ${project.build.directory}/${project.build.finalName}.rat + false + true + true + + + **/target/**/* + **/dependency-reduced-pom.xml + **/hs_err_pid*.log + .github/**/* + **/*.iml + **/.idea/**/* + **/*.egg-info/**/* + **/package-list + **/user.avsc + **/test/resources/**/*.txt + **/test/**/.placeholder + .repository/**/* + **/nose-*.egg/**/* + **/.eggs/**/* + **/.tox/**/* + + + **/.checkstyle + **/.classpath + **/.factorypath + **/.project + **/.settings/**/* + + + **/sdks/python/apache_beam/runners/api/*.py + + + + + + org.codehaus.mojo + versions-maven-plugin + 2.3 + + + + org.codehaus.mojo + exec-maven-plugin + 1.5.0 + + false + + + java.util.logging.config.file + logging.properties + + + + + + + + org.jacoco + jacoco-maven-plugin + 0.7.8 + + + + prepare-agent + + + file + true + + + + report + prepare-package + + report + + + + + + **/AutoValue_*.class + + + + + + + org.eluder.coveralls + coveralls-maven-plugin + 4.3.0 + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + false + false + true + ${beamSurefireArgline} + + + + org.apache.maven.surefire + surefire-junit47 + ${surefire-plugin.version} + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + ${failsafe-plugin.version} + + false + true + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + schema + + + + + false + + + + + + org.apache.maven.plugins + maven-jar-plugin + [2.5,) + + jar + test-jar + + + + + + + + + org.jacoco + jacoco-maven-plugin + [0.7.5,) + + report + prepare-agent + + + + + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${maven-shade-plugin.version} + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + + + com.google.common.**.testing.* + + + + org.apache.${renderedArtifactId}.repackaged.com.google.common + + + + com.google.thirdparty + + + org.apache.${renderedArtifactId}.repackaged.com.google.thirdparty + + + + + + + + + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + + org.apache.maven.plugins + maven-release-plugin + 2.5.3 + + release + clean install + deploy + true + + + + + + org.apache.maven.plugins + maven-assembly-plugin + 3.0.0 + + + source-release-assembly + + apache-beam-${project.version} + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + 3.0.4 + + + org.apache.beam + beam-sdks-java-build-tools + ${project.version} + + + + + com.google.code.findbugs + findbugs + ${findbugs.version} + + + com.apple + AppleJavaExtensions + + + + + + + beam/findbugs-filter.xml + + + + + test-compile + + check + + + + + + + org.apache.maven.plugins + maven-clean-plugin + 3.0.0 + + + + . + + **/*.pyc + **/*.egg-info/ + **/sdks/python/LICENSE + **/sdks/python/NOTICE + **/sdks/python/README.md + + false + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 3.0.0 + + + render-artifact-id + + regex-properties + + prepare-package + + + + renderedArtifactId + [^A-Za-z0-9] + . + ${project.artifactId} + false + + + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.0.0 + + + analyze-only + + + true + true + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce + + enforce + + + + + 1.7 + + + jdk.tools:jdk.tools + + + + [1.7,) + + + + [3.2,) + + + + + + enforce-banned-dependencies + + enforce + + + + + + com.google.guava:guava-jdk5 + com.google.protobuf:protobuf-lite + + + + true + + + + + + org.codehaus.mojo + extra-enforcer-rules + 1.0-beta-6 + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-shade-plugin + + + org.apache.maven.plugins + maven-resources-plugin + 3.0.2 + false + + + python-copy-resources + prepare-package + + copy-resources + + + ${basedir}/sdks/python + + + ${basedir} + + LICENSE + NOTICE + README.md + + + + + + + + + + + + + + org.codehaus.mojo + versions-maven-plugin + 2.3 + + + + dependency-updates-report + plugin-updates-report + + + + + + + + + + 3.2 + diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java index 887db618ea3f..61652e736e90 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java @@ -1,3 +1,20 @@ +/* + * 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 com.google.cloud.ByteArray; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 8444ac8bce96..c5325bb5a85f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -77,19 +77,22 @@ * } * *

    The default size of the batch is set to 1MB, to override this use {@link - * Write#withBatchSize(long)}. Setting batch size to a small value or zero practically disables + * Write#withBatchSizeBytes(long)}. Setting batch size to a small value or zero practically disables * batching. * *

    The transform does not provide same transactional guarantees as Cloud Spanner. In particular, - *

  • Mutations are not submitted atomically; - *
  • A mutation is applied at least once; - *
  • If the pipeline was unexpectedly stopped, mutations that were already applied will not get - * rolled back. + * + *
      + *
    • Mutations are not submitted atomically; + *
    • A mutation is applied at least once; + *
    • If the pipeline was unexpectedly stopped, mutations that were already applied will not get + * rolled back. + *
    */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - private static final long DEFAULT_BATCH_SIZE = 1024 * 1024; // 1 MB + private static final long DEFAULT_BATCH_SIZE_BYTES = 1024 * 1024; // 1 MB /** * Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be @@ -98,7 +101,9 @@ public class SpannerIO { */ @Experimental public static Write write() { - return new AutoValue_SpannerIO_Write.Builder().setBatchSize(DEFAULT_BATCH_SIZE).build(); + return new AutoValue_SpannerIO_Write.Builder() + .setBatchSizeBytes(DEFAULT_BATCH_SIZE_BYTES) + .build(); } /** @@ -119,7 +124,7 @@ public abstract static class Write extends PTransform, PDo @Nullable abstract String getDatabaseId(); - abstract long getBatchSize(); + abstract long getBatchSizeBytes(); @Nullable @VisibleForTesting @@ -136,7 +141,7 @@ abstract static class Builder { abstract Builder setDatabaseId(String databaseId); - abstract Builder setBatchSize(long batchSize); + abstract Builder setBatchSizeBytes(long batchSizeBytes); @VisibleForTesting abstract Builder setServiceFactory(ServiceFactory serviceFactory); @@ -176,8 +181,8 @@ public Write withInstanceId(String instanceId) { * *

    Does not modify this object. */ - public Write withBatchSize(long batchSize) { - return toBuilder().setBatchSize(batchSize).build(); + public Write withBatchSizeBytes(long batchSizeBytes) { + return toBuilder().setBatchSizeBytes(batchSizeBytes).build(); } /** @@ -215,10 +220,18 @@ public PDone expand(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder + .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("Output Project")) .addIfNotNull( DisplayData.item("instanceId", getInstanceId()).withLabel("Output Instance")) .addIfNotNull( - DisplayData.item("databaseId", getDatabaseId()).withLabel("Output Database")); + DisplayData.item("databaseId", getDatabaseId()).withLabel("Output Database")) + .add(DisplayData.item("batchSizeBytes", getBatchSizeBytes()) + .withLabel("Batch Size in Bytes")); + if (getServiceFactory() != null) { + builder.addIfNotNull( + DisplayData.item("serviceFactory", getServiceFactory().getClass().getName()) + .withLabel("Service Factory")); + } } } @@ -231,7 +244,7 @@ static class SpannerWriteFn extends DoFn { private transient DatabaseClient dbClient; // Current batch of mutations to be written. private List mutations; - private long batchSize = 0; + private long batchSizeBytes = 0; private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -251,15 +264,15 @@ public void setup() throws Exception { spanner.getDatabaseClient( DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); mutations = new ArrayList<>(); - batchSize = 0; + batchSizeBytes = 0; } @ProcessElement public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); mutations.add(m); - batchSize += MutationSizeEstimator.sizeOf(m); - if (batchSize >= spec.getBatchSize()) { + batchSizeBytes += MutationSizeEstimator.sizeOf(m); + if (batchSizeBytes >= spec.getBatchSizeBytes()) { flushBatch(); } } @@ -283,6 +296,7 @@ public void teardown() throws Exception { return; } spanner.closeAsync().get(); + spanner = null; } /** @@ -319,15 +333,13 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } LOG.debug("Successfully wrote {} mutations", mutations.size()); mutations = new ArrayList<>(); - batchSize = 0; + batchSizeBytes = 0; } @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("instanceId", spec.getInstanceId()).withLabel("Instance")) - .addIfNotNull(DisplayData.item("databaseId", spec.getDatabaseId()).withLabel("Database")); + spec.populateDisplayData(builder); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java index da7c42a3e47d..03eb28ed943d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -1,3 +1,20 @@ +/* + * 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.hamcrest.Matchers.is; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java index 6fbc5f12c78b..5bdfea5522b2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -1,3 +1,20 @@ +/* + * 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.mockito.Mockito.argThat; @@ -105,7 +122,7 @@ public void batching() throws Exception { .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") - .withBatchSize(1000000000) + .withBatchSizeBytes(1000000000) .withServiceFactory(serviceFactory); SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); @@ -131,7 +148,7 @@ public void batchingGroups() throws Exception { .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") - .withBatchSize(batchSize) + .withBatchSizeBytes(batchSize) .withServiceFactory(serviceFactory); SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); @@ -154,7 +171,7 @@ public void noBatching() throws Exception { .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") - .withBatchSize(0) // turn off batching. + .withBatchSizeBytes(0) // turn off batching. .withServiceFactory(serviceFactory); SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); DoFnTester fnTester = DoFnTester.of(writerFn); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java index 248f64645768..064c65eedcef 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java @@ -1,3 +1,20 @@ +/* + * 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.hamcrest.Matchers.equalTo; @@ -42,25 +59,21 @@ public interface SpannerTestPipelineOptions extends TestPipelineOptions { @Description("Project ID for Spanner") @Default.String("apache-beam-testing") String getProjectId(); - void setProjectId(String value); @Description("Instance ID to write to in Spanner") @Default.String("beam-test") String getInstanceId(); - void setInstanceId(String value); @Description("Database ID to write to in Spanner") @Default.String("beam-testdb") String getDatabaseId(); - void setDatabaseId(String value); @Description("Table name") @Default.String("users") String getTable(); - void setTable(String value); } @@ -76,6 +89,10 @@ public void setUp() throws Exception { spanner = SpannerOptions.newBuilder().setProjectId(options.getProjectId()).build().getService(); databaseAdminClient = spanner.getDatabaseAdminClient(); + + // Delete database if exists. + databaseAdminClient.dropDatabase(options.getInstanceId(), options.getDatabaseId()); + Operation op = databaseAdminClient.createDatabase( options.getInstanceId(),