From 86ad4e2d633d5306fe326a5af27e5ac176695d41 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Fri, 3 Mar 2017 13:19:36 -0800 Subject: [PATCH 01/28] Added very raw basic version of SpannerIO. No unit tests. Must skip checkstyle to compile --- sdks/java/io/google-cloud-platform/pom.xml | 14 +- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 749 ++++++++++++++++++ .../io/gcp/spanner/SpannerMutationCoder.java | 209 +++++ .../io/gcp/spanner/SpannerStructCoder.java | 187 +++++ 4 files changed, 1158 insertions(+), 1 deletion(-) 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/SpannerMutationCoder.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 66a42075e317..baffe0af64b1 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -132,6 +132,18 @@ joda-time + + com.google.cloud + google-cloud-core + 0.9.3-alpha + + + + com.google.cloud + google-cloud-spanner + 0.9.3-beta + + com.google.cloud.bigtable bigtable-protos @@ -224,7 +236,7 @@ org.apache.beam beam-runners-google-cloud-dataflow-java - ${project.version} + test 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..10ad36bc91f7 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -0,0 +1,749 @@ +/* + * Copyright (C) 2017 Google Inc. + * + * Licensed 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.api.client.auth.oauth2.Credential; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SpannerStructCoder; +import org.apache.beam.sdk.coders.SpannerMutationCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.Sink.WriteOperation; +import org.apache.beam.sdk.io.Sink.Writer; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.Write; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.values.PCollection; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Ints; + + +import static com.google.cloud.spanner.TransactionRunner.TransactionCallable; +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.Key; +import com.google.cloud.spanner.KeySet; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.ReadOnlyTransaction; +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.cloud.spanner.Struct; +import com.google.cloud.spanner.TransactionContext; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import com.google.cloud.spanner.SpannerException; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * {@link SpannerIO} provides an API to Read and Write {@link PCollection PCollections} of + * Google Cloud Spanner + * + *

Google Cloud Spanner is a fully managed NoSQL data storage service. + * An {@code Entity} is an object in Spanner, analogous to a row in traditional + * database table. + * + *

This API currently requires an authentication workaround. To use {@link SpannerIO}, users + * must use the {@code gcloud} command line tool to get credentials for Spanner: + *

+ * $ gcloud auth login
+ * 
+ * + *

To read a {@link PCollection} from a query to Spanner, use {@link SpannerIO#source} and + * its methods {@link SpannerIO.Source#withDatabase} and {@link SpannerIO.Source#withQuery} to + * specify the dataset to query and the query to read from. You can optionally provide a namespace + * to query within using {@link SpannerIO.Source#withNamespace} or a Spanner instance using + * {@link SpannerIO.Source#withHost}. + * + *

For example: + * + *

 {@code
+ * // Read a query from Spanner
+ * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
+ * Query query = ...;
+ * String dataset = "...";
+ *
+ * Pipeline p = Pipeline.create(options);
+ * PCollection rows = p.apply(
+ *     Read.from(SpannerIO.source()
+ *         .withDatabaseId(databaseId)
+ *         .withQuery(query));
+ * } 
+ * + *

or: + * + *

 {@code
+ * // Read a query from Spanner using 
+ * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
+ * Query query = ...;
+ * String databaseId = "...";
+ *
+ * Pipeline p = Pipeline.create(options);
+ * PCollection rows = p.apply(SpannerIO.readFrom(databaseId, query));
+ * p.run();
+ * } 
+ * + *

Note: Normally, a Cloud Dataflow job will read from Cloud Spanner in parallel across + * many workers. However, when the {@link Query} is configured with a limit using + * {@link com.google.api.services.datastore.SpannerV1.Query.Builder#setLimit(int)}, then + * all returned results will be read by a single Dataflow worker in order to ensure correct data. + * + *

To write a {@link PCollection} to a Spanner, use {@link SpannerIO#writeTo}, + * specifying the datastore to write to: + * + *

 {@code
+ * PCollection rows = ...;
+ * rows.apply(SpannerIO.writeTo(databaseId));
+ * p.run();
+ * } 
+ * + *

To optionally change the host that is used to write to the Spanner, use {@link + * SpannerIO#sink} to build a {@link SpannerIO.Sink} and write to it using the {@link Write} + * transform: + * + *

 {@code
+ * PCollection entities = ...;
+ * entities.apply(Write.to(SpannerIO.sink().withDatabaseId(databaseId)));
+ * } 
+ * + *

{@link Entity Entities} in the {@code PCollection} to be written must have complete + * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the + * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than the + * project default may be written to by specifying it in the {@code Entity} {@code Keys}. + * + *

{@code
+ * Key.Builder keyBuilder = SpannerHelper.makeKey(...);
+ * keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
+ * }
+ * + *

{@code Entities} will be committed as upsert (update or insert) mutations. Please read + * Entities, Properties, and + * Keys for more information about {@code Entity} keys. + * + *

Permissions

+ * Permission requirements depend on the {@code PipelineRunner} that is used to execute the + * Dataflow job. Please refer to the documentation of corresponding {@code PipelineRunner}s for + * more details. + * + *

Please see Cloud Spanner Sign Up + * for security and permission related information specific to Spanner. + * + * @see org.apache.beam.sdk.runners.PipelineRunner + * + * @deprecated replaced by {@link org.apache.beam.sdk.io.datastore.SpannerIO} + */ + +@Deprecated +@Experimental(Experimental.Kind.SOURCE_SINK) +public class SpannerIO { + /** + * Spanner has a limit of 500 mutations per batch operation, so we flush + * changes to Spanner every 500 entities. + */ + public static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + + /** + * Returns an empty {@link SpannerIO.Source} builder with the default {@code host}. + * Configure the {@code dataset}, {@code query}, and {@code namespace} using + * {@link SpannerIO.Source#withDatabase}, {@link SpannerIO.Source#withQuery}, + * and {@link SpannerIO.Source#withNamespace}. + * + * @deprecated the name and return type do not match. Use {@link #source()}. + */ + @Deprecated + public static Source read() { + return source(); + } + + /** + * Returns an empty {@link SpannerIO.Source} builder with the default {@code host}. + * Configure the {@code dataset}, {@code query}, and {@code namespace} using + * {@link SpannerIO.Source#withDatabase}, {@link SpannerIO.Source#withQuery}, + * and {@link SpannerIO.Source#withNamespace}. + * + *

The resulting {@link Source} object can be passed to {@link Read} to create a + * {@code PTransform} that will read from Spanner. + */ + public static Source source() { + return new Source(null, null, null, null); + } + + /** + * Returns a {@code PTransform} that reads Spanner entities from the query + * against the given dataset. + */ + public static Read.Bounded readFrom(String projectId, String instanceId, String databaseId, String query) { + return Read.from(new Source(projectId, instanceId, databaseId, query)); + } + + /** + * A {@link Source} that reads the result rows of a Spanner query as {@code Entity} objects. + */ + public static class Source extends BoundedSource { + + public String getProjectId() { + return projectId; + } + + public String getInstanceId() { + return instanceId; + } + + public String getDatabaseId() { + return databaseId; + } + + public String getQuery() { + return query; + } + + public Source withDatabaseId(String projectdId, String instanceId, String databaseId) { + //checkNotNull(databaseId, "databaseId"); + return new Source(projectId, instanceId, databaseId, query); + } + + /** + * Returns a new {@link Source} that reads the results of the specified query. + * + *

Does not modify this object. + * + *

Note: Normally, a Cloud Dataflow job will read from Cloud Spanner in parallel + * across many workers. However, when the {@link Query} is configured with a limit using + * {@link com.google.api.services.datastore.SpannerV1.Query.Builder#setLimit(int)}, then all + * returned results will be read by a single Dataflow worker in order to ensure correct data. + */ + public Source withQuery(String query) { + //checkNotNull(query, "query"); + //checkArgument(!query.hasLimit() || query.getLimit() > 0, + // "Invalid query limit %s: must be positive", query.getLimit()); + return new Source(projectId, instanceId, databaseId, query); + } + + @Override + public Coder getDefaultOutputCoder() { + return SpannerStructCoder.of(); + } + + @Override + public List splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options) + throws Exception { + + return ImmutableList.of(this); + } + + @Override + public BoundedReader createReader(PipelineOptions pipelineOptions) throws IOException { + return new SpannerReader(this, getDbClient(DatabaseId.of(projectId, instanceId, databaseId))); + } + + @Override + public void validate() { + //checkNotNull(query, "query"); + //checkNotNull(databaseId, "databaseId"); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 0L; // TODO implement this. + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Input Project")) + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Input Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Input Database")); + if (query != null) { + builder.add(DisplayData.item("query", query) + .withLabel("Query")); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("projectId", projectId) + .add("instanceId", instanceId) + .add("databaseId", databaseId) + .add("query", query) + .toString(); + } + + /////////////////////////////////////////////////////////////////////////////////////////// + + private static final Logger LOG = LoggerFactory.getLogger(Source.class); + private final String projectId; + private final String instanceId; + private final String databaseId; + /** Not really nullable, but it may be {@code null} for in-progress {@code Source}s. */ + @Nullable + private final String query; + + /** For testing only. TODO: This could be much cleaner with dependency injection. */ +/* + @Nullable + private QuerySplitter mockSplitter; + @Nullable + private Long mockEstimateSizeBytes; +*/ + + /** + * + */ + private Source(String projectId, String instanceId, String databaseId, @Nullable String query) { + this.projectId = projectId; + this.instanceId = instanceId; + this.databaseId = databaseId; + this.query = query; + } + + +// /** For testing only. */ +// Source withMockSplitter(QuerySplitter splitter) { +// Source res = new Source(host, databaseId, query, namespace); +// res.mockSplitter = splitter; +// res.mockEstimateSizeBytes = mockEstimateSizeBytes; +// return res; +// } +// +// /** For testing only. */ +// Source withMockEstimateSizeBytes(Long estimateSizeBytes) { +// Source res = new Source(host, databaseId, query, namespace); +// res.mockSplitter = mockSplitter; +// res.mockEstimateSizeBytes = estimateSizeBytes; +// return res; +// } + } + + ///////////////////// Write Class ///////////////////////////////// + + public static Sink sink() { + return new Sink(); + } + + /** + * Returns a new {@link Write} transform that will write to a {@link Sink}. + * + *

For example: {@code p.apply(SpannerIO.writeTo(databaseId));} + */ + //public static Write.Bound writeTo(String projectId, String instanceId, String databaseId) { + // return Write.to(sink().withDatabase(projectId, instanceId, databaseId)); + //} + + /** + * A {@link Sink} that writes a {@link PCollection} containing + * {@link Mutation} instances to a Spanner table. + * + */ + public static class Sink extends org.apache.beam.sdk.io.Sink { + + String projectId; + String instanceId; + String databaseId; + + public Sink() { + } + + public Sink(String projectId, String instanceId, String databaseId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.databaseId = databaseId; + } + + + /** + * Returns a {@link Sink} that is like this one, but will write to the specified database + */ + public Sink withDatabase(String projectId, String instanceId, String databaseId) { + //checkNotNull(databaseId, "databaseId"); + return new Sink(projectId, instanceId, databaseId); + } + + /** + * Ensures the databaseId is set. + */ + @Override + public void validate(PipelineOptions options) { + //checkNotNull( + // databaseId, + // "Database ID is a required parameter. Please use withDatabase to to set the databaseId."); + } + + @Override + public SpannerWriteOperation createWriteOperation(PipelineOptions options) { + return new SpannerWriteOperation(this); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Database")); + } + } + + /** + * A {@link WriteOperation} that will manage a parallel write to a Spanner sink. + */ + private static class SpannerWriteOperation + extends WriteOperation { + + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteOperation.class); + + private final SpannerIO.Sink sink; + + public SpannerWriteOperation(SpannerIO.Sink sink) { + this.sink = sink; + } + + @Override + public Coder getWriterResultCoder() { + return SerializableCoder.of(SpannerWriteResult.class); + } + + @Override + public void initialize(PipelineOptions options) throws Exception {} + + /** + * Finalizes the write. Logs the number of entities written to the Spanner. + */ + @Override + public void finalize(Iterable writerResults, PipelineOptions options) + throws Exception { + long totalRows = 0; + for (SpannerWriteResult result : writerResults) { + totalRows += result.rowsWritten; + } + LOG.info("Wrote {} rows.", totalRows); + } + + @Override + public SpannerWriter createWriter(PipelineOptions options) throws Exception { + return new SpannerWriter(this, getDbClient(DatabaseId.of(sink.projectId, sink.instanceId, sink.databaseId))); + } + + @Override + public SpannerIO.Sink getSink() { + return sink; + } + } + + /** + * {@link Writer} that writes entities to a Spanner Sink. Entities are written in batches, + * where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. Entities + * are committed as upsert mutations (either update if the key already exists, or insert if it is + * a new key). If an entity does not have a complete key (i.e., it has no name or id), the bundle + * will fail. + * + *

See + * Spanner: Entities, Properties, and Keys for information about entity keys and upsert + * mutations. + * + *

Commits are non-transactional. If a commit fails because of a conflict over an entity + * group, the commit will be retried (up to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT} + * times). + * + *

Visible for testing purposes. + */ + static class SpannerWriter extends Writer { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriter.class); + private final SpannerWriteOperation writeOp; + private final DatabaseClient dbClient; + private long totalWritten = 0; + + // Visible for testing. + final List rows = new ArrayList<>(); + + /** + * Since a bundle is written in batches, we should retry the commit of a batch in order to + * prevent transient errors from causing the bundle to fail. + */ + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + + // Visible for testing + SpannerWriter(SpannerWriteOperation writeOp, DatabaseClient dbClient) { + this.writeOp = writeOp; + this.dbClient = dbClient; + } + + @Override + public void open(String uId) throws Exception {} + + /** + * Writes an entity to Spanner. Writes are batched, up to {@link + * SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + */ + @Override + public void write(Mutation row) throws Exception { + + rows.add(row); + + if (rows.size() * row.asMap().size() >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { + flushBatch(); + } + } + + /** + * Flushes any pending batch writes and returns a SpannerWriteResult. + */ + @Override + public SpannerWriteResult close() throws Exception { + if (rows.size() > 0) { + flushBatch(); + } + return new SpannerWriteResult(totalWritten); + } + + @Override + public SpannerWriteOperation getWriteOperation() { + return writeOp; + } + + /** + * Writes a batch of entities to Spanner. + * + *

If a commit fails, it will be retried (up to {@link SpannerWriter#MAX_RETRIES} + * times). All entities in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from the Spanner will be + * thrown. + * + * @throws SpannerException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws SpannerException, IOException, InterruptedException { + LOG.debug("Writing batch of {} rows", rows.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); + + while (true) { + // Batch upsert rows. + try { + dbClient.writeAtLeastOnce(rows); + + // Break if the commit threw no exception. + break; + + } catch (SpannerException 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 the Spanner ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + totalWritten += rows.size(); + LOG.debug("Successfully wrote {} rows", rows.size()); + rows.clear(); + } + } + + private static class SpannerWriteResult implements Serializable { + final long rowsWritten; + + public SpannerWriteResult(long recordsWritten) { + this.rowsWritten = recordsWritten; + } + } + + /** + * A {@link Source.Reader} over the records from a query of the datastore. + * + *

Timestamped records are currently not supported. + * All records implicitly have the timestamp of {@code BoundedWindow.TIMESTAMP_MIN_VALUE}. + */ + public static class SpannerReader extends BoundedSource.BoundedReader { + private final Source source; + + /** + * Database to read from. + */ + private final DatabaseClient dbClient; + + /** + * Current batch of query results. + */ + private ResultSet currentBatch; + + /** + * Maximum number of results to request per query. + * + *

Must be set, or it may result in an I/O error when querying + * Cloud Spanner. + */ + private static final int QUERY_BATCH_LIMIT = Integer.MAX_VALUE; + + /** + * Remaining user-requested limit on the number of sources to return. If the user did not set a + * limit, then this variable will always have the value {@link Integer#MAX_VALUE} and will never + * be decremented. + */ + private int userLimit; + + private volatile boolean done = false; + + private Struct currentRow; + + /** + * Returns a SpannerReader with Source and Spanner object set. + * + * @param datastore a datastore connection to use. + */ + public SpannerReader(Source source, DatabaseClient dbClient) { + this.source = source; + this.dbClient = dbClient; + // If the user set a limit on the query, remember it. Otherwise pin to MAX_VALUE. + //userLimit = source.query.hasLimit() ? source.query.getLimit() : Integer.MAX_VALUE; + } + + @Override + public Struct getCurrent() { + return currentRow; + } + + @Override + public final long getSplitPointsConsumed() { + return done ? 1 : 0; + } + + @Override + public final long getSplitPointsRemaining() { + return done ? 0 : 1; + } + + @Override + public boolean start() throws IOException { + return advance(); + } + + @Override + public boolean advance() throws IOException { + if (currentBatch == null) { + if (source.query != null) { + currentBatch = dbClient.singleUse().executeQuery(Statement.of(source.query)); + } + } + + if (currentBatch != null) { + try { + done = currentBatch.next(); + if (done) { + currentRow = null; + return false; + } + else { + currentRow = currentBatch.getCurrentRowAsStruct(); + return true; + } + } + catch (SpannerException s) { + currentBatch = null; + currentRow = null; + throw new IOException(s); + } + } + return false; + } + + @Override + public void close() throws IOException { + if (currentBatch != null) { + currentBatch.close(); + currentBatch = null; + } + } + + @Override + public SpannerIO.Source getCurrentSource() { + return source; + } + + @Override + public SpannerIO.Source splitAtFraction(double fraction) { + // Not supported. + return null; + } + + @Override + public Double getFractionConsumed() { + // Not supported. + return null; + } + } + + private static DatabaseClient getDbClient(DatabaseId databaseId) throws IOException { + + SpannerOptions options = SpannerOptions.newBuilder().build(); + Spanner spanner = options.getService(); + try { + String clientProject = spanner.getOptions().getProjectId(); + if (!databaseId.getInstanceId().getProject().equals(clientProject)) { + String err = "Invalid project specified. Project in the database id should match" + + "the project name set in the environment variable GCLOUD_PROJECT. Expected: " + + clientProject; + throw new IllegalArgumentException(err); + } + return spanner.getDatabaseClient(databaseId); + } + catch (Exception e) { + throw new IOException(e); + } + finally { + try { + spanner.closeAsync().get(); + } + catch (Exception e) { + throw new IOException(e); + } + } + } + +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java new file mode 100644 index 000000000000..cef5b809eebd --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java @@ -0,0 +1,209 @@ +package org.apache.beam.sdk.coders; + +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Value; +import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.ValueBinder; +import com.google.cloud.spanner.Timestamp; +import com.google.cloud.spanner.Date; + +import com.google.cloud.ByteArray; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.DataOutput; +import java.io.DataInput; +import java.util.Map; +import com.google.common.io.ByteArrayDataOutput; +import com.google.common.io.ByteArrayDataInput; +import com.google.common.io.ByteStreams; + +public class SpannerMutationCoder extends AtomicCoder { + + @JsonCreator + public static SpannerMutationCoder of() { + return INSTANCE; + } + + /***************************/ + + private static final SpannerMutationCoder INSTANCE = new SpannerMutationCoder(); + + private SpannerMutationCoder() {} + + @Override + public void encode(Mutation value, OutputStream outStream, Context context) throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Mutation"); + } + + if (Mutation.Op.DELETE == value.getOperation()) + throw new UnsupportedOperationException("DELETE Mutations not supported!"); + + ByteArrayDataOutput out = ByteStreams.newDataOutput(); + + out.writeUTF(value.getOperation().name()); + out.writeUTF(value.getTable()); + + Map state = value.asMap(); + + // Write number of columns + out.writeInt(state.size()); + + // Write out column names, types and values + ValueSerializer ser = ValueSerializer.of(); + for (String columnName : state.keySet()) { + Value v = state.get(columnName); + out.writeUTF(columnName); + out.writeUTF(v.getType().getCode().name()); + ser.writeTo(out, v); + } + + byte[] buf = out.toByteArray(); + outStream.write(java.nio.ByteBuffer.allocate(4).putInt(buf.length).array()); + outStream.write(buf); + outStream.flush(); + } + + @Override + public Mutation decode(InputStream inStream, Context context) throws IOException { + + byte[] lengthSize = new byte[4]; + inStream.read(lengthSize, 0, 4); + int expectedSize = java.nio.ByteBuffer.wrap(lengthSize).getInt(); + byte[] data = new byte[expectedSize]; + inStream.read(data, 0, expectedSize); + ByteArrayDataInput in = ByteStreams.newDataInput(data); + + Mutation.Op operation = Enum.valueOf(Mutation.Op.class, in.readUTF()); + String tableName = in.readUTF(); + Mutation.WriteBuilder builder = makeBuilder(operation, tableName); + int columnCount = in.readInt(); + ValueDeserializer ser = ValueDeserializer.of(); + // Deserialize column values + for (int i = 0; i < columnCount; i++) { + String columnName = in.readUTF(); + builder = ser.readFrom(in, builder.set(columnName)); + } + + return builder.build(); + } + + private Mutation.WriteBuilder makeBuilder(Mutation.Op operation, String tableName) { + switch(operation) { + case INSERT: + return Mutation.newInsertBuilder(tableName); + case UPDATE: + return Mutation.newUpdateBuilder(tableName); + case REPLACE: + return Mutation.newReplaceBuilder(tableName); + case INSERT_OR_UPDATE: + return Mutation.newInsertOrUpdateBuilder(tableName); + } + throw new UnsupportedOperationException("Cannot determinate mutation operation or operation unsupported."); + } + + @Override + protected long getEncodedElementByteSize(Mutation value, Context context) + throws Exception { + //return value.getSerializedSize(); + return 0L; //TODO: Implement this. + } + + /** + * {@inheritDoc} + * + * @throws NonDeterministicException always. + * A Datastore kind can hold arbitrary {@link Object} instances, which + * makes the encoding non-deterministic. + */ + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Datastore encodings can hold arbitrary Object instances"); + } + + + static class ValueSerializer { + + private static final ValueSerializer INSTANCE = new ValueSerializer(); + + public static ValueSerializer of() { + return INSTANCE; + } + + public void writeTo(DataOutput out, Value v) throws IOException { + Type.Code c = v.getType().getCode(); + switch (c) { + case BOOL: + out.writeBoolean(v.getBool()); + break; + case INT64: + out.writeLong(v.getInt64()); + break; + case FLOAT64: + out.writeDouble(v.getFloat64()); + break; + case STRING: + out.writeUTF(v.getString()); + break; + case BYTES: + byte[] b = v.getBytes().toByteArray(); + out.writeInt(b.length); + out.write(b); + break; + case TIMESTAMP: + out.writeUTF(v.getTimestamp().toString()); + break; + case DATE: + out.writeUTF(v.getDate().toString()); + break; + case ARRAY: + throw new UnsupportedOperationException("ARRAY type not implemented yet."); + case STRUCT: + throw new UnsupportedOperationException("STRUCT type not implemented yet."); + } + } + } + + static class ValueDeserializer { + + private static final ValueDeserializer INSTANCE = new ValueDeserializer(); + + public static ValueDeserializer of() { + return INSTANCE; + } + + public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder vb) throws IOException { + Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); + switch (c) { + case BOOL: + return vb.to(in.readBoolean()); + case INT64: + return vb.to(in.readLong()); + case FLOAT64: + return vb.to(in.readDouble()); + case STRING: + return vb.to(in.readUTF()); + case BYTES: + int size = in.readInt(); + byte[] buf = new byte[size]; + in.readFully(buf); + return vb.to(ByteArray.copyFrom(buf)); + case TIMESTAMP: + return vb.to(Timestamp.parseTimestamp(in.readUTF())); + case DATE: + return vb.to(Date.parseDate(in.readUTF())); + case ARRAY: + throw new UnsupportedOperationException("ARRAY type not implemented yet."); + case STRUCT: + throw new UnsupportedOperationException("STRUCT type not implemented yet."); + } + throw new UnsupportedOperationException("Cannot determine type from input stream or type unsupported."); + } + } + +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java new file mode 100644 index 000000000000..36c76b0e317c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java @@ -0,0 +1,187 @@ +package org.apache.beam.sdk.coders; + +import com.google.cloud.spanner.Struct; +import com.google.cloud.spanner.Value; +import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.ValueBinder; +import com.google.cloud.spanner.Timestamp; +import com.google.cloud.spanner.Date; + +import com.google.cloud.ByteArray; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.DataOutput; +import java.io.DataInput; +import java.util.List; +import com.google.common.io.ByteArrayDataOutput; +import com.google.common.io.ByteArrayDataInput; +import com.google.common.io.ByteStreams; + +public class SpannerStructCoder extends AtomicCoder { + + @JsonCreator + public static SpannerStructCoder of() { + return INSTANCE; + } + + /***************************/ + + private static final SpannerStructCoder INSTANCE = new SpannerStructCoder(); + + private SpannerStructCoder() {} + + @Override + public void encode(Struct value, OutputStream outStream, Context context) throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Struct"); + } + + ByteArrayDataOutput out = ByteStreams.newDataOutput(); + + List fields = value.getType().getStructFields(); + + // Write number of columns + out.writeInt(value.getColumnCount()); + + // Write out column names, types and values + ValueSerializer ser = ValueSerializer.of(); + + for (Type.StructField f : fields) { + out.writeUTF(f.getName()); + out.writeUTF(f.getType().getCode().name()); + ser.writeTo(out, value, f.getName(), f.getType().getCode()); + } + + byte[] buf = out.toByteArray(); + out.write(java.nio.ByteBuffer.allocate(4).putInt(buf.length).array()); + outStream.write(buf); + outStream.flush(); + } + + @Override + public Struct decode(InputStream inStream, Context context) throws IOException { + + byte[] lengthSize = new byte[4]; + inStream.read(lengthSize, 0, 4); + int expectedSize = java.nio.ByteBuffer.wrap(lengthSize).getInt(); + byte[] data = new byte[expectedSize]; + inStream.read(data, 0, expectedSize); + ByteArrayDataInput in = ByteStreams.newDataInput(data); + + Struct.Builder builder = Struct.newBuilder(); + + int columnCount = in.readInt(); + ValueDeserializer ser = ValueDeserializer.of(); + // Deserialize column values + for (int i = 0; i < columnCount; i++) { + String columnName = in.readUTF(); + builder = ser.readFrom(in, builder.set(columnName)); + } + + return builder.build(); + } + + @Override + protected long getEncodedElementByteSize(Struct value, Context context) + throws Exception { + //return value.getSerializedSize(); + return 0L; //TODO: Implement this. + } + + /** + * {@inheritDoc} + * + * @throws NonDeterministicException always. + * A Datastore kind can hold arbitrary {@link Object} instances, which + * makes the encoding non-deterministic. + */ + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Datastore encodings can hold arbitrary Object instances"); + } + + + static class ValueSerializer { + + private static final ValueSerializer INSTANCE = new ValueSerializer(); + + public static ValueSerializer of() { + return INSTANCE; + } + + public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) throws IOException { + switch (c) { + case BOOL: + out.writeBoolean(v.getBoolean(colName)); + break; + case INT64: + out.writeLong(v.getLong(colName)); + break; + case FLOAT64: + out.writeDouble(v.getDouble(colName)); + break; + case STRING: + out.writeUTF(v.getString(colName)); + break; + case BYTES: + byte[] b = v.getBytes(colName).toByteArray(); + out.writeInt(b.length); + out.write(b); + break; + case TIMESTAMP: + out.writeUTF(v.getTimestamp(colName).toString()); + break; + case DATE: + out.writeUTF(v.getDate(colName).toString()); + break; + case ARRAY: + throw new UnsupportedOperationException("ARRAY type not implemented yet."); + case STRUCT: + throw new UnsupportedOperationException("STRUCT type not implemented yet."); + } + } + } + + static class ValueDeserializer { + + private static final ValueDeserializer INSTANCE = new ValueDeserializer(); + + public static ValueDeserializer of() { + return INSTANCE; + } + + public Struct.Builder readFrom(DataInput in, ValueBinder vb) throws IOException { + Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); + switch (c) { + case BOOL: + return vb.to(in.readBoolean()); + case INT64: + return vb.to(in.readLong()); + case FLOAT64: + return vb.to(in.readDouble()); + case STRING: + return vb.to(in.readUTF()); + case BYTES: + int size = in.readInt(); + byte[] buf = new byte[size]; + in.readFully(buf); + return vb.to(ByteArray.copyFrom(buf)); + case TIMESTAMP: + return vb.to(Timestamp.parseTimestamp(in.readUTF())); + case DATE: + return vb.to(Date.parseDate(in.readUTF())); + case ARRAY: + throw new UnsupportedOperationException("ARRAY type not implemented yet."); + case STRUCT: + throw new UnsupportedOperationException("STRUCT type not implemented yet."); + } + throw new UnsupportedOperationException("Cannot determine type from input stream or type unsupported."); + } + } + +} From 564e71405ece4d016a16a3501d26b4346aa64602 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Sun, 5 Mar 2017 22:11:36 -0800 Subject: [PATCH 02/28] Simplified SpannerIO getting compilation error on OutputT --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 740 ++++-------------- 1 file changed, 141 insertions(+), 599 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 10ad36bc91f7..a299dae8458e 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 @@ -1,304 +1,125 @@ -/* - * Copyright (C) 2017 Google Inc. - * - * Licensed 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.api.client.auth.oauth2.Credential; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; - -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SpannerStructCoder; -import org.apache.beam.sdk.coders.SpannerMutationCoder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.io.Sink.WriteOperation; -import org.apache.beam.sdk.io.Sink.Writer; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.io.Write; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.values.PCollection; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; -import com.google.common.primitives.Ints; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Verify.verify; -import static com.google.cloud.spanner.TransactionRunner.TransactionCallable; -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.Key; -import com.google.cloud.spanner.KeySet; import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Operation; -import com.google.cloud.spanner.ReadOnlyTransaction; -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.cloud.spanner.Struct; -import com.google.cloud.spanner.TransactionContext; -import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; import com.google.cloud.spanner.SpannerException; -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Values; +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.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +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; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; + -/** - * {@link SpannerIO} provides an API to Read and Write {@link PCollection PCollections} of - * Google Cloud Spanner - * - *

Google Cloud Spanner is a fully managed NoSQL data storage service. - * An {@code Entity} is an object in Spanner, analogous to a row in traditional - * database table. - * - *

This API currently requires an authentication workaround. To use {@link SpannerIO}, users - * must use the {@code gcloud} command line tool to get credentials for Spanner: - *

- * $ gcloud auth login
- * 
- * - *

To read a {@link PCollection} from a query to Spanner, use {@link SpannerIO#source} and - * its methods {@link SpannerIO.Source#withDatabase} and {@link SpannerIO.Source#withQuery} to - * specify the dataset to query and the query to read from. You can optionally provide a namespace - * to query within using {@link SpannerIO.Source#withNamespace} or a Spanner instance using - * {@link SpannerIO.Source#withHost}. - * - *

For example: - * - *

 {@code
- * // Read a query from Spanner
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String dataset = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection rows = p.apply(
- *     Read.from(SpannerIO.source()
- *         .withDatabaseId(databaseId)
- *         .withQuery(query));
- * } 
- * - *

or: - * - *

 {@code
- * // Read a query from Spanner using 
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String databaseId = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection rows = p.apply(SpannerIO.readFrom(databaseId, query));
- * p.run();
- * } 
- * - *

Note: Normally, a Cloud Dataflow job will read from Cloud Spanner in parallel across - * many workers. However, when the {@link Query} is configured with a limit using - * {@link com.google.api.services.datastore.SpannerV1.Query.Builder#setLimit(int)}, then - * all returned results will be read by a single Dataflow worker in order to ensure correct data. - * - *

To write a {@link PCollection} to a Spanner, use {@link SpannerIO#writeTo}, - * specifying the datastore to write to: - * - *

 {@code
- * PCollection rows = ...;
- * rows.apply(SpannerIO.writeTo(databaseId));
- * p.run();
- * } 
- * - *

To optionally change the host that is used to write to the Spanner, use {@link - * SpannerIO#sink} to build a {@link SpannerIO.Sink} and write to it using the {@link Write} - * transform: - * - *

 {@code
- * PCollection entities = ...;
- * entities.apply(Write.to(SpannerIO.sink().withDatabaseId(databaseId)));
- * } 
- * - *

{@link Entity Entities} in the {@code PCollection} to be written must have complete - * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the - * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than the - * project default may be written to by specifying it in the {@code Entity} {@code Keys}. - * - *

{@code
- * Key.Builder keyBuilder = SpannerHelper.makeKey(...);
- * keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
- * }
- * - *

{@code Entities} will be committed as upsert (update or insert) mutations. Please read - * Entities, Properties, and - * Keys for more information about {@code Entity} keys. - * - *

Permissions

- * Permission requirements depend on the {@code PipelineRunner} that is used to execute the - * Dataflow job. Please refer to the documentation of corresponding {@code PipelineRunner}s for - * more details. - * - *

Please see Cloud Spanner Sign Up - * for security and permission related information specific to Spanner. - * - * @see org.apache.beam.sdk.runners.PipelineRunner - * - * @deprecated replaced by {@link org.apache.beam.sdk.io.datastore.SpannerIO} - */ - -@Deprecated @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - /** - * Spanner has a limit of 500 mutations per batch operation, so we flush - * changes to Spanner every 500 entities. - */ + + @VisibleForTesting public static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; - /** - * Returns an empty {@link SpannerIO.Source} builder with the default {@code host}. - * Configure the {@code dataset}, {@code query}, and {@code namespace} using - * {@link SpannerIO.Source#withDatabase}, {@link SpannerIO.Source#withQuery}, - * and {@link SpannerIO.Source#withNamespace}. - * - * @deprecated the name and return type do not match. Use {@link #source()}. - */ - @Deprecated - public static Source read() { - return source(); + public Writer writeTo(String projectId, String instanceId, String databaseId) { + return new Writer(projectId, instanceId, databaseId); } /** - * Returns an empty {@link SpannerIO.Source} builder with the default {@code host}. - * Configure the {@code dataset}, {@code query}, and {@code namespace} using - * {@link SpannerIO.Source#withDatabase}, {@link SpannerIO.Source#withQuery}, - * and {@link SpannerIO.Source#withNamespace}. + * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner * - *

The resulting {@link Source} object can be passed to {@link Read} to create a - * {@code PTransform} that will read from Spanner. + * @see SpannerIO */ - public static Source source() { - return new Source(null, null, null, null); - } - - /** - * Returns a {@code PTransform} that reads Spanner entities from the query - * against the given dataset. - */ - public static Read.Bounded readFrom(String projectId, String instanceId, String databaseId, String query) { - return Read.from(new Source(projectId, instanceId, databaseId, query)); - } - - /** - * A {@link Source} that reads the result rows of a Spanner query as {@code Entity} objects. - */ - public static class Source extends BoundedSource { - - public String getProjectId() { - return projectId; - } - - public String getInstanceId() { - return instanceId; - } - - public String getDatabaseId() { - return databaseId; - } - - public String getQuery() { - return query; - } - - public Source withDatabaseId(String projectdId, String instanceId, String databaseId) { - //checkNotNull(databaseId, "databaseId"); - return new Source(projectId, instanceId, databaseId, query); + public static class Writer extends MutationTransform { + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + Writer(String projectId, String instanceId, String databaseId) { + super(projectId, instanceId, databaseId); } /** - * Returns a new {@link Source} that reads the results of the specified query. - * - *

Does not modify this object. - * - *

Note: Normally, a Cloud Dataflow job will read from Cloud Spanner in parallel - * across many workers. However, when the {@link Query} is configured with a limit using - * {@link com.google.api.services.datastore.SpannerV1.Query.Builder#setLimit(int)}, then all - * returned results will be read by a single Dataflow worker in order to ensure correct data. + * Returns a new {@link Write} that writes to the Cloud Spanner for the specified location. */ - public Source withQuery(String query) { - //checkNotNull(query, "query"); - //checkArgument(!query.hasLimit() || query.getLimit() > 0, - // "Invalid query limit %s: must be positive", query.getLimit()); - return new Source(projectId, instanceId, databaseId, query); + public Writer withLocation(String projectId, String instanceId, String databaseId) { + checkNotNull(projectId, "projectId"); + checkNotNull(instanceId, "instanceId"); + checkNotNull(databaseId, "databaseId"); + return new Writer(projectId, instanceId, databaseId); } - @Override - public Coder getDefaultOutputCoder() { - return SpannerStructCoder.of(); - } + } - @Override - public List splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options) - throws Exception { - return ImmutableList.of(this); - } + /** + * A {@link PTransform} that writes mutations to Cloud Spanner + * + * Note: Only idempotent Cloud Spanner mutation operations (upsert, etc.) should + * be used by the {@code DoFn} provided, as the commits are retried when failures occur. + */ + private abstract static class MutationTransform extends PTransform, PDone> { + private final String projectId; + private final String instanceId; + private final String databaseId; - @Override - public BoundedReader createReader(PipelineOptions pipelineOptions) throws IOException { - return new SpannerReader(this, getDbClient(DatabaseId.of(projectId, instanceId, databaseId))); + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + MutationTransform(String projectId, String instanceId, String databaseId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.databaseId = databaseId; } @Override - public void validate() { - //checkNotNull(query, "query"); - //checkNotNull(databaseId, "databaseId"); - } + public PDone expand(PCollection input) { + input.apply("Write Mutation to Spanner", ParDo.of( + new SpannerWriterFn(projectId, instanceId, databaseId))); - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - return 0L; // TODO implement this. + return PDone.in(input.getPipeline()); } @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Input Project")) - .addIfNotNull(DisplayData.item("instanceId", instanceId) - .withLabel("Input Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) - .withLabel("Input Database")); - if (query != null) { - builder.add(DisplayData.item("query", query) - .withLabel("Query")); - } + public void validate(PCollection input) { + checkNotNull(projectId, "projectId"); + checkNotNull(instanceId, "instanceId"); + checkNotNull(databaseId, "databaseId"); } @Override @@ -307,272 +128,115 @@ public String toString() { .add("projectId", projectId) .add("instanceId", instanceId) .add("databaseId", databaseId) - .add("query", query) .toString(); } - /////////////////////////////////////////////////////////////////////////////////////////// - - private static final Logger LOG = LoggerFactory.getLogger(Source.class); - private final String projectId; - private final String instanceId; - private final String databaseId; - /** Not really nullable, but it may be {@code null} for in-progress {@code Source}s. */ - @Nullable - private final String query; - - /** For testing only. TODO: This could be much cleaner with dependency injection. */ -/* - @Nullable - private QuerySplitter mockSplitter; - @Nullable - private Long mockEstimateSizeBytes; -*/ - - /** - * - */ - private Source(String projectId, String instanceId, String databaseId, @Nullable String query) { - this.projectId = projectId; - this.instanceId = instanceId; - this.databaseId = databaseId; - this.query = query; - } - - -// /** For testing only. */ -// Source withMockSplitter(QuerySplitter splitter) { -// Source res = new Source(host, databaseId, query, namespace); -// res.mockSplitter = splitter; -// res.mockEstimateSizeBytes = mockEstimateSizeBytes; -// return res; -// } -// -// /** For testing only. */ -// Source withMockEstimateSizeBytes(Long estimateSizeBytes) { -// Source res = new Source(host, databaseId, query, namespace); -// res.mockSplitter = mockSplitter; -// res.mockEstimateSizeBytes = estimateSizeBytes; -// return res; -// } - } - - ///////////////////// Write Class ///////////////////////////////// - - public static Sink sink() { - return new Sink(); - } - - /** - * Returns a new {@link Write} transform that will write to a {@link Sink}. - * - *

For example: {@code p.apply(SpannerIO.writeTo(databaseId));} - */ - //public static Write.Bound writeTo(String projectId, String instanceId, String databaseId) { - // return Write.to(sink().withDatabase(projectId, instanceId, databaseId)); - //} - - /** - * A {@link Sink} that writes a {@link PCollection} containing - * {@link Mutation} instances to a Spanner table. - * - */ - public static class Sink extends org.apache.beam.sdk.io.Sink { - - String projectId; - String instanceId; - String databaseId; - - public Sink() { - } - - public Sink(String projectId, String instanceId, String databaseId) { - this.projectId = projectId; - this.instanceId = instanceId; - this.databaseId = databaseId; - } - - - /** - * Returns a {@link Sink} that is like this one, but will write to the specified database - */ - public Sink withDatabase(String projectId, String instanceId, String databaseId) { - //checkNotNull(databaseId, "databaseId"); - return new Sink(projectId, instanceId, databaseId); - } - - /** - * Ensures the databaseId is set. - */ - @Override - public void validate(PipelineOptions options) { - //checkNotNull( - // databaseId, - // "Database ID is a required parameter. Please use withDatabase to to set the databaseId."); - } - - @Override - public SpannerWriteOperation createWriteOperation(PipelineOptions options) { - return new SpannerWriteOperation(this); - } - @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Database")); - } - } - - /** - * A {@link WriteOperation} that will manage a parallel write to a Spanner sink. - */ - private static class SpannerWriteOperation - extends WriteOperation { - - private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteOperation.class); - - private final SpannerIO.Sink sink; - - public SpannerWriteOperation(SpannerIO.Sink sink) { - this.sink = sink; - } - - @Override - public Coder getWriterResultCoder() { - return SerializableCoder.of(SpannerWriteResult.class); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")) + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Instance")) + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Database")); } - @Override - public void initialize(PipelineOptions options) throws Exception {} - - /** - * Finalizes the write. Logs the number of entities written to the Spanner. - */ - @Override - public void finalize(Iterable writerResults, PipelineOptions options) - throws Exception { - long totalRows = 0; - for (SpannerWriteResult result : writerResults) { - totalRows += result.rowsWritten; - } - LOG.info("Wrote {} rows.", totalRows); + public String getProjectId() { + return projectId; } - @Override - public SpannerWriter createWriter(PipelineOptions options) throws Exception { - return new SpannerWriter(this, getDbClient(DatabaseId.of(sink.projectId, sink.instanceId, sink.databaseId))); + public String getInstanceId() { + return instanceId; } - @Override - public SpannerIO.Sink getSink() { - return sink; + public String getDatabaseId() { + return databaseId; } } + /** - * {@link Writer} that writes entities to a Spanner Sink. Entities are written in batches, - * where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. Entities - * are committed as upsert mutations (either update if the key already exists, or insert if it is - * a new key). If an entity does not have a complete key (i.e., it has no name or id), the bundle - * will fail. + * {@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 - * Spanner: Entities, Properties, and Keys for information about entity keys and upsert - * mutations. + * href="https://cloud.google.com/spanner"> * - *

Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT} - * times). - * - *

Visible for testing purposes. + *

Commits are non-transactional. If a commit fails, it will be retried (up to + * {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. times). This means that the + * mutation operation should be idempotent. */ - static class SpannerWriter extends Writer { - private static final Logger LOG = LoggerFactory.getLogger(SpannerWriter.class); - private final SpannerWriteOperation writeOp; - private final DatabaseClient dbClient; - private long totalWritten = 0; - - // Visible for testing. - final List rows = new ArrayList<>(); + @VisibleForTesting + static class SpannerWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private final String projectId; + private final String instanceId; + private final String databaseId; + private transient DatabaseClient dbClient; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = FluentBackoff.DEFAULT .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); - // Visible for testing - SpannerWriter(SpannerWriteOperation writeOp, DatabaseClient dbClient) { - this.writeOp = writeOp; - this.dbClient = dbClient; + @VisibleForTesting + SpannerWriterFn(String projectId, String instanceId, String databaseId) { + this.projectId = checkNotNull(projectId, "projectId"); + this.instanceId = checkNotNull(instanceId, "instanceId"); + this.databaseId = checkNotNull(databaseId, "databaseId"); } - @Override - public void open(String uId) throws Exception {} - - /** - * Writes an entity to Spanner. Writes are batched, up to {@link - * SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. - */ - @Override - public void write(Mutation row) throws Exception { - - rows.add(row); + @StartBundle + public void startBundle(Context c) throws IOException { + dbClient = getDbClient(DatabaseId.of(projectId, instanceId, databaseId)); + } - if (rows.size() * row.asMap().size() >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + mutations.add(c.element()); + if (mutations.size() >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { flushBatch(); } } - /** - * Flushes any pending batch writes and returns a SpannerWriteResult. - */ - @Override - public SpannerWriteResult close() throws Exception { - if (rows.size() > 0) { + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (!mutations.isEmpty()) { flushBatch(); } - return new SpannerWriteResult(totalWritten); - } - - @Override - public SpannerWriteOperation getWriteOperation() { - return writeOp; } /** - * Writes a batch of entities to Spanner. + * Writes a batch of mutations to Cloud Spanner. * - *

If a commit fails, it will be retried (up to {@link SpannerWriter#MAX_RETRIES} - * times). All entities in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Spanner will be + *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. All + * mutations in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from Cloud Spanner will be * thrown. * * @throws SpannerException if the commit fails or IOException or InterruptedException if * backing off between retries fails. */ private void flushBatch() throws SpannerException, IOException, InterruptedException { - LOG.debug("Writing batch of {} rows", rows.size()); + LOG.debug("Writing batch of {} mutations", mutations.size()); Sleeper sleeper = Sleeper.DEFAULT; BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); while (true) { - // Batch upsert rows. + // Batch upsert entities. try { - dbClient.writeAtLeastOnce(rows); + dbClient.writeAtLeastOnce(mutations); // Break if the commit threw no exception. break; - } catch (SpannerException 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 the Spanner ({}): {}", exception.getCode(), + LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), exception.getMessage()); if (!BackOffUtils.next(sleeper, backoff)) { LOG.error("Aborting after {} retries.", MAX_RETRIES); @@ -580,142 +244,20 @@ private void flushBatch() throws SpannerException, IOException, InterruptedExcep } } } - totalWritten += rows.size(); - LOG.debug("Successfully wrote {} rows", rows.size()); - rows.clear(); - } - } - - private static class SpannerWriteResult implements Serializable { - final long rowsWritten; - - public SpannerWriteResult(long recordsWritten) { - this.rowsWritten = recordsWritten; - } - } - - /** - * A {@link Source.Reader} over the records from a query of the datastore. - * - *

Timestamped records are currently not supported. - * All records implicitly have the timestamp of {@code BoundedWindow.TIMESTAMP_MIN_VALUE}. - */ - public static class SpannerReader extends BoundedSource.BoundedReader { - private final Source source; - - /** - * Database to read from. - */ - private final DatabaseClient dbClient; - - /** - * Current batch of query results. - */ - private ResultSet currentBatch; - - /** - * Maximum number of results to request per query. - * - *

Must be set, or it may result in an I/O error when querying - * Cloud Spanner. - */ - private static final int QUERY_BATCH_LIMIT = Integer.MAX_VALUE; - - /** - * Remaining user-requested limit on the number of sources to return. If the user did not set a - * limit, then this variable will always have the value {@link Integer#MAX_VALUE} and will never - * be decremented. - */ - private int userLimit; - - private volatile boolean done = false; - - private Struct currentRow; - - /** - * Returns a SpannerReader with Source and Spanner object set. - * - * @param datastore a datastore connection to use. - */ - public SpannerReader(Source source, DatabaseClient dbClient) { - this.source = source; - this.dbClient = dbClient; - // If the user set a limit on the query, remember it. Otherwise pin to MAX_VALUE. - //userLimit = source.query.hasLimit() ? source.query.getLimit() : Integer.MAX_VALUE; - } - - @Override - public Struct getCurrent() { - return currentRow; - } - - @Override - public final long getSplitPointsConsumed() { - return done ? 1 : 0; - } - - @Override - public final long getSplitPointsRemaining() { - return done ? 0 : 1; - } - - @Override - public boolean start() throws IOException { - return advance(); - } - - @Override - public boolean advance() throws IOException { - if (currentBatch == null) { - if (source.query != null) { - currentBatch = dbClient.singleUse().executeQuery(Statement.of(source.query)); - } - } - - if (currentBatch != null) { - try { - done = currentBatch.next(); - if (done) { - currentRow = null; - return false; - } - else { - currentRow = currentBatch.getCurrentRowAsStruct(); - return true; - } - } - catch (SpannerException s) { - currentBatch = null; - currentRow = null; - throw new IOException(s); - } - } - return false; - } - - @Override - public void close() throws IOException { - if (currentBatch != null) { - currentBatch.close(); - currentBatch = null; - } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); } @Override - public SpannerIO.Source getCurrentSource() { - return source; - } - - @Override - public SpannerIO.Source splitAtFraction(double fraction) { - // Not supported. - return null; - } - - @Override - public Double getFractionConsumed() { - // Not supported. - return null; + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")) + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Output Database")); } } From 49beb06d6f5e88b90614d9bfa40d5302b6c88557 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Sun, 19 Mar 2017 17:11:16 -0700 Subject: [PATCH 03/28] Compiliation fixed and basic Sink functionality working. --- sdks/java/core/pom.xml | 12 +++++ .../apache/beam/sdk/coders/CoderRegistry.java | 5 ++ .../sdk/coders}/SpannerMutationCoder.java | 28 +++++++---- .../beam/sdk/coders}/SpannerStructCoder.java | 4 +- sdks/java/io/google-cloud-platform/pom.xml | 6 --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 48 +++++++++++-------- 6 files changed, 64 insertions(+), 39 deletions(-) rename sdks/java/{io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner => core/src/main/java/org/apache/beam/sdk/coders}/SpannerMutationCoder.java (85%) rename sdks/java/{io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner => core/src/main/java/org/apache/beam/sdk/coders}/SpannerStructCoder.java (97%) diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 4f8955096325..717a7a6e6eef 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -257,6 +257,18 @@ google-api-services-bigquery + + com.google.cloud + google-cloud-core + 0.9.3-alpha + + + + com.google.cloud + google-cloud-spanner + 0.9.3-beta + + com.google.apis google-api-services-cloudresourcemanager diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 65f4209acfb1..5e1a762a1b80 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Struct; import com.google.api.services.bigquery.model.TableRow; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; @@ -105,6 +107,9 @@ public void registerStandardCoders() { registerCoder(Set.class, SetCoder.class); registerCoder(String.class, StringUtf8Coder.class); registerCoder(TableRow.class, TableRowJsonCoder.class); + registerCoder(Mutation.class, SpannerMutationCoder.class); + registerCoder(Struct.class, SpannerStructCoder.class); + registerCoder(Object.class, VoidCoder.class); registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class); registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java similarity index 85% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java index cef5b809eebd..85be32b4053f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerMutationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java @@ -117,17 +117,17 @@ protected long getEncodedElementByteSize(Mutation value, Context context) * {@inheritDoc} * * @throws NonDeterministicException always. - * A Datastore kind can hold arbitrary {@link Object} instances, which + * A Spanner table can hold arbitrary {@link Object} instances, which * makes the encoding non-deterministic. */ @Override public void verifyDeterministic() throws NonDeterministicException { throw new NonDeterministicException(this, - "Datastore encodings can hold arbitrary Object instances"); + "Spanner encodings can hold arbitrary Object instances"); } - static class ValueSerializer { + static class ValueSerializer implements java.io.Serializable { private static final ValueSerializer INSTANCE = new ValueSerializer(); @@ -136,6 +136,11 @@ public static ValueSerializer of() { } public void writeTo(DataOutput out, Value v) throws IOException { + if (v.isNull()) { + out.writeByte(0); //NULL indicator byte + return; + } + out.writeByte(1); // Not a null value Type.Code c = v.getType().getCode(); switch (c) { case BOOL: @@ -169,7 +174,7 @@ public void writeTo(DataOutput out, Value v) throws IOException { } } - static class ValueDeserializer { + static class ValueDeserializer implements java.io.Serializable { private static final ValueDeserializer INSTANCE = new ValueDeserializer(); @@ -179,24 +184,27 @@ public static ValueDeserializer of() { public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder vb) throws IOException { Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); + byte b = in.readByte(); // NULL indicator switch (c) { case BOOL: - return vb.to(in.readBoolean()); + return b == 1 ? vb.to(in.readBoolean()) : vb.to((Boolean) null); case INT64: - return vb.to(in.readLong()); + return b == 1 ? vb.to(in.readLong()) : vb.to((Long) null); case FLOAT64: - return vb.to(in.readDouble()); + return b == 1 ? vb.to(in.readDouble()) : vb.to((Double) null); case STRING: - return vb.to(in.readUTF()); + return b == 1 ? vb.to(in.readUTF()) : vb.to((String) null); case BYTES: + if (b == 0) + return vb.to((ByteArray) null); int size = in.readInt(); byte[] buf = new byte[size]; in.readFully(buf); return vb.to(ByteArray.copyFrom(buf)); case TIMESTAMP: - return vb.to(Timestamp.parseTimestamp(in.readUTF())); + return b == 1 ? vb.to(Timestamp.parseTimestamp(in.readUTF())) : vb.to((Timestamp) null); case DATE: - return vb.to(Date.parseDate(in.readUTF())); + return b == 1 ? vb.to(Date.parseDate(in.readUTF())) : vb.to((Date) null); case ARRAY: throw new UnsupportedOperationException("ARRAY type not implemented yet."); case STRUCT: diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java similarity index 97% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java index 36c76b0e317c..2025cc271dd5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerStructCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java @@ -106,7 +106,7 @@ public void verifyDeterministic() throws NonDeterministicException { } - static class ValueSerializer { + static class ValueSerializer implements java.io.Serializable { private static final ValueSerializer INSTANCE = new ValueSerializer(); @@ -147,7 +147,7 @@ public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) throw } } - static class ValueDeserializer { + static class ValueDeserializer implements java.io.Serializable { private static final ValueDeserializer INSTANCE = new ValueDeserializer(); diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index baffe0af64b1..f0d7f7a48994 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -132,12 +132,6 @@ joda-time - - com.google.cloud - google-cloud-core - 0.9.3-alpha - - com.google.cloud google-cloud-spanner 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 a299dae8458e..cb61bf8a274b 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 @@ -48,6 +48,9 @@ import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SpannerMutationCoder; + @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { @@ -55,7 +58,7 @@ public class SpannerIO { @VisibleForTesting public static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; - public Writer writeTo(String projectId, String instanceId, String databaseId) { + public static Writer writeTo(String projectId, String instanceId, String databaseId) { return new Writer(projectId, instanceId, databaseId); } @@ -139,7 +142,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Output Project")) .addIfNotNull(DisplayData.item("instanceId", instanceId) .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("instanceId", instanceId) + .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Output Database")); } @@ -154,6 +157,7 @@ public String getInstanceId() { public String getDatabaseId() { return databaseId; } + } @@ -169,14 +173,15 @@ public String getDatabaseId() { * mutation operation should be idempotent. */ @VisibleForTesting - static class SpannerWriterFn extends DoFn { + static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private Spanner spanner; private final String projectId; private final String instanceId; private final String databaseId; private transient DatabaseClient dbClient; // Current batch of mutations to be written. - private final List mutations = new ArrayList<>(); + private final List mutations = new ArrayList(); private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -190,9 +195,15 @@ static class SpannerWriterFn extends DoFn { this.databaseId = checkNotNull(databaseId, "databaseId"); } + @Setup + public void setup() throws Exception { + SpannerOptions options = SpannerOptions.newBuilder().build(); + spanner = options.getService(); + } + @StartBundle public void startBundle(Context c) throws IOException { - dbClient = getDbClient(DatabaseId.of(projectId, instanceId, databaseId)); + dbClient = getDbClient(spanner, DatabaseId.of(projectId, instanceId, databaseId)); } @ProcessElement @@ -210,12 +221,18 @@ public void finishBundle(Context c) throws Exception { } } + @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. All - * mutations in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from Cloud Spanner will be + *

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 SpannerException if the commit fails or IOException or InterruptedException if @@ -227,7 +244,7 @@ private void flushBatch() throws SpannerException, IOException, InterruptedExcep BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); while (true) { - // Batch upsert entities. + // Batch upsert rows. try { dbClient.writeAtLeastOnce(mutations); @@ -261,10 +278,8 @@ public void populateDisplayData(Builder builder) { } } - private static DatabaseClient getDbClient(DatabaseId databaseId) throws IOException { + private static DatabaseClient getDbClient(Spanner spanner, DatabaseId databaseId) throws IOException { - SpannerOptions options = SpannerOptions.newBuilder().build(); - Spanner spanner = options.getService(); try { String clientProject = spanner.getOptions().getProjectId(); if (!databaseId.getInstanceId().getProject().equals(clientProject)) { @@ -278,14 +293,5 @@ private static DatabaseClient getDbClient(DatabaseId databaseId) throws IOExcept catch (Exception e) { throw new IOException(e); } - finally { - try { - spanner.closeAsync().get(); - } - catch (Exception e) { - throw new IOException(e); - } - } } - } From 90754bbccd0369b10449e2ab7ac79699608031a9 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Sat, 25 Mar 2017 14:57:51 -0700 Subject: [PATCH 04/28] Fixed mutation batch limit calculation. --- .../main/java/org/apache/beam/sdk/coders/CoderRegistry.java | 2 +- .../java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 5e1a762a1b80..988dbbb76015 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -109,11 +109,11 @@ public void registerStandardCoders() { registerCoder(TableRow.class, TableRowJsonCoder.class); registerCoder(Mutation.class, SpannerMutationCoder.class); registerCoder(Struct.class, SpannerStructCoder.class); - registerCoder(Object.class, VoidCoder.class); registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class); registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); registerCoder(IntervalWindow.class, IntervalWindow.getCoder()); + registerCoder(Object.class, VoidCoder.class); } /** 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 cb61bf8a274b..5a8a7972d12e 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 @@ -208,10 +208,12 @@ public void startBundle(Context c) throws IOException { @ProcessElement public void processElement(ProcessContext c) throws Exception { - mutations.add(c.element()); - if (mutations.size() >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { + Mutation m = c.element(); + int columnCount = m.asMap().size(); + if ((mutations.size() * columnCount) >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { flushBatch(); } + mutations.add(m); } @FinishBundle From caa6b582da0c2b8d52378cbf0da146c59209bfe9 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Sun, 26 Mar 2017 11:44:36 -0700 Subject: [PATCH 05/28] Minor tweak of batch logic. --- .../main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5a8a7972d12e..e35dd7d2cd8b 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 @@ -210,7 +210,7 @@ public void startBundle(Context c) throws IOException { public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); int columnCount = m.asMap().size(); - if ((mutations.size() * columnCount) >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { + if ((mutations.size() + 1) * columnCount >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { flushBatch(); } mutations.add(m); From be81ec73cfc0cc45848c33636cca9f3a20d42fd9 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Mon, 27 Mar 2017 14:24:03 -0700 Subject: [PATCH 06/28] Checkstyle plugin cleanup. Existing unit tests passing. --- .../apache/beam/sdk/coders/CoderRegistry.java | 4 +- .../beam/sdk/coders/SpannerMutationCoder.java | 53 ++++++++------- .../beam/sdk/coders/SpannerStructCoder.java | 43 ++++++------ .../beam/sdk/io/gcp/spanner/SpannerIO.java | 68 +++++++------------ .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 2 + 5 files changed, 83 insertions(+), 87 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 988dbbb76015..de4661c74c70 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -19,9 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Struct; -import com.google.api.services.bigquery.model.TableRow; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -113,7 +113,7 @@ public void registerStandardCoders() { registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); registerCoder(IntervalWindow.class, IntervalWindow.getCoder()); - registerCoder(Object.class, VoidCoder.class); + //registerCoder(Object.class, VoidCoder.class); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java index 85be32b4053f..9dbb5fade440 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java @@ -1,26 +1,26 @@ package org.apache.beam.sdk.coders; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.cloud.ByteArray; +import com.google.cloud.spanner.Date; import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Value; +import com.google.cloud.spanner.Timestamp; import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.Value; import com.google.cloud.spanner.ValueBinder; -import com.google.cloud.spanner.Timestamp; -import com.google.cloud.spanner.Date; - -import com.google.cloud.ByteArray; - -import com.fasterxml.jackson.annotation.JsonCreator; - +import com.google.common.io.ByteArrayDataInput; +import com.google.common.io.ByteArrayDataOutput; +import com.google.common.io.ByteStreams; +import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.io.DataOutput; -import java.io.DataInput; import java.util.Map; -import com.google.common.io.ByteArrayDataOutput; -import com.google.common.io.ByteArrayDataInput; -import com.google.common.io.ByteStreams; +/** + * A {@link Coder} that encodes Spanner {@link Mutation} objects. + */ public class SpannerMutationCoder extends AtomicCoder { @JsonCreator @@ -35,13 +35,16 @@ public static SpannerMutationCoder of() { private SpannerMutationCoder() {} @Override - public void encode(Mutation value, OutputStream outStream, Context context) throws IOException, CoderException { + public void encode(Mutation value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { throw new CoderException("cannot encode a null Mutation"); } - if (Mutation.Op.DELETE == value.getOperation()) + if (Mutation.Op.DELETE == value.getOperation()) { throw new UnsupportedOperationException("DELETE Mutations not supported!"); + } ByteArrayDataOutput out = ByteStreams.newDataOutput(); @@ -52,7 +55,7 @@ public void encode(Mutation value, OutputStream outStream, Context context) thro // Write number of columns out.writeInt(state.size()); - + // Write out column names, types and values ValueSerializer ser = ValueSerializer.of(); for (String columnName : state.keySet()) { @@ -103,12 +106,12 @@ private Mutation.WriteBuilder makeBuilder(Mutation.Op operation, String tableNam case INSERT_OR_UPDATE: return Mutation.newInsertOrUpdateBuilder(tableName); } - throw new UnsupportedOperationException("Cannot determinate mutation operation or operation unsupported."); + throw new UnsupportedOperationException("Mutation operation unsupported."); } @Override protected long getEncodedElementByteSize(Mutation value, Context context) - throws Exception { + throws Exception { //return value.getSerializedSize(); return 0L; //TODO: Implement this. } @@ -173,7 +176,7 @@ public void writeTo(DataOutput out, Value v) throws IOException { } } } - + static class ValueDeserializer implements java.io.Serializable { private static final ValueDeserializer INSTANCE = new ValueDeserializer(); @@ -182,7 +185,9 @@ public static ValueDeserializer of() { return INSTANCE; } - public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder vb) throws IOException { + public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder vb) + throws IOException { + Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); byte b = in.readByte(); // NULL indicator switch (c) { @@ -195,14 +200,16 @@ public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder { @JsonCreator @@ -35,7 +34,8 @@ public static SpannerStructCoder of() { private SpannerStructCoder() {} @Override - public void encode(Struct value, OutputStream outStream, Context context) throws IOException, CoderException { + public void encode(Struct value, OutputStream outStream, Context context) + throws IOException, CoderException { if (value == null) { throw new CoderException("cannot encode a null Struct"); } @@ -46,10 +46,10 @@ public void encode(Struct value, OutputStream outStream, Context context) throws // Write number of columns out.writeInt(value.getColumnCount()); - + // Write out column names, types and values ValueSerializer ser = ValueSerializer.of(); - + for (Type.StructField f : fields) { out.writeUTF(f.getName()); out.writeUTF(f.getType().getCode().name()); @@ -114,7 +114,8 @@ public static ValueSerializer of() { return INSTANCE; } - public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) throws IOException { + public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) + throws IOException { switch (c) { case BOOL: out.writeBoolean(v.getBoolean(colName)); @@ -146,7 +147,7 @@ public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) throw } } } - + static class ValueDeserializer implements java.io.Serializable { private static final ValueDeserializer INSTANCE = new ValueDeserializer(); @@ -155,7 +156,9 @@ public static ValueDeserializer of() { return INSTANCE; } - public Struct.Builder readFrom(DataInput in, ValueBinder vb) throws IOException { + public Struct.Builder readFrom(DataInput in, ValueBinder vb) + throws IOException { + Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); switch (c) { case BOOL: @@ -180,7 +183,7 @@ public Struct.Builder readFrom(DataInput in, ValueBinder vb) th case STRUCT: throw new UnsupportedOperationException("STRUCT type not implemented yet."); } - throw new UnsupportedOperationException("Cannot determine type from input stream or type unsupported."); + throw new UnsupportedOperationException("Type unsupported."); } } 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 e35dd7d2cd8b..0120de9bb8fc 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 @@ -1,57 +1,41 @@ package org.apache.beam.sdk.io.gcp.spanner; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; - -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Verify.verify; +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.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.SpannerException; - +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import java.util.NoSuchElementException; -import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Values; 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.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; 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; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SpannerMutationCoder; - +/** + * {@link Spanner} provides an API for reading from and writing to + * Google Cloud Spanner over different + * versions of the Cloud Spanner Client libraries. + * + */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { @@ -63,7 +47,7 @@ public static Writer writeTo(String projectId, String instanceId, String databas } /** - * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner + * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. * * @see SpannerIO */ @@ -165,15 +149,14 @@ public String getDatabaseId() { * {@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 + *

See * - *

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

Commits are non-transactional. If a commit fails, it will be retried (up to + * {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. times). This means that the + * mutation operation should be idempotent. */ @VisibleForTesting - static class SpannerWriterFn extends DoFn { + static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); private Spanner spanner; private final String projectId; @@ -225,8 +208,9 @@ public void finishBundle(Context c) throws Exception { @Teardown public void teardown() throws Exception { - if (spanner == null) + if (spanner == null) { return; + } spanner.closeAsync().get(); } @@ -272,15 +256,16 @@ public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")) + .withLabel("Project")) .addIfNotNull(DisplayData.item("instanceId", instanceId) - .withLabel("Output Instance")) + .withLabel("Instance")) .addIfNotNull(DisplayData.item("databaseId", databaseId) - .withLabel("Output Database")); + .withLabel("Database")); } } - private static DatabaseClient getDbClient(Spanner spanner, DatabaseId databaseId) throws IOException { + private static DatabaseClient getDbClient(Spanner spanner, DatabaseId databaseId) + throws IOException { try { String clientProject = spanner.getOptions().getProjectId(); @@ -291,8 +276,7 @@ private static DatabaseClient getDbClient(Spanner spanner, DatabaseId databaseId throw new IllegalArgumentException(err); } return spanner.getDatabaseClient(databaseId); - } - catch (Exception e) { + } catch (Exception e) { throw new IOException(e); } } 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 717c6d3caf4b..2bb387c49674 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 @@ -54,7 +54,9 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.api.services.bigquery.model"), classesInPackage("com.google.auth"), classesInPackage("com.google.bigtable.v2"), + classesInPackage("com.google.cloud"), classesInPackage("com.google.cloud.bigtable.config"), + classesInPackage("com.google.cloud.spanner"), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class), // https://github.com/GoogleCloudPlatform/cloud-bigtable-client/pull/1056 From 69b0eb76951e8d2b452203c38715e30f5c60378e Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 6 Apr 2017 09:56:07 -0700 Subject: [PATCH 07/28] Code review changes 4/6/17 --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 32 +++++++++++++++---- 1 file changed, 26 insertions(+), 6 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 0120de9bb8fc..871f11d35dbd 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 @@ -31,17 +31,37 @@ /** - * {@link Spanner} provides an API for reading from and writing to - * Google Cloud Spanner over different - * versions of the Cloud Spanner Client libraries. + * {@link SpannerIO} provides an API for reading from and writing to + * Google Cloud Spanner. * */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { + private SpannerIO() { } + @VisibleForTesting public static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; +/* + *

To write a {@link PCollection} to Spanner, use {@link SpannerIO#writeTo} to + * specify the database location for output. + * + *

For example: + * + *

 {@code
+ * // Write data to a table.
+ * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
+ * options.getInput = 
+ *
+ * Pipeline p = Pipeline.create(options);
+ *   p.apply(TextIO.Read.from(options.getInput()))
+ *       .apply(ParDo.of(new ParseLineFn(tableInfo)))
+ *       .apply(SpannerIO.writeTo(options.getProjectId(),
+ *           options.getInstanceId(), options.getDatabaseId()));
+ *
+ * } 
+ */ public static Writer writeTo(String projectId, String instanceId, String databaseId) { return new Writer(projectId, instanceId, databaseId); } @@ -96,7 +116,7 @@ private abstract static class MutationTransform extends PTransform input) { - input.apply("Write Mutation to Spanner", ParDo.of( + input.apply("Write mutations to Spanner", ParDo.of( new SpannerWriterFn(projectId, instanceId, databaseId))); return PDone.in(input.getPipeline()); @@ -158,13 +178,13 @@ public String getDatabaseId() { @VisibleForTesting static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); - private Spanner spanner; + private transient Spanner spanner; private final String projectId; private final String instanceId; private final String databaseId; private transient DatabaseClient dbClient; // Current batch of mutations to be written. - private final List mutations = new ArrayList(); + private final List mutations = new ArrayList(); private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = From ba7e7eecd3f1de618bca0a9b9d119003502164f7 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 6 Apr 2017 15:39:06 -0700 Subject: [PATCH 08/28] Try to resolve merge conflict. --- .../main/java/org/apache/beam/sdk/coders/CoderRegistry.java | 1 - .../java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index de4661c74c70..475654b73388 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -113,7 +113,6 @@ public void registerStandardCoders() { registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); registerCoder(IntervalWindow.class, IntervalWindow.getCoder()); - //registerCoder(Object.class, VoidCoder.class); } /** 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 2bb387c49674..cc8097a174ed 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 @@ -54,9 +54,7 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.api.services.bigquery.model"), classesInPackage("com.google.auth"), classesInPackage("com.google.bigtable.v2"), - classesInPackage("com.google.cloud"), classesInPackage("com.google.cloud.bigtable.config"), - classesInPackage("com.google.cloud.spanner"), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class), // https://github.com/GoogleCloudPlatform/cloud-bigtable-client/pull/1056 @@ -73,6 +71,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 f583eaede4618fa49f1dcbbf4ad9d83eceae1b7b Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 6 Apr 2017 22:25:42 -0700 Subject: [PATCH 09/28] Code review related cleanup. --- pom.xml | 2 + sdks/java/core/pom.xml | 8 +- .../beam/sdk/coders/SpannerMutationCoder.java | 9 ++- .../beam/sdk/coders/SpannerStructCoder.java | 9 ++- .../apache/beam/SdkCoreApiSurfaceTest.java | 2 + sdks/java/io/google-cloud-platform/pom.xml | 2 +- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 79 ++++++++----------- 7 files changed, 50 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index eded6840dfd8..08ab30d56dec 100644 --- a/pom.xml +++ b/pom.xml @@ -120,6 +120,8 @@ 0.5.160304 20.0 1.0.1 + 0.11.2-beta + 0.11.2-alpha 1.3 2.7.2 3.0.1 diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 717a7a6e6eef..099741c43efa 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -259,14 +259,14 @@ com.google.cloud - google-cloud-core - 0.9.3-alpha + google-cloud-spanner + ${spanner.version} com.google.cloud - google-cloud-spanner - 0.9.3-beta + google-cloud-core + ${cloud-core.version} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java index 9dbb5fade440..59ba9e24a3d7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java @@ -2,9 +2,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.google.cloud.ByteArray; -import com.google.cloud.spanner.Date; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Timestamp; import com.google.cloud.spanner.Type; import com.google.cloud.spanner.Value; import com.google.cloud.spanner.ValueBinder; @@ -16,6 +16,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.Map; /** @@ -66,7 +67,7 @@ public void encode(Mutation value, OutputStream outStream, Context context) } byte[] buf = out.toByteArray(); - outStream.write(java.nio.ByteBuffer.allocate(4).putInt(buf.length).array()); + outStream.write(ByteBuffer.allocate(4).putInt(buf.length).array()); outStream.write(buf); outStream.flush(); } @@ -76,7 +77,7 @@ public Mutation decode(InputStream inStream, Context context) throws IOException byte[] lengthSize = new byte[4]; inStream.read(lengthSize, 0, 4); - int expectedSize = java.nio.ByteBuffer.wrap(lengthSize).getInt(); + int expectedSize = ByteBuffer.wrap(lengthSize).getInt(); byte[] data = new byte[expectedSize]; inStream.read(data, 0, expectedSize); ByteArrayDataInput in = ByteStreams.newDataInput(data); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java index bd708b572d3e..0d572995aac5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java @@ -2,9 +2,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.google.cloud.ByteArray; -import com.google.cloud.spanner.Date; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; import com.google.cloud.spanner.Struct; -import com.google.cloud.spanner.Timestamp; import com.google.cloud.spanner.Type; import com.google.cloud.spanner.ValueBinder; import com.google.common.io.ByteArrayDataInput; @@ -15,6 +15,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.List; /** @@ -57,7 +58,7 @@ public void encode(Struct value, OutputStream outStream, Context context) } byte[] buf = out.toByteArray(); - out.write(java.nio.ByteBuffer.allocate(4).putInt(buf.length).array()); + out.write(ByteBuffer.allocate(4).putInt(buf.length).array()); outStream.write(buf); outStream.flush(); } @@ -67,7 +68,7 @@ public Struct decode(InputStream inStream, Context context) throws IOException { byte[] lengthSize = new byte[4]; inStream.read(lengthSize, 0, 4); - int expectedSize = java.nio.ByteBuffer.wrap(lengthSize).getInt(); + int expectedSize = ByteBuffer.wrap(lengthSize).getInt(); byte[] data = new byte[expectedSize]; inStream.read(data, 0, expectedSize); ByteArrayDataInput in = ByteStreams.newDataInput(data); diff --git a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java index c4b3a9f551f4..e2320e5f6160 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java @@ -45,6 +45,8 @@ public void testSdkApiSurface() throws Exception { "com.google.api.services.storage", "com.google.auth", "com.google.protobuf", + "com.google.cloud", + "com.google.cloud.spanner", "com.fasterxml.jackson.annotation", "com.fasterxml.jackson.core", "com.fasterxml.jackson.databind", diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index f0d7f7a48994..23a41efde65b 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -135,7 +135,7 @@ com.google.cloud google-cloud-spanner - 0.9.3-beta + ${spanner.version} 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 871f11d35dbd..c4fcff9ce0cb 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 @@ -52,18 +52,20 @@ private SpannerIO() { } *
 {@code
  * // Write data to a table.
  * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * options.getInput = 
+ * options.getInput = ;
+ * PCollection mutations = ...;
  *
  * Pipeline p = Pipeline.create(options);
  *   p.apply(TextIO.Read.from(options.getInput()))
- *       .apply(ParDo.of(new ParseLineFn(tableInfo)))
+ *       .apply(mutations)
  *       .apply(SpannerIO.writeTo(options.getProjectId(),
  *           options.getInstanceId(), options.getDatabaseId()));
+ *   p.run();
  *
  * } 
*/ public static Writer writeTo(String projectId, String instanceId, String databaseId) { - return new Writer(projectId, instanceId, databaseId); + return new Writer(projectId, instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); } /** @@ -71,59 +73,38 @@ public static Writer writeTo(String projectId, String instanceId, String databas * * @see SpannerIO */ - public static class Writer extends MutationTransform { - /** - * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if - * it is {@code null} at instantiation time, an error will be thrown. - */ - Writer(String projectId, String instanceId, String databaseId) { - super(projectId, instanceId, databaseId); - } - - /** - * Returns a new {@link Write} that writes to the Cloud Spanner for the specified location. - */ - public Writer withLocation(String projectId, String instanceId, String databaseId) { - checkNotNull(projectId, "projectId"); - checkNotNull(instanceId, "instanceId"); - checkNotNull(databaseId, "databaseId"); - return new Writer(projectId, instanceId, databaseId); - } + public static class Writer extends PTransform, PDone> { - } - - - /** - * A {@link PTransform} that writes mutations to Cloud Spanner - * - * Note: Only idempotent Cloud Spanner mutation operations (upsert, etc.) should - * be used by the {@code DoFn} provided, as the commits are retried when failures occur. - */ - private abstract static class MutationTransform extends PTransform, PDone> { private final String projectId; private final String instanceId; private final String databaseId; + private int batchSize; - /** - * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if - * it is {@code null} at instantiation time, an error will be thrown. - */ - MutationTransform(String projectId, String instanceId, String databaseId) { + Writer(String projectId, String instanceId, String databaseId, int batchSize) { this.projectId = projectId; this.instanceId = instanceId; this.databaseId = databaseId; + this.batchSize = batchSize; + } + + /** + * Returns a new {@link Write} with a limit on the number of mutations per batch. + * Defaults to SPANNER_MUTATIONS_PER_COMMIT_LIMIT. + */ + public Writer withBatchSize(Integer batchSize) { + return new Writer(projectId, instanceId, databaseId, batchSize); } @Override - public PDone expand(PCollection input) { + public PDone expand(PCollection input) { input.apply("Write mutations to Spanner", ParDo.of( - new SpannerWriterFn(projectId, instanceId, databaseId))); + new SpannerWriterFn(projectId, instanceId, databaseId, batchSize))); return PDone.in(input.getPipeline()); } @Override - public void validate(PCollection input) { + public void validate(PCollection input) { checkNotNull(projectId, "projectId"); checkNotNull(instanceId, "instanceId"); checkNotNull(databaseId, "databaseId"); @@ -162,6 +143,10 @@ public String getDatabaseId() { return databaseId; } + public int getBatchSize() { + return batchSize; + } + } @@ -182,9 +167,10 @@ static class SpannerWriterFn extends DoFn { private final String projectId; 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 final List mutations = new ArrayList<>(); private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -192,31 +178,28 @@ static class SpannerWriterFn extends DoFn { .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(String projectId, String instanceId, String databaseId) { + SpannerWriterFn(String projectId, String instanceId, String databaseId, int batchSize) { this.projectId = checkNotNull(projectId, "projectId"); 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(); - } - - @StartBundle - public void startBundle(Context c) throws IOException { - dbClient = getDbClient(spanner, DatabaseId.of(projectId, instanceId, databaseId)); + dbClient = getDbClient(spanner, DatabaseId.of(projectId, 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 >= SpannerIO.SPANNER_MUTATIONS_PER_COMMIT_LIMIT) { + if ((mutations.size() + 1) * columnCount >= batchSize) { flushBatch(); } - mutations.add(m); } @FinishBundle From 77cfbb62d1318773b3cc80d3949edb62834b33ba Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 6 Apr 2017 22:36:09 -0700 Subject: [PATCH 10/28] Resolve pom file merge conflict. --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 08ab30d56dec..d1bed973157a 100644 --- a/pom.xml +++ b/pom.xml @@ -120,8 +120,6 @@ 0.5.160304 20.0 1.0.1 - 0.11.2-beta - 0.11.2-alpha 1.3 2.7.2 3.0.1 @@ -142,6 +140,8 @@ -Werror -Xpkginfo:always nothing + 0.11.2-beta + 0.11.2-alpha pom From dc97e1e0e3f5d046bb138c8089d784bfa9979070 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Fri, 7 Apr 2017 07:30:00 -0700 Subject: [PATCH 11/28] More review cleanup, etc. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 64 ++++++------------- 1 file changed, 18 insertions(+), 46 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 c4fcff9ce0cb..353c124a6f72 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 @@ -5,11 +5,11 @@ 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.SpannerException; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; @@ -50,7 +50,10 @@ private SpannerIO() { } *

For example: * *

 {@code
- * // Write data to a table.
+ *
+ * Write data to a table.  Input read from {@link TextIO} will be parsed
+ * and transformed to a {@link PCollection} of {@link Mutation}s.
+ *
  * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
  * options.getInput = ;
  * PCollection mutations = ...;
@@ -58,14 +61,13 @@ private SpannerIO() { }
  * Pipeline p = Pipeline.create(options);
  *   p.apply(TextIO.Read.from(options.getInput()))
  *       .apply(mutations)
- *       .apply(SpannerIO.writeTo(options.getProjectId(),
- *           options.getInstanceId(), options.getDatabaseId()));
+ *       .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
  *   p.run();
  *
  * } 
*/ - public static Writer writeTo(String projectId, String instanceId, String databaseId) { - return new Writer(projectId, instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + public static Writer writeTo(String instanceId, String databaseId) { + return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); } /** @@ -75,13 +77,11 @@ public static Writer writeTo(String projectId, String instanceId, String databas */ public static class Writer extends PTransform, PDone> { - private final String projectId; private final String instanceId; private final String databaseId; private int batchSize; - Writer(String projectId, String instanceId, String databaseId, int batchSize) { - this.projectId = projectId; + Writer(String instanceId, String databaseId, int batchSize) { this.instanceId = instanceId; this.databaseId = databaseId; this.batchSize = batchSize; @@ -89,23 +89,22 @@ public static class Writer extends PTransform, PDone> { /** * Returns a new {@link Write} with a limit on the number of mutations per batch. - * Defaults to SPANNER_MUTATIONS_PER_COMMIT_LIMIT. + * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. */ public Writer withBatchSize(Integer batchSize) { - return new Writer(projectId, instanceId, databaseId, batchSize); + return new Writer(instanceId, databaseId, batchSize); } @Override public PDone expand(PCollection input) { input.apply("Write mutations to Spanner", ParDo.of( - new SpannerWriterFn(projectId, instanceId, databaseId, batchSize))); + new SpannerWriterFn(instanceId, databaseId, batchSize))); return PDone.in(input.getPipeline()); } @Override public void validate(PCollection input) { - checkNotNull(projectId, "projectId"); checkNotNull(instanceId, "instanceId"); checkNotNull(databaseId, "databaseId"); } @@ -113,7 +112,6 @@ public void validate(PCollection input) { @Override public String toString() { return MoreObjects.toStringHelper(getClass()) - .add("projectId", projectId) .add("instanceId", instanceId) .add("databaseId", databaseId) .toString(); @@ -123,18 +121,12 @@ public String toString() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")) .addIfNotNull(DisplayData.item("instanceId", instanceId) .withLabel("Output Instance")) .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Output Database")); } - public String getProjectId() { - return projectId; - } - public String getInstanceId() { return instanceId; } @@ -164,7 +156,6 @@ public int getBatchSize() { static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); private transient Spanner spanner; - private final String projectId; private final String instanceId; private final String databaseId; private final int batchSize; @@ -178,8 +169,7 @@ static class SpannerWriterFn extends DoFn { .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(String projectId, String instanceId, String databaseId, int batchSize) { - this.projectId = checkNotNull(projectId, "projectId"); + SpannerWriterFn(String instanceId, String databaseId, int batchSize) { this.instanceId = checkNotNull(instanceId, "instanceId"); this.databaseId = checkNotNull(databaseId, "databaseId"); this.batchSize = batchSize; @@ -189,7 +179,8 @@ static class SpannerWriterFn extends DoFn { public void setup() throws Exception { SpannerOptions options = SpannerOptions.newBuilder().build(); spanner = options.getService(); - dbClient = getDbClient(spanner, DatabaseId.of(projectId, instanceId, databaseId)); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(options.getProjectId(), instanceId, databaseId)); } @ProcessElement @@ -224,10 +215,10 @@ public void teardown() throws Exception { * If the retry limit is exceeded, the last exception from Cloud Spanner will be * thrown. * - * @throws SpannerException if the commit fails or IOException or InterruptedException if + * @throws AbortedException if the commit fails or IOException or InterruptedException if * backing off between retries fails. */ - private void flushBatch() throws SpannerException, IOException, InterruptedException { + 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(); @@ -239,7 +230,7 @@ private void flushBatch() throws SpannerException, IOException, InterruptedExcep // Break if the commit threw no exception. break; - } catch (SpannerException exception) { + } 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(), @@ -258,29 +249,10 @@ private void flushBatch() throws SpannerException, IOException, InterruptedExcep public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Project")) .addIfNotNull(DisplayData.item("instanceId", instanceId) .withLabel("Instance")) .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Database")); } } - - private static DatabaseClient getDbClient(Spanner spanner, DatabaseId databaseId) - throws IOException { - - try { - String clientProject = spanner.getOptions().getProjectId(); - if (!databaseId.getInstanceId().getProject().equals(clientProject)) { - String err = "Invalid project specified. Project in the database id should match" - + "the project name set in the environment variable GCLOUD_PROJECT. Expected: " - + clientProject; - throw new IllegalArgumentException(err); - } - return spanner.getDatabaseClient(databaseId); - } catch (Exception e) { - throw new IOException(e); - } - } } From 0c7cfd4c727fc56594e8d1a7788d18c84b49b5c6 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Fri, 7 Apr 2017 19:52:57 -0700 Subject: [PATCH 12/28] Added package-info file. --- .../beam/sdk/io/gcp/spanner/package-info.java | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) 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/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..12c1f1299527 --- /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; From 0f176fc432a62c2eb6b40225c7a3f2c6a9b749c6 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Tue, 18 Apr 2017 20:50:12 -0700 Subject: [PATCH 13/28] Added spanner csv loader example. --- .../beam/examples/spanner-csv-loader/pom.xml | 457 ++++++++++++++++++ .../dtss/desa/tools/SpannerCSVLoader.java | 212 ++++++++ .../desa/tools/SpannerMutationBuilder.java | 179 +++++++ 3 files changed, 848 insertions(+) create mode 100644 examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml create mode 100644 examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java create mode 100644 examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java diff --git a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml new file mode 100644 index 000000000000..3d402a8c9fa4 --- /dev/null +++ b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml @@ -0,0 +1,457 @@ + + + + 4.0.0 + + + + com.disney.dtss.desa.tools + SpannerCSVLoader + Spanner migration tool + A tool for the parallel transfer of data from text files + to Spanner. + http://cloud.google.com/dataflow + + 0.0.1-SNAPSHOT + + jar + + + 2.0.0 + ${settings.localRepository}/org/mortbay/jetty/alpn/jetty-alpn-agent/${jetty.alpnAgent.version}/jetty-alpn-agent-${jetty.alpnAgent.version}.jar + + + + + java8 + + [1.8,) + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-java8-main-source + initialize + + add-source + + + + ${project.basedir}/src/main/java8 + + + + + + add-java8-test-source + initialize + + add-test-source + + + + ${project.basedir}/src/test/java8 + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + default-testCompile + test-compile + + testCompile + + + 1.7 + 1.7 + + + **/*Java8Test.java + **/game/**/*.java + + + + + + + java8-test-compile + test-compile + + testCompile + + + 1.8 + 1.8 + + + **/*Java8Test.java + **/game/**/*.java + + + + + + + default-compile + compile + + compile + + + 1.7 + 1.7 + + + **/*Java8*.java + **/game/**/*.java + + + + + + + java8-compile + compile + + compile + + + 1.8 + 1.8 + + + **/*Java8*.java + **/game/**/*.java + + + + + + + + + + + + + + maven-compiler-plugin + + + + maven-shade-plugin + 2.4.1 + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-dependency-plugin + 2.10 + + + get-jetty-alpn-agent + validate + + get + + + org.mortbay.jetty.alpn + jetty-alpn-agent + ${jetty.alpnAgent.version} + + + + + + maven-surefire-plugin + 2.19.1 + + -javaagent:${jetty.alpnAgent.path} + + + + + + + + + + + com.google.api.grpc + grpc-google-cloud-spanner-v1 + 0.1.5 + + + io.grpc + grpc-all + + + + + io.netty + netty-tcnative-boringssl-static + 1.1.33.Fork26 + + + + + + org.apache.beam + beam-runners-direct-java + 0.6.0-SNAPSHOT + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + 0.6.0-SNAPSHOT + + + org.apache.beam + beam-sdks-java-core + 0.6.0-SNAPSHOT + + + + + + + com.google.apis + google-api-services-bigquery + v2-rev295-1.22.0 + + + com.google.guava + guava-jdk5 + + + + + + + + com.google.cloud + google-cloud-spanner + 0.11.2-beta + + + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + 0.6.0-SNAPSHOT + + + + org.apache.commons + commons-csv + 1.1 + + + + org.yaml + snakeyaml + 1.11 + + + + + + + + joda-time + joda-time + 2.9.2 + + + + + + org.slf4j + slf4j-api + 1.7.14 + + + + org.slf4j + slf4j-jdk14 + 1.7.14 + runtime + + + + javax.servlet + javax.servlet-api + 3.1.0 + + + + + + + + junit + junit + 4.11 + + + + org.mockito + mockito-all + 1.10.19 + test + + + diff --git a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java new file mode 100644 index 000000000000..e42d3bb77e39 --- /dev/null +++ b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java @@ -0,0 +1,212 @@ +package com.disney.dtss.desa.tools; + +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.SpannerOptions; + +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.coders.SpannerMutationCoder; +import org.apache.beam.sdk.coders.Coder; +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.BigQueryOptions; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; + +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.transforms.Count; +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.values.PCollection; +import org.apache.beam.sdk.util.Transport; + + +import java.io.Serializable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.LinkedHashMap; + +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import org.yaml.snakeyaml.Yaml; + +import javax.annotation.Nullable; +import javax.servlet.http.HttpServletResponse; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Generalized bulk loader for importing BigQuery tables as Datastore kinds. + * + *

To execute this pipeline locally, specify general pipeline configuration: + *

{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * 
+ * and the Datastore info for the output, with the form + *
{@code
+ *   --dataset=DATASET_ID
+ *   --key=Key specification for the kind.   
+ * }
+ * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * 
+ * and the BigQuery table reference for input + *
{@code
+ *   --input=:.
+ * }
+ * + */ +public class SpannerCSVLoader { + + private static final Logger mLog = LoggerFactory.getLogger(SpannerCSVLoader.class); + private static Options options; + + + /** + * Command options specificiation. + */ + private static interface Options extends PipelineOptions { + @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("Schema YAML") + @Validation.Required + String getSchema(); + void setSchema(String value); + } + + + /** + * Constructs and executes the processing pipeline based upon command options. + */ + public static void main(String[] args) throws Exception { + + options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Yaml yaml = new Yaml(); + InputStream in = Files.newInputStream(Paths.get(options.getSchema())); + TableInfo tableInfo = yaml.loadAs(in, TableInfo.class); + + Pipeline p = Pipeline.create(options); + p.apply(TextIO.Read.from(options.getInput())) + .apply(ParDo.of(new ParseLineFn(tableInfo))) + .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); + p.run().waitUntilFinish(); + + } + + + /** + * A DoFn that creates a Spanner Mutation for each BigQuery row. + */ + static class ParseLineFn extends DoFn { + private final TableInfo tableInfo; + private final SpannerMutationBuilder rowBuilder; + + ParseLineFn(TableInfo tableInfo) { + this.tableInfo = tableInfo; + rowBuilder = new SpannerMutationBuilder(tableInfo); + } + + + @ProcessElement + public void processElement(ProcessContext c) { + try { + Mutation e = rowBuilder.build(c.element()); + if (e != null) + c.output(e); + } + catch (Exception e) { + mLog.error(e.getMessage(), e); + } + } + } + + + public static class TableInfo implements Serializable { + + private String tableName; + private List schema; + + public String getTableName() { return tableName; } + public void setTableName(String tableName) { this.tableName = tableName; } + + public List getSchema() { return this.schema; } + public void setSchema(List schema) { this.schema = schema; } + } + + + public static class Schema implements Serializable { + + private String name; + private String type; + private boolean nullable; + private boolean exclude; + private boolean pk; + + + Schema() { + exclude = false; + nullable = true; + } + + public String getName() { return name; } + public void setName(String name) { this.name = name; } + public String getType() { return type; } + public void setType(String type) { this.type = type; } + public boolean isNullable() { return nullable; } + public void setNullable(boolean flag) { nullable = flag; } + public boolean isExclude() { return this.exclude; } + public void setExclude(boolean flag) { this.exclude = flag; } + public boolean isPk() { return pk; } + public void setPk(boolean flag) { pk = flag; } + + + @Override + public int hashCode() { + return name.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Schema other = (Schema) obj; + return name.equals(other.getName()); + } + + } + +} diff --git a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java new file mode 100644 index 000000000000..629b51c06866 --- /dev/null +++ b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java @@ -0,0 +1,179 @@ +package com.disney.dtss.desa.tools; + +import java.lang.Iterable; +import java.io.Serializable; +import java.util.List; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.Date; +import java.util.Calendar; +import java.text.SimpleDateFormat; +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.Arrays; + +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.ValueBinder; +import com.google.cloud.Timestamp; + +import org.apache.commons.csv.CSVParser; +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVRecord; + +import com.disney.dtss.desa.tools.SpannerCSVLoader.TableInfo; +import com.disney.dtss.desa.tools.SpannerCSVLoader.Schema; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SpannerMutationBuilder implements Serializable { + + private static final SimpleDateFormat mRFC3339 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'"); + + private static final Logger mLog = LoggerFactory.getLogger(SpannerMutationBuilder.class); + + private final TableInfo tableInfo; + + public SpannerMutationBuilder(TableInfo tableInfo) { + this.tableInfo = tableInfo; + } + + public Mutation build(String csvData) throws Exception { + + CSVParser parser = CSVParser.parse(csvData, CSVFormat.EXCEL); + List list = parser.getRecords(); + if (list.size() != 1) + throw new IllegalStateException(String.format("CSV Record count = %d, should only be 1", list.size())); + + String[] fields = new String[list.get(0).size()]; + for (int i = 0; i < fields.length; i++) + fields[i] = list.get(0).get(i); + + Mutation.WriteBuilder builder = Mutation.newInsertOrUpdateBuilder(tableInfo.getTableName()); + + int i = -1; + for (Schema column : tableInfo.getSchema()) { + i++; + String name = column.getName(); + String type = column.getType(); + String mode = "N/A"; + if (column.isExclude()) + continue; + convert(builder.set(name), mode, type, fields[i]); + } + + return builder.build(); + } + + + private Date makeDate(Object value) throws Exception { + Date d = null; + String s = (String) value; + if (s != null && ! "".equals(s.trim())) { + if (s.indexOf('.') != -1) + try { + d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS z").parse(s); + } + catch (Exception e) { + d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS").parse(s); + } + else + d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z").parse(s); + //d = new Date(Long.parseLong((String) value) / 1000); + } + return d; + } + + + private Timestamp toTimestamp(Date d) { + return Timestamp.parseTimestamp(mRFC3339.format(d)); + } + + + private void convert(ValueBinder vb, String mode, String type, Object value) throws Exception { + if("REPEATED".equals(mode)) { + if ("STRING".equals(type)) { + vb.toStringArray((Iterable) value); + return; + } + if ("INTEGER".equals(type)) { + List vl = new ArrayList(); + for (String s : (List) value) + if (s == null || "".equals(s.trim())) + vl.add(0L); + else + vl.add(Long.parseLong(s)); + vb.toInt64Array(vl); + return; + } + if ("DOUBLE".equals(type)) { + List vl = new ArrayList(); + for (Double d : (List) value) + vl.add(d); + vb.toFloat64Array(vl); + return; + } + if ("BOOLEAN".equals(type)) { + List vl = new ArrayList(); + for (Boolean b : (List) value) + vl.add(b); + vb.toBoolArray(vl); + return; + } + if ("TIMESTAMP".equals(type)) { + List vl = new ArrayList(); + for (String s : (List) value) + vl.add(toTimestamp(makeDate(s))); + vb.toTimestampArray(vl); + return; + } + } + else { + String s = (String) value; + if (s != null) + s = s.trim(); + if ("".equals(s) || "NULL".equals(s)) + s = null; + + if ("STRING".equals(type)) { + vb.to(s); + return; + } + if ("INTEGER".equals(type)) { + if (s == null) + vb.to((Long) null); + else + vb.to(Long.parseLong(s)); + return; + } + if ("DOUBLE".equals(type)) { + if (s == null) + vb.to((Double) null); + else + vb.to(Double.parseDouble(s)); + return; + } + if ("BOOLEAN".equals(type)) { + if ( s != null && "1".equals(s)) + s = "true"; + if ( s != null && "0".equals(s)) + s = "false"; + if (s == null) + vb.to((Boolean) null); + else + vb.to(Boolean.parseBoolean(s)); + return; + } + if ("TIMESTAMP".equals(type)) { + if (s == null) + vb.to((Timestamp) null); + else + vb.to(toTimestamp(makeDate(s))); + return; + } + } + mLog.error("The BigQuery type " + type + " is not currently supported."); + System.exit(0); + } +} From c6bef41c3301e9a128ebeb613049fbaf16d127f9 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Tue, 18 Apr 2017 21:09:37 -0700 Subject: [PATCH 14/28] Added Spanner CSV loader example --- .../spanner/csvloader}/SpannerCSVLoader.java | 28 ++----------------- .../csvloader}/SpannerMutationBuilder.java | 2 +- 2 files changed, 4 insertions(+), 26 deletions(-) rename examples/{java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools => java/src/main/java/org/apache/beam/examples/spanner/csvloader}/SpannerCSVLoader.java (86%) rename examples/{java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools => java/src/main/java/org/apache/beam/examples/spanner/csvloader}/SpannerMutationBuilder.java (99%) diff --git a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java similarity index 86% rename from examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java rename to examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java index e42d3bb77e39..3c664c7c911d 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerCSVLoader.java +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java @@ -1,3 +1,4 @@ +package org.apache.beam.examples.spanner.csvloader; package com.disney.dtss.desa.tools; import com.google.cloud.spanner.Mutation; @@ -46,30 +47,7 @@ /** - * Generalized bulk loader for importing BigQuery tables as Datastore kinds. - * - *

To execute this pipeline locally, specify general pipeline configuration: - *

{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * 
- * and the Datastore info for the output, with the form - *
{@code
- *   --dataset=DATASET_ID
- *   --key=Key specification for the kind.   
- * }
- * - *

To execute this pipeline using the Dataflow service, specify pipeline configuration: - *

{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * 
- * and the BigQuery table reference for input - *
{@code
- *   --input=:.
- * }
+ * Generalized bulk loader for importing CSV files into Spanner * */ public class SpannerCSVLoader { @@ -125,7 +103,7 @@ public static void main(String[] args) throws Exception { /** - * A DoFn that creates a Spanner Mutation for each BigQuery row. + * A DoFn that creates a Spanner Mutation for each CSV line. */ static class ParseLineFn extends DoFn { private final TableInfo tableInfo; diff --git a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java similarity index 99% rename from examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java rename to examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java index 629b51c06866..647625f0b50e 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/src/main/java/com/disney/dtss/desa/tools/SpannerMutationBuilder.java +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java @@ -1,4 +1,4 @@ -package com.disney.dtss.desa.tools; +package org.apache.beam.examples.spanner.csvloader; import java.lang.Iterable; import java.io.Serializable; From f4118f7f5d4bbe46fb61bcf158fdba5a6d3ff419 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 21 Apr 2017 14:25:01 -0700 Subject: [PATCH 15/28] Code cosmetics and documentation update --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 95 +++++++++---------- 1 file changed, 43 insertions(+), 52 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 353c124a6f72..7bd07fe90c74 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 @@ -1,21 +1,11 @@ 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.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -29,43 +19,56 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; -/** - * {@link SpannerIO} provides an API for reading from and writing to - * Google Cloud Spanner. - * - */ -@Experimental(Experimental.Kind.SOURCE_SINK) -public class SpannerIO { - - private SpannerIO() { } +import static com.google.common.base.Preconditions.checkNotNull; - @VisibleForTesting - public static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; -/* - *

To write a {@link PCollection} to Spanner, use {@link SpannerIO#writeTo} to - * specify the database location for output. +/** + * Google Cloud Spanner connectors. * - *

For example: + *

Reading from Cloud Spanner

+ * Status: Not implemented. * - *
 {@code
+ * 

Writing to Cloud Spanner

+ * Status: Experimental. + *

+ * {@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. * - * Write data to a table. Input read from {@link TextIO} will be parsed - * and transformed to a {@link PCollection} 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. * - * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); - * options.getInput = ; - * PCollection mutations = ...; + *

{@code
  *
- * Pipeline p = Pipeline.create(options);
- *   p.apply(TextIO.Read.from(options.getInput()))
- *       .apply(mutations)
- *       .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
- *   p.run();
+ * 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); } @@ -88,7 +91,7 @@ public static class Writer extends PTransform, PDone> { } /** - * Returns a new {@link Write} with a limit on the number of mutations per batch. + * 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) { @@ -127,18 +130,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Output Database")); } - public String getInstanceId() { - return instanceId; - } - - public String getDatabaseId() { - return databaseId; - } - - public int getBatchSize() { - return batchSize; - } - } @@ -153,7 +144,7 @@ public int getBatchSize() { * mutation operation should be idempotent. */ @VisibleForTesting - static class SpannerWriterFn extends DoFn { + static class SpannerWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); private transient Spanner spanner; private final String instanceId; From 68854f510e43560f85bee406fa842387ed18c718 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 21 Apr 2017 17:10:29 -0700 Subject: [PATCH 16/28] Simplified Spanner example --- .../examples/spanner/SpannerCSVLoader.java | 115 +++++++++++ .../spanner/csvloader/SpannerCSVLoader.java | 190 ------------------ .../csvloader/SpannerMutationBuilder.java | 179 ----------------- 3 files changed, 115 insertions(+), 369 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.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..f06ac61c17d8 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java @@ -0,0 +1,115 @@ +package org.apache.beam.examples.spanner; + +import com.google.cloud.spanner.*; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +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.*; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + +import java.util.Collections; + + +/** + * Generalized bulk loader for importing CSV files into Spanner + * + */ +public class SpannerCSVLoader { + + /** + * Command options specification. + */ + private static 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/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java deleted file mode 100644 index 3c664c7c911d..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerCSVLoader.java +++ /dev/null @@ -1,190 +0,0 @@ -package org.apache.beam.examples.spanner.csvloader; -package com.disney.dtss.desa.tools; - -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.DatabaseId; -import com.google.cloud.spanner.SpannerOptions; - -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.coders.SpannerMutationCoder; -import org.apache.beam.sdk.coders.Coder; -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.BigQueryOptions; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; - -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.transforms.Count; -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.values.PCollection; -import org.apache.beam.sdk.util.Transport; - - -import java.io.Serializable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.LinkedHashMap; - -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import org.yaml.snakeyaml.Yaml; - -import javax.annotation.Nullable; -import javax.servlet.http.HttpServletResponse; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * Generalized bulk loader for importing CSV files into Spanner - * - */ -public class SpannerCSVLoader { - - private static final Logger mLog = LoggerFactory.getLogger(SpannerCSVLoader.class); - private static Options options; - - - /** - * Command options specificiation. - */ - private static interface Options extends PipelineOptions { - @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("Schema YAML") - @Validation.Required - String getSchema(); - void setSchema(String value); - } - - - /** - * Constructs and executes the processing pipeline based upon command options. - */ - public static void main(String[] args) throws Exception { - - options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - Yaml yaml = new Yaml(); - InputStream in = Files.newInputStream(Paths.get(options.getSchema())); - TableInfo tableInfo = yaml.loadAs(in, TableInfo.class); - - Pipeline p = Pipeline.create(options); - p.apply(TextIO.Read.from(options.getInput())) - .apply(ParDo.of(new ParseLineFn(tableInfo))) - .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); - p.run().waitUntilFinish(); - - } - - - /** - * A DoFn that creates a Spanner Mutation for each CSV line. - */ - static class ParseLineFn extends DoFn { - private final TableInfo tableInfo; - private final SpannerMutationBuilder rowBuilder; - - ParseLineFn(TableInfo tableInfo) { - this.tableInfo = tableInfo; - rowBuilder = new SpannerMutationBuilder(tableInfo); - } - - - @ProcessElement - public void processElement(ProcessContext c) { - try { - Mutation e = rowBuilder.build(c.element()); - if (e != null) - c.output(e); - } - catch (Exception e) { - mLog.error(e.getMessage(), e); - } - } - } - - - public static class TableInfo implements Serializable { - - private String tableName; - private List schema; - - public String getTableName() { return tableName; } - public void setTableName(String tableName) { this.tableName = tableName; } - - public List getSchema() { return this.schema; } - public void setSchema(List schema) { this.schema = schema; } - } - - - public static class Schema implements Serializable { - - private String name; - private String type; - private boolean nullable; - private boolean exclude; - private boolean pk; - - - Schema() { - exclude = false; - nullable = true; - } - - public String getName() { return name; } - public void setName(String name) { this.name = name; } - public String getType() { return type; } - public void setType(String type) { this.type = type; } - public boolean isNullable() { return nullable; } - public void setNullable(boolean flag) { nullable = flag; } - public boolean isExclude() { return this.exclude; } - public void setExclude(boolean flag) { this.exclude = flag; } - public boolean isPk() { return pk; } - public void setPk(boolean flag) { pk = flag; } - - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Schema other = (Schema) obj; - return name.equals(other.getName()); - } - - } - -} diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java deleted file mode 100644 index 647625f0b50e..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/csvloader/SpannerMutationBuilder.java +++ /dev/null @@ -1,179 +0,0 @@ -package org.apache.beam.examples.spanner.csvloader; - -import java.lang.Iterable; -import java.io.Serializable; -import java.util.List; -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.Date; -import java.util.Calendar; -import java.text.SimpleDateFormat; -import java.util.Map; -import java.util.Set; -import java.util.HashSet; -import java.util.Arrays; - -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.ValueBinder; -import com.google.cloud.Timestamp; - -import org.apache.commons.csv.CSVParser; -import org.apache.commons.csv.CSVFormat; -import org.apache.commons.csv.CSVRecord; - -import com.disney.dtss.desa.tools.SpannerCSVLoader.TableInfo; -import com.disney.dtss.desa.tools.SpannerCSVLoader.Schema; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SpannerMutationBuilder implements Serializable { - - private static final SimpleDateFormat mRFC3339 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'"); - - private static final Logger mLog = LoggerFactory.getLogger(SpannerMutationBuilder.class); - - private final TableInfo tableInfo; - - public SpannerMutationBuilder(TableInfo tableInfo) { - this.tableInfo = tableInfo; - } - - public Mutation build(String csvData) throws Exception { - - CSVParser parser = CSVParser.parse(csvData, CSVFormat.EXCEL); - List list = parser.getRecords(); - if (list.size() != 1) - throw new IllegalStateException(String.format("CSV Record count = %d, should only be 1", list.size())); - - String[] fields = new String[list.get(0).size()]; - for (int i = 0; i < fields.length; i++) - fields[i] = list.get(0).get(i); - - Mutation.WriteBuilder builder = Mutation.newInsertOrUpdateBuilder(tableInfo.getTableName()); - - int i = -1; - for (Schema column : tableInfo.getSchema()) { - i++; - String name = column.getName(); - String type = column.getType(); - String mode = "N/A"; - if (column.isExclude()) - continue; - convert(builder.set(name), mode, type, fields[i]); - } - - return builder.build(); - } - - - private Date makeDate(Object value) throws Exception { - Date d = null; - String s = (String) value; - if (s != null && ! "".equals(s.trim())) { - if (s.indexOf('.') != -1) - try { - d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS z").parse(s); - } - catch (Exception e) { - d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS").parse(s); - } - else - d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z").parse(s); - //d = new Date(Long.parseLong((String) value) / 1000); - } - return d; - } - - - private Timestamp toTimestamp(Date d) { - return Timestamp.parseTimestamp(mRFC3339.format(d)); - } - - - private void convert(ValueBinder vb, String mode, String type, Object value) throws Exception { - if("REPEATED".equals(mode)) { - if ("STRING".equals(type)) { - vb.toStringArray((Iterable) value); - return; - } - if ("INTEGER".equals(type)) { - List vl = new ArrayList(); - for (String s : (List) value) - if (s == null || "".equals(s.trim())) - vl.add(0L); - else - vl.add(Long.parseLong(s)); - vb.toInt64Array(vl); - return; - } - if ("DOUBLE".equals(type)) { - List vl = new ArrayList(); - for (Double d : (List) value) - vl.add(d); - vb.toFloat64Array(vl); - return; - } - if ("BOOLEAN".equals(type)) { - List vl = new ArrayList(); - for (Boolean b : (List) value) - vl.add(b); - vb.toBoolArray(vl); - return; - } - if ("TIMESTAMP".equals(type)) { - List vl = new ArrayList(); - for (String s : (List) value) - vl.add(toTimestamp(makeDate(s))); - vb.toTimestampArray(vl); - return; - } - } - else { - String s = (String) value; - if (s != null) - s = s.trim(); - if ("".equals(s) || "NULL".equals(s)) - s = null; - - if ("STRING".equals(type)) { - vb.to(s); - return; - } - if ("INTEGER".equals(type)) { - if (s == null) - vb.to((Long) null); - else - vb.to(Long.parseLong(s)); - return; - } - if ("DOUBLE".equals(type)) { - if (s == null) - vb.to((Double) null); - else - vb.to(Double.parseDouble(s)); - return; - } - if ("BOOLEAN".equals(type)) { - if ( s != null && "1".equals(s)) - s = "true"; - if ( s != null && "0".equals(s)) - s = "false"; - if (s == null) - vb.to((Boolean) null); - else - vb.to(Boolean.parseBoolean(s)); - return; - } - if ("TIMESTAMP".equals(type)) { - if (s == null) - vb.to((Timestamp) null); - else - vb.to(toTimestamp(makeDate(s))); - return; - } - } - mLog.error("The BigQuery type " + type + " is not currently supported."); - System.exit(0); - } -} From 335021f8464c84942b69a95ffbc39edf928b0468 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 21 Apr 2017 17:43:28 -0700 Subject: [PATCH 17/28] Restore coders --- .../main/java/org/apache/beam/sdk/coders/CoderRegistry.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 423829344ba0..9bebcc841efe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Struct; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import com.google.common.collect.HashMultimap; @@ -111,6 +113,8 @@ public class CoderRegistry implements CoderProvider { codersToRegister.put(Void.class, CoderFactories.fromStaticMethods(VoidCoder.class)); codersToRegister.put(byte[].class, CoderFactories.fromStaticMethods(ByteArrayCoder.class)); codersToRegister.put(IntervalWindow.class, CoderFactories.forCoder(IntervalWindow.getCoder())); + codersToRegister.put(Mutation.class, CoderFactories.forCoder(SpannerMutationCoder.of())); + codersToRegister.put(Struct.class, CoderFactories.forCoder(SpannerStructCoder.of())); // Enumerate all the CoderRegistrars in a deterministic order, adding all coders to register Set registrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE); From b1daa91acd5365c99f39f509606de3403047e2a8 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 21 Apr 2017 17:43:41 -0700 Subject: [PATCH 18/28] Bump Spanner version --- pom.xml | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 483b1b68f72a..13686cb475aa 100644 --- a/pom.xml +++ b/pom.xml @@ -121,7 +121,7 @@ 0.5.160304 20.0 1.2.0 - 0.1.0 + 0.1.5 1.3 2.8.8 3.0.1 @@ -143,7 +143,7 @@ -Werror -Xpkginfo:always nothing - 0.11.2-beta + 0.13.0-beta 0.11.2-alpha @@ -813,6 +813,13 @@ ${google-cloud-bigdataoss.version}
+ + com.google.cloud + google-cloud-spanner + ${spanner.version} + + + com.google.cloud.bigdataoss util From d8db6507e29153de58c1b3dcbc0708908d4937d5 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 21 Apr 2017 18:00:13 -0700 Subject: [PATCH 19/28] Format the example code --- .../examples/spanner/SpannerCSVLoader.java | 133 +++++++++--------- .../apache/beam/sdk/coders/CoderRegistry.java | 1 + 2 files changed, 70 insertions(+), 64 deletions(-) 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 f06ac61c17d8..803e8f11a4a0 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 @@ -12,104 +12,109 @@ import java.util.Collections; - /** - * Generalized bulk loader for importing CSV files into Spanner - * + * An example pipleine to import data into Spanner. */ public class SpannerCSVLoader { - - /** - * Command options specification. - */ - private static interface Options extends PipelineOptions { + /** Command options specification. */ + private static 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); + void setDatabaseId(String databaseId); @Description("Table name") @Validation.Required String getTable(); + void setTable(String value); } - - - /** - * Constructs and executes the processing pipeline based upon command options. - */ + /** 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(); + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + if (options.isCreateDatabase()) { + createDatabase(options); + } + + 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(); + 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(MAX)," + + " Email STRING(MAX)," + + " Age INT64" + + ") PRIMARY KEY (Key)")); + op.waitFor(); } - - /** - * A DoFn that creates a Spanner Mutation for each CSV line. - */ + /** 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); + 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/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 9bebcc841efe..032900051295 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -113,6 +113,7 @@ public class CoderRegistry implements CoderProvider { codersToRegister.put(Void.class, CoderFactories.fromStaticMethods(VoidCoder.class)); codersToRegister.put(byte[].class, CoderFactories.fromStaticMethods(ByteArrayCoder.class)); codersToRegister.put(IntervalWindow.class, CoderFactories.forCoder(IntervalWindow.getCoder())); + // TODO(mairbek): Remove once Spanner classes are serializable. codersToRegister.put(Mutation.class, CoderFactories.forCoder(SpannerMutationCoder.of())); codersToRegister.put(Struct.class, CoderFactories.forCoder(SpannerStructCoder.of())); From 7d1336f022f9f58aef37edaf5b1463e5ce8d476c Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 24 Apr 2017 16:25:04 -0700 Subject: [PATCH 20/28] Typo --- .../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 803e8f11a4a0..e4192e3a2631 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 @@ -13,7 +13,7 @@ import java.util.Collections; /** - * An example pipleine to import data into Spanner. + * An example pipeline to import data into Spanner. */ public class SpannerCSVLoader { /** Command options specification. */ From 5defd2185a3e02f4aaa3ca56fbc95267bda719d3 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Tue, 25 Apr 2017 14:36:01 -0700 Subject: [PATCH 21/28] Updated google-spanner-client. Removed custom coders, Mutation is serializable now. --- pom.xml | 6 +- sdks/java/core/pom.xml | 5 - .../apache/beam/sdk/coders/CoderRegistry.java | 5 - .../beam/sdk/coders/SpannerMutationCoder.java | 225 ------------------ .../beam/sdk/coders/SpannerStructCoder.java | 191 --------------- .../apache/beam/SdkCoreApiSurfaceTest.java | 2 - 6 files changed, 2 insertions(+), 432 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java diff --git a/pom.xml b/pom.xml index 13686cb475aa..3db52170165f 100644 --- a/pom.xml +++ b/pom.xml @@ -121,7 +121,7 @@ 0.5.160304 20.0 1.2.0 - 0.1.5 + 0.1.9 1.3 2.8.8 3.0.1 @@ -143,8 +143,7 @@ -Werror -Xpkginfo:always nothing - 0.13.0-beta - 0.11.2-alpha + 0.16.0-beta pom @@ -819,7 +818,6 @@ ${spanner.version} - com.google.cloud.bigdataoss util diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 6512ff035407..85b8defa8310 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -231,11 +231,6 @@ joda-time joda-time - - com.google.cloud - google-cloud-spanner - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 032900051295..423829344ba0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -19,8 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Struct; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import com.google.common.collect.HashMultimap; @@ -113,9 +111,6 @@ public class CoderRegistry implements CoderProvider { codersToRegister.put(Void.class, CoderFactories.fromStaticMethods(VoidCoder.class)); codersToRegister.put(byte[].class, CoderFactories.fromStaticMethods(ByteArrayCoder.class)); codersToRegister.put(IntervalWindow.class, CoderFactories.forCoder(IntervalWindow.getCoder())); - // TODO(mairbek): Remove once Spanner classes are serializable. - codersToRegister.put(Mutation.class, CoderFactories.forCoder(SpannerMutationCoder.of())); - codersToRegister.put(Struct.class, CoderFactories.forCoder(SpannerStructCoder.of())); // Enumerate all the CoderRegistrars in a deterministic order, adding all coders to register Set registrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java deleted file mode 100644 index 59ba9e24a3d7..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerMutationCoder.java +++ /dev/null @@ -1,225 +0,0 @@ -package org.apache.beam.sdk.coders; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.google.cloud.ByteArray; -import com.google.cloud.Date; -import com.google.cloud.Timestamp; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Type; -import com.google.cloud.spanner.Value; -import com.google.cloud.spanner.ValueBinder; -import com.google.common.io.ByteArrayDataInput; -import com.google.common.io.ByteArrayDataOutput; -import com.google.common.io.ByteStreams; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.util.Map; - -/** - * A {@link Coder} that encodes Spanner {@link Mutation} objects. - */ -public class SpannerMutationCoder extends AtomicCoder { - - @JsonCreator - public static SpannerMutationCoder of() { - return INSTANCE; - } - - /***************************/ - - private static final SpannerMutationCoder INSTANCE = new SpannerMutationCoder(); - - private SpannerMutationCoder() {} - - @Override - public void encode(Mutation value, OutputStream outStream, Context context) - throws IOException, CoderException { - - if (value == null) { - throw new CoderException("cannot encode a null Mutation"); - } - - if (Mutation.Op.DELETE == value.getOperation()) { - throw new UnsupportedOperationException("DELETE Mutations not supported!"); - } - - ByteArrayDataOutput out = ByteStreams.newDataOutput(); - - out.writeUTF(value.getOperation().name()); - out.writeUTF(value.getTable()); - - Map state = value.asMap(); - - // Write number of columns - out.writeInt(state.size()); - - // Write out column names, types and values - ValueSerializer ser = ValueSerializer.of(); - for (String columnName : state.keySet()) { - Value v = state.get(columnName); - out.writeUTF(columnName); - out.writeUTF(v.getType().getCode().name()); - ser.writeTo(out, v); - } - - byte[] buf = out.toByteArray(); - outStream.write(ByteBuffer.allocate(4).putInt(buf.length).array()); - outStream.write(buf); - outStream.flush(); - } - - @Override - public Mutation decode(InputStream inStream, Context context) throws IOException { - - byte[] lengthSize = new byte[4]; - inStream.read(lengthSize, 0, 4); - int expectedSize = ByteBuffer.wrap(lengthSize).getInt(); - byte[] data = new byte[expectedSize]; - inStream.read(data, 0, expectedSize); - ByteArrayDataInput in = ByteStreams.newDataInput(data); - - Mutation.Op operation = Enum.valueOf(Mutation.Op.class, in.readUTF()); - String tableName = in.readUTF(); - Mutation.WriteBuilder builder = makeBuilder(operation, tableName); - int columnCount = in.readInt(); - ValueDeserializer ser = ValueDeserializer.of(); - // Deserialize column values - for (int i = 0; i < columnCount; i++) { - String columnName = in.readUTF(); - builder = ser.readFrom(in, builder.set(columnName)); - } - - return builder.build(); - } - - private Mutation.WriteBuilder makeBuilder(Mutation.Op operation, String tableName) { - switch(operation) { - case INSERT: - return Mutation.newInsertBuilder(tableName); - case UPDATE: - return Mutation.newUpdateBuilder(tableName); - case REPLACE: - return Mutation.newReplaceBuilder(tableName); - case INSERT_OR_UPDATE: - return Mutation.newInsertOrUpdateBuilder(tableName); - } - throw new UnsupportedOperationException("Mutation operation unsupported."); - } - - @Override - protected long getEncodedElementByteSize(Mutation value, Context context) - throws Exception { - //return value.getSerializedSize(); - return 0L; //TODO: Implement this. - } - - /** - * {@inheritDoc} - * - * @throws NonDeterministicException always. - * A Spanner table can hold arbitrary {@link Object} instances, which - * makes the encoding non-deterministic. - */ - @Override - public void verifyDeterministic() throws NonDeterministicException { - throw new NonDeterministicException(this, - "Spanner encodings can hold arbitrary Object instances"); - } - - - static class ValueSerializer implements java.io.Serializable { - - private static final ValueSerializer INSTANCE = new ValueSerializer(); - - public static ValueSerializer of() { - return INSTANCE; - } - - public void writeTo(DataOutput out, Value v) throws IOException { - if (v.isNull()) { - out.writeByte(0); //NULL indicator byte - return; - } - out.writeByte(1); // Not a null value - Type.Code c = v.getType().getCode(); - switch (c) { - case BOOL: - out.writeBoolean(v.getBool()); - break; - case INT64: - out.writeLong(v.getInt64()); - break; - case FLOAT64: - out.writeDouble(v.getFloat64()); - break; - case STRING: - out.writeUTF(v.getString()); - break; - case BYTES: - byte[] b = v.getBytes().toByteArray(); - out.writeInt(b.length); - out.write(b); - break; - case TIMESTAMP: - out.writeUTF(v.getTimestamp().toString()); - break; - case DATE: - out.writeUTF(v.getDate().toString()); - break; - case ARRAY: - throw new UnsupportedOperationException("ARRAY type not implemented yet."); - case STRUCT: - throw new UnsupportedOperationException("STRUCT type not implemented yet."); - } - } - } - - static class ValueDeserializer implements java.io.Serializable { - - private static final ValueDeserializer INSTANCE = new ValueDeserializer(); - - public static ValueDeserializer of() { - return INSTANCE; - } - - public Mutation.WriteBuilder readFrom(DataInput in, ValueBinder vb) - throws IOException { - - Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); - byte b = in.readByte(); // NULL indicator - switch (c) { - case BOOL: - return b == 1 ? vb.to(in.readBoolean()) : vb.to((Boolean) null); - case INT64: - return b == 1 ? vb.to(in.readLong()) : vb.to((Long) null); - case FLOAT64: - return b == 1 ? vb.to(in.readDouble()) : vb.to((Double) null); - case STRING: - return b == 1 ? vb.to(in.readUTF()) : vb.to((String) null); - case BYTES: - if (b == 0) { - return vb.to((ByteArray) null); - } - int size = in.readInt(); - byte[] buf = new byte[size]; - in.readFully(buf); - return vb.to(ByteArray.copyFrom(buf)); - case TIMESTAMP: - return b == 1 ? vb.to(Timestamp.parseTimestamp(in.readUTF())) : - vb.to((Timestamp) null); - case DATE: - return b == 1 ? vb.to(Date.parseDate(in.readUTF())) : vb.to((Date) null); - case ARRAY: - throw new UnsupportedOperationException("ARRAY type not implemented yet."); - case STRUCT: - throw new UnsupportedOperationException("STRUCT type not implemented yet."); - } - throw new UnsupportedOperationException("Type unsupported."); - } - } - -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java deleted file mode 100644 index 0d572995aac5..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SpannerStructCoder.java +++ /dev/null @@ -1,191 +0,0 @@ -package org.apache.beam.sdk.coders; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.google.cloud.ByteArray; -import com.google.cloud.Date; -import com.google.cloud.Timestamp; -import com.google.cloud.spanner.Struct; -import com.google.cloud.spanner.Type; -import com.google.cloud.spanner.ValueBinder; -import com.google.common.io.ByteArrayDataInput; -import com.google.common.io.ByteArrayDataOutput; -import com.google.common.io.ByteStreams; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.util.List; - -/** - * A {@link Coder} that encodes Spanner {@link Struct} objects. - */ -public class SpannerStructCoder extends AtomicCoder { - - @JsonCreator - public static SpannerStructCoder of() { - return INSTANCE; - } - - /***************************/ - - private static final SpannerStructCoder INSTANCE = new SpannerStructCoder(); - - private SpannerStructCoder() {} - - @Override - public void encode(Struct value, OutputStream outStream, Context context) - throws IOException, CoderException { - if (value == null) { - throw new CoderException("cannot encode a null Struct"); - } - - ByteArrayDataOutput out = ByteStreams.newDataOutput(); - - List fields = value.getType().getStructFields(); - - // Write number of columns - out.writeInt(value.getColumnCount()); - - // Write out column names, types and values - ValueSerializer ser = ValueSerializer.of(); - - for (Type.StructField f : fields) { - out.writeUTF(f.getName()); - out.writeUTF(f.getType().getCode().name()); - ser.writeTo(out, value, f.getName(), f.getType().getCode()); - } - - byte[] buf = out.toByteArray(); - out.write(ByteBuffer.allocate(4).putInt(buf.length).array()); - outStream.write(buf); - outStream.flush(); - } - - @Override - public Struct decode(InputStream inStream, Context context) throws IOException { - - byte[] lengthSize = new byte[4]; - inStream.read(lengthSize, 0, 4); - int expectedSize = ByteBuffer.wrap(lengthSize).getInt(); - byte[] data = new byte[expectedSize]; - inStream.read(data, 0, expectedSize); - ByteArrayDataInput in = ByteStreams.newDataInput(data); - - Struct.Builder builder = Struct.newBuilder(); - - int columnCount = in.readInt(); - ValueDeserializer ser = ValueDeserializer.of(); - // Deserialize column values - for (int i = 0; i < columnCount; i++) { - String columnName = in.readUTF(); - builder = ser.readFrom(in, builder.set(columnName)); - } - - return builder.build(); - } - - @Override - protected long getEncodedElementByteSize(Struct value, Context context) - throws Exception { - //return value.getSerializedSize(); - return 0L; //TODO: Implement this. - } - - /** - * {@inheritDoc} - * - * @throws NonDeterministicException always. - * A Datastore kind can hold arbitrary {@link Object} instances, which - * makes the encoding non-deterministic. - */ - @Override - public void verifyDeterministic() throws NonDeterministicException { - throw new NonDeterministicException(this, - "Datastore encodings can hold arbitrary Object instances"); - } - - - static class ValueSerializer implements java.io.Serializable { - - private static final ValueSerializer INSTANCE = new ValueSerializer(); - - public static ValueSerializer of() { - return INSTANCE; - } - - public void writeTo(DataOutput out, Struct v, String colName, Type.Code c) - throws IOException { - switch (c) { - case BOOL: - out.writeBoolean(v.getBoolean(colName)); - break; - case INT64: - out.writeLong(v.getLong(colName)); - break; - case FLOAT64: - out.writeDouble(v.getDouble(colName)); - break; - case STRING: - out.writeUTF(v.getString(colName)); - break; - case BYTES: - byte[] b = v.getBytes(colName).toByteArray(); - out.writeInt(b.length); - out.write(b); - break; - case TIMESTAMP: - out.writeUTF(v.getTimestamp(colName).toString()); - break; - case DATE: - out.writeUTF(v.getDate(colName).toString()); - break; - case ARRAY: - throw new UnsupportedOperationException("ARRAY type not implemented yet."); - case STRUCT: - throw new UnsupportedOperationException("STRUCT type not implemented yet."); - } - } - } - - static class ValueDeserializer implements java.io.Serializable { - - private static final ValueDeserializer INSTANCE = new ValueDeserializer(); - - public static ValueDeserializer of() { - return INSTANCE; - } - - public Struct.Builder readFrom(DataInput in, ValueBinder vb) - throws IOException { - - Type.Code c = Enum.valueOf(Type.Code.class, in.readUTF()); - switch (c) { - case BOOL: - return vb.to(in.readBoolean()); - case INT64: - return vb.to(in.readLong()); - case FLOAT64: - return vb.to(in.readDouble()); - case STRING: - return vb.to(in.readUTF()); - case BYTES: - int size = in.readInt(); - byte[] buf = new byte[size]; - in.readFully(buf); - return vb.to(ByteArray.copyFrom(buf)); - case TIMESTAMP: - return vb.to(Timestamp.parseTimestamp(in.readUTF())); - case DATE: - return vb.to(Date.parseDate(in.readUTF())); - case ARRAY: - throw new UnsupportedOperationException("ARRAY type not implemented yet."); - case STRUCT: - throw new UnsupportedOperationException("STRUCT type not implemented yet."); - } - throw new UnsupportedOperationException("Type unsupported."); - } - } - -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java index 5b88b32c500f..153bd8478b28 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java @@ -40,8 +40,6 @@ public void testSdkApiSurface() throws Exception { "org.apache.beam", "com.google.api.client", "com.google.protobuf", - "com.google.cloud", - "com.google.cloud.spanner", "com.fasterxml.jackson.annotation", "com.fasterxml.jackson.core", "com.fasterxml.jackson.databind", From 084d68e1f99489ab347b67941532d5087f2bad66 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Tue, 25 Apr 2017 14:38:13 -0700 Subject: [PATCH 22/28] Some cleanup --- .../org/apache/beam/examples/spanner/SpannerCSVLoader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 e4192e3a2631..8df713f4c917 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 @@ -47,7 +47,7 @@ private static interface Options extends PipelineOptions { 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); @@ -63,7 +63,7 @@ public static void main(String[] args) throws Exception { p.run().waitUntilFinish(); } - public static void createDatabase(Options options) { + private static void createDatabase(Options options) { Spanner client = SpannerOptions.getDefaultInstance().getService(); DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); From 839fe058f3bbcc287310605282d7f55ec64fb3d9 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Wed, 26 Apr 2017 09:35:40 -0700 Subject: [PATCH 23/28] Minor cleanup of checkstyle issues on SpannerIO, undeclared dependencies --- sdks/java/io/google-cloud-platform/pom.xml | 27 ++++++++----- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 39 +++++++++++++++---- 2 files changed, 48 insertions(+), 18 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 61a6d63d2b9f..064d0e9bf88e 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -81,11 +81,28 @@ jackson-databind
+ + io.grpc + grpc-core + + + + com.google.api.grpc + grpc-google-common-protos + 0.1.0 + + com.google.apis google-api-services-bigquery + + com.google.api + api-common + 1.0.0-rc2 + + com.google.apis google-api-services-storage @@ -121,11 +138,6 @@ grpc-auth - - io.grpc - grpc-core - - io.grpc grpc-netty @@ -203,11 +215,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 index 7bd07fe90c74..fba6b37ca8d2 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 @@ -1,11 +1,39 @@ +/* + * 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.*; + +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; @@ -19,11 +47,7 @@ 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; /** @@ -34,11 +58,10 @@ * *

Writing to Cloud Spanner

* Status: Experimental. - *

- * {@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. * + *

{@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. + * {@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. * From 500a22a2f56269e44d3e17b3d2ccf61b37cf2373 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Wed, 26 Apr 2017 20:31:03 -0700 Subject: [PATCH 24/28] Minor checkstyle issue with SpannerCSVLoader example. --- .../examples/spanner/SpannerCSVLoader.java | 167 ++++--- .../beam/examples/spanner-csv-loader/pom.xml | 457 ------------------ 2 files changed, 95 insertions(+), 529 deletions(-) delete mode 100644 examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml 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 8df713f4c917..eee581da2148 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 @@ -1,120 +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.*; +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.*; +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; -import java.util.Collections; + /** - * An example pipeline to import data into Spanner. + * Generalized bulk loader for importing CSV files into Spanner. + * */ public class SpannerCSVLoader { - /** Command options specification. */ - private static interface Options extends PipelineOptions { + + /** + * 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 databaseId); + 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); - - if (options.isCreateDatabase()) { - createDatabase(options); - } - - 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(); + 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(); } - private 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(MAX)," - + " Email STRING(MAX)," - + " Age INT64" - + ") PRIMARY KEY (Key)")); - op.waitFor(); + 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. */ + + /** + * 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; + 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); } - 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/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml b/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml deleted file mode 100644 index 3d402a8c9fa4..000000000000 --- a/examples/java8/src/main/java/org/apache/beam/examples/spanner-csv-loader/pom.xml +++ /dev/null @@ -1,457 +0,0 @@ - - - - 4.0.0 - - - - com.disney.dtss.desa.tools - SpannerCSVLoader - Spanner migration tool - A tool for the parallel transfer of data from text files - to Spanner. - http://cloud.google.com/dataflow - - 0.0.1-SNAPSHOT - - jar - - - 2.0.0 - ${settings.localRepository}/org/mortbay/jetty/alpn/jetty-alpn-agent/${jetty.alpnAgent.version}/jetty-alpn-agent-${jetty.alpnAgent.version}.jar - - - - - java8 - - [1.8,) - - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-java8-main-source - initialize - - add-source - - - - ${project.basedir}/src/main/java8 - - - - - - add-java8-test-source - initialize - - add-test-source - - - - ${project.basedir}/src/test/java8 - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - - - default-testCompile - test-compile - - testCompile - - - 1.7 - 1.7 - - - **/*Java8Test.java - **/game/**/*.java - - - - - - - java8-test-compile - test-compile - - testCompile - - - 1.8 - 1.8 - - - **/*Java8Test.java - **/game/**/*.java - - - - - - - default-compile - compile - - compile - - - 1.7 - 1.7 - - - **/*Java8*.java - **/game/**/*.java - - - - - - - java8-compile - compile - - compile - - - 1.8 - 1.8 - - - **/*Java8*.java - **/game/**/*.java - - - - - - - - - - - - - - maven-compiler-plugin - - - - maven-shade-plugin - 2.4.1 - - - package - - shade - - - ${project.artifactId}-bundled-${project.version} - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - maven-dependency-plugin - 2.10 - - - get-jetty-alpn-agent - validate - - get - - - org.mortbay.jetty.alpn - jetty-alpn-agent - ${jetty.alpnAgent.version} - - - - - - maven-surefire-plugin - 2.19.1 - - -javaagent:${jetty.alpnAgent.path} - - - - - - - - - - - com.google.api.grpc - grpc-google-cloud-spanner-v1 - 0.1.5 - - - io.grpc - grpc-all - - - - - io.netty - netty-tcnative-boringssl-static - 1.1.33.Fork26 - - - - - - org.apache.beam - beam-runners-direct-java - 0.6.0-SNAPSHOT - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - 0.6.0-SNAPSHOT - - - org.apache.beam - beam-sdks-java-core - 0.6.0-SNAPSHOT - - - - - - - com.google.apis - google-api-services-bigquery - v2-rev295-1.22.0 - - - com.google.guava - guava-jdk5 - - - - - - - - com.google.cloud - google-cloud-spanner - 0.11.2-beta - - - - org.apache.beam - beam-sdks-java-io-google-cloud-platform - 0.6.0-SNAPSHOT - - - - org.apache.commons - commons-csv - 1.1 - - - - org.yaml - snakeyaml - 1.11 - - - - - - - - joda-time - joda-time - 2.9.2 - - - - - - org.slf4j - slf4j-api - 1.7.14 - - - - org.slf4j - slf4j-jdk14 - 1.7.14 - runtime - - - - javax.servlet - javax.servlet-api - 3.1.0 - - - - - - - - junit - junit - 4.11 - - - - org.mockito - mockito-all - 1.10.19 - test - - - From 66ccbcd742ab01d4a36da4a14bce11d67d53cf6b Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Wed, 26 Apr 2017 21:07:15 -0700 Subject: [PATCH 25/28] Minor changes from Mairbek --- .../main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 2 +- .../java/org/apache/beam/sdk/io/gcp/spanner/package-info.java | 2 +- 2 files changed, 2 insertions(+), 2 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 fba6b37ca8d2..172ed8f02687 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 @@ -163,7 +163,7 @@ public void populateDisplayData(DisplayData.Builder builder) { *

See * *

Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. times). This means that the + * {@link SpannerIO#MAX_RETRIES}. times). This means that the * mutation operation should be idempotent. */ @VisibleForTesting 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 index 12c1f1299527..19e468cce041 100644 --- 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 @@ -18,6 +18,6 @@ /** *

Provides an API for reading from and writing to - * Google Cloud Spanner. + * Google Cloud Spanner. */ package org.apache.beam.sdk.io.gcp.spanner; From 202f6e1e338d96480f213e121371e2e9b41e808e Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Mon, 1 May 2017 11:03:49 -0700 Subject: [PATCH 26/28] Parameterize new dependencies in pom file. --- pom.xml | 1 + sdks/java/io/google-cloud-platform/pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index c7dbc949bbe3..bc40414554da 100644 --- a/pom.xml +++ b/pom.xml @@ -139,6 +139,7 @@ 4.4.1 4.3.5.RELEASE 2.0 + 1.0.0-rc2 -Werror -Xpkginfo:always diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 064d0e9bf88e..e7a14892043a 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -89,7 +89,7 @@ com.google.api.grpc grpc-google-common-protos - 0.1.0 + ${grpc-google-common-protos.version} @@ -100,7 +100,7 @@ com.google.api api-common - 1.0.0-rc2 + ${api-common.version} From 71b62955bfdcd3b77ad9f39ad28d8fe909124768 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Mon, 1 May 2017 11:47:34 -0700 Subject: [PATCH 27/28] Fixed common protos version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index bc40414554da..b188a06c5c89 100644 --- a/pom.xml +++ b/pom.xml @@ -121,7 +121,7 @@ 0.5.160304 20.0 1.2.0 - 0.1.9 + 0.1.0 1.3 2.8.8 3.0.1 From 9656e228fd0b77a212ed5dbf4b390fd18aca8591 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Mon, 15 May 2017 08:42:58 -0700 Subject: [PATCH 28/28] Attempt to resolve merge conflict --- pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index b188a06c5c89..01a48b340fb4 100644 --- a/pom.xml +++ b/pom.xml @@ -139,12 +139,11 @@ 4.4.1 4.3.5.RELEASE 2.0 - 1.0.0-rc2 - -Werror -Xpkginfo:always nothing 0.16.0-beta + 1.0.0-rc2 pom