From 391682935ac51d72eff3b21196c18867297a336d Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Wed, 19 Apr 2017 07:56:52 -0700 Subject: [PATCH 1/6] Support dynamic schemas and remove Write reference from nested classes. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 73 ++++----- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 149 ++++++++++++++---- .../sdk/io/gcp/bigquery/CreateTables.java | 38 +++-- .../sdk/io/gcp/bigquery/SchemaFunction.java | 58 +++++++ .../sdk/io/gcp/bigquery/StreamingInserts.java | 57 +++---- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 94 ++++++----- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 84 ++++++---- 7 files changed, 365 insertions(+), 188 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 593c580bd4f88..47c598982a96e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -23,10 +23,10 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; +import com.google.common.collect.Lists; import java.io.IOException; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -40,7 +40,6 @@ 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.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; @@ -59,26 +58,28 @@ /** PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ class BatchLoads extends PTransform>, WriteResult> { - BigQueryIO.Write write; - - private static class ConstantSchemaFunction - implements SerializableFunction { - private final @Nullable ValueProvider jsonSchema; - - ConstantSchemaFunction(ValueProvider jsonSchema) { - this.jsonSchema = jsonSchema; - } - - @Override - @Nullable - public TableSchema apply(TableDestination table) { - return BigQueryHelpers.fromJsonString( - jsonSchema == null ? null : jsonSchema.get(), TableSchema.class); - } + private BigQueryServices bigQueryServices; + private final WriteDisposition writeDisposition; + private final CreateDisposition createDisposition; + private final ValueProvider singletonJsonTableRef; + private final String singletonTableDescription; + private final SchemaFunction schemaFunction; + + BatchLoads(WriteDisposition writeDisposition, CreateDisposition createDisposition, + ValueProvider singletonJsonTableRef, + String singletonTableDescription, + SchemaFunction schemaFunction) { + bigQueryServices = new BigQueryServicesImpl(); + this.writeDisposition = writeDisposition; + this.createDisposition = createDisposition; + this.singletonJsonTableRef = singletonJsonTableRef; + this.singletonTableDescription = singletonTableDescription; + this.schemaFunction = schemaFunction; } - BatchLoads(BigQueryIO.Write write) { - this.write = write; + BatchLoads withTestServices(BigQueryServices bigQueryServices) { + this.bigQueryServices = bigQueryServices; + return this; } @Override @@ -167,19 +168,19 @@ public String apply(String input) { "WritePartition", ParDo.of( new WritePartition( - write.getJsonTableRef(), - write.getTableDescription(), + singletonJsonTableRef, + singletonTableDescription, resultsView, multiPartitionsTag, singlePartitionTag)) .withSideInputs(resultsView) .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); - // Since BigQueryIO.java does not yet have support for per-table schemas, inject a constant - // schema function here. If no schema is specified, this function will return null. - // TODO: Turn this into a side-input instead. - SerializableFunction schemaFunction = - new ConstantSchemaFunction(write.getJsonSchema()); + List> writeTablesSideInputs = Lists.newArrayList(); + writeTablesSideInputs.add(jobIdTokenView); + if (schemaFunction.getSideInput() != null) { + writeTablesSideInputs.add(schemaFunction.getSideInput()); + } Coder, List>> partitionsCoder = KvCoder.of( @@ -201,13 +202,13 @@ public String apply(String input) { ParDo.of( new WriteTables( false, - write.getBigQueryServices(), + bigQueryServices, jobIdTokenView, tempFilePrefix, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, schemaFunction)) - .withSideInputs(jobIdTokenView)); + .withSideInputs(writeTablesSideInputs)); // This view maps each final table destination to the set of temporary partitioned tables // the PCollection was loaded into. @@ -218,10 +219,10 @@ public String apply(String input) { "WriteRename", ParDo.of( new WriteRename( - write.getBigQueryServices(), + bigQueryServices, jobIdTokenView, - write.getWriteDisposition(), - write.getCreateDisposition(), + writeDisposition, + createDisposition, tempTablesView)) .withSideInputs(tempTablesView, jobIdTokenView)); @@ -238,13 +239,13 @@ public String apply(String input) { ParDo.of( new WriteTables( true, - write.getBigQueryServices(), + bigQueryServices, jobIdTokenView, tempFilePrefix, - write.getWriteDisposition(), - write.getCreateDisposition(), + writeDisposition, + createDisposition, schemaFunction)) - .withSideInputs(jobIdTokenView)); + .withSideInputs(writeTablesSideInputs)); return WriteResult.in(input.getPipeline()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index ea9790658c6d2..069e629dbb029 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -30,12 +30,14 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.regex.Pattern; import javax.annotation.Nullable; @@ -69,6 +71,7 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -181,7 +184,20 @@ * {@link BigQueryIO.Write#withFormatFunction} to convert each element into a {@link TableRow} * object. * - *

Per-value tables currently do not perform well in batch mode. + *

Each table can be given it's own schema using {@link BigQueryIO.Write#withSchemaFunction}. + * {@link SchemaFunction} is a user-defined function that maps a destination table to a BigQuery + * schema. Note that this function must return a valid schema for all tables (unless the create + * disposition is CREATE_NEVER), otherwise the pipeline will fail to create tables. + * + *

Per-table schemas can also be provided using {@link BigQueryIO.Write#withSchemaFromView}. + * This allows you the schemas to be calculated based on a previous pipeline stage or statically + * via a {@link org.apache.beam.sdk.transforms.Create} transform. This method expects to receive a + * map-valued {@link PCollectionView}, mapping table specifications (project:dataset.table-id), + * to JSON formatted {@link TableSchema} objects. All destination tables must be present in this + * map, or the pipeline will fail to create tables. Care should be taken if the map value is based + * on a triggered aggregation over and unbounded {@link PCollection}. This method can also be useful + * when writing to a single table, as it allows a previous stage to calculate the schema (possibly + * based on the full collection of records being written to BigQuery). * *

Permissions

* @@ -686,7 +702,7 @@ public abstract static class Write extends PTransform, WriteRe getTableFunction(); @Nullable abstract SerializableFunction getFormatFunction(); /** Table schema. The schema is required only if the table does not exist. */ - @Nullable abstract ValueProvider getJsonSchema(); + @Nullable abstract SchemaFunction getSchemaFunction(); abstract CreateDisposition getCreateDisposition(); abstract WriteDisposition getWriteDisposition(); /** Table description. Default is empty. */ @@ -704,7 +720,7 @@ abstract Builder setTableFunction( SerializableFunction, TableDestination> tableFunction); abstract Builder setFormatFunction( SerializableFunction formatFunction); - abstract Builder setJsonSchema(ValueProvider jsonSchema); + abstract Builder setSchemaFunction(SchemaFunction schemaFunction); abstract Builder setCreateDisposition(CreateDisposition createDisposition); abstract Builder setWriteDisposition(WriteDisposition writeDisposition); abstract Builder setTableDescription(String tableDescription); @@ -824,16 +840,6 @@ public Write to( return toBuilder().setTableFunction(tableFunction).build(); } - /** - * Like {@link BigQueryIO.Write#to(SerializableFunction)}, but the function returns a - * {@link TableReference} instead of a string table specification. - */ - private Write toTableReference( - SerializableFunction, TableDestination> tableFunction) { - ensureToNotCalledYet(); - return toBuilder().setTableFunction(tableFunction).build(); - } - /** * Formats the user's type into a {@link TableRow} to be written to BigQuery. */ @@ -857,6 +863,41 @@ public TableDestination apply(ValueInSingleWindow value) { } } + /** + * A function that returns the same schema for every table. + */ + static class ConstantSchemaFunction extends SchemaFunction { + private final @Nullable ValueProvider jsonSchema; + + private ConstantSchemaFunction(@Nullable ValueProvider jsonSchema) { + this.jsonSchema = jsonSchema; + } + + public static ConstantSchemaFunction fromTableSchema(ValueProvider tableSchema) { + if (tableSchema.isAccessible()) { + // TableSchema isn't Serializable, so convert to JSON. + return new ConstantSchemaFunction(StaticValueProvider.of( + BigQueryHelpers.toJsonString(tableSchema.get()))); + } else { + return new ConstantSchemaFunction( + NestedValueProvider.of(tableSchema, new TableSchemaToJsonSchema())); + } + } + + public static ConstantSchemaFunction fromJsonSchema(ValueProvider jsonSchema) { + return new ConstantSchemaFunction(jsonSchema); + } + + @Override + public TableSchema apply(TableDestination table) { + if (jsonSchema == null) { + return null; + } else { + return BigQueryHelpers.fromJsonString(jsonSchema.get(), TableSchema.class); + } + } + } + /** * Uses the specified schema for rows to be written. * @@ -865,20 +906,61 @@ public TableDestination apply(ValueInSingleWindow value) { * {@link CreateDisposition#CREATE_IF_NEEDED}. */ public Write withSchema(TableSchema schema) { - return toBuilder() - .setJsonSchema(StaticValueProvider.of(BigQueryHelpers.toJsonString(schema))) - .build(); + return withSchemaFunction(ConstantSchemaFunction.fromTableSchema( + StaticValueProvider.of(schema))); } /** - * Use the specified schema for rows to be written. + * Same as {@link #withSchema(TableSchema)} but using a deferred {@link ValueProvider}. */ public Write withSchema(ValueProvider schema) { + return withSchemaFunction(ConstantSchemaFunction.fromTableSchema(schema)); + } + + /** + * Similar to {@link #withSchema(TableSchema)} but takes in a JSON-serialized + * {@link TableSchema}. + */ + public Write withJsonSchema(String jsonSchema) { + return withSchemaFunction(ConstantSchemaFunction.fromJsonSchema( + StaticValueProvider.of(jsonSchema))); + } + + /** + * Same as {@link #withSchema(String)} but using a deferred {@link ValueProvider}. + */ + public Write withJsonSchema(ValueProvider jsonSchema) { + return withSchemaFunction(ConstantSchemaFunction.fromJsonSchema(jsonSchema)); + } + + /** + * Use the specified {@link SchemaFunction} to assign schemas for tables. + * + *

This function is ignored if the create disposition is set to + * {@link CreateDisposition#CREATE_NEVER}. + */ + public Write withSchemaFunction(SchemaFunction schemaFunction) { return toBuilder() - .setJsonSchema(NestedValueProvider.of(schema, new TableSchemaToJsonSchema())) + .setSchemaFunction(schemaFunction) .build(); } + /** + * Allows the schemas for each table to be computed within the pipeline itself. + * + *

The input is a map-valued {@link PCollectionView} mapping string tablespecs to + * JSON-formatted {@link TableSchema}s. + */ + public Write withSchemaFromView(PCollectionView> view) { + return withSchemaFunction(new SchemaFunction() { + @Override + public TableSchema apply(TableDestination input) { + return BigQueryHelpers.fromJsonString(getSideInputValue().get(input.getTableSpec()), + TableSchema.class); + } + }.withSideInput(view)); + } + /** Specifies whether the table should be created if it does not exist. */ public Write withCreateDisposition(CreateDisposition createDisposition) { return toBuilder().setCreateDisposition(createDisposition).build(); @@ -918,7 +1000,8 @@ public void validate(PipelineOptions pipelineOptions) { // Require a schema if creating one or more tables. checkArgument( - getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED || getJsonSchema() != null, + getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED + || getSchemaFunction() != null, "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided."); // The user specified a table. @@ -951,6 +1034,9 @@ public WriteResult expand(PCollection input) { getTableFunction(), getFormatFunction())) .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); + // The transforms always expect to get a non-null schema function. + SchemaFunction schemaFunction = MoreObjects.firstNonNull( + this.getSchemaFunction(), schemaFunction = ConstantSchemaFunction.fromJsonSchema(null)); // When writing an Unbounded PCollection, or when a tablespec function is defined, we use // StreamingInserts and BigQuery's streaming import API. @@ -959,10 +1045,12 @@ public WriteResult expand(PCollection input) { getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + " PCollection."); - - return rowsWithDestination.apply(new StreamingInserts(this)); + return rowsWithDestination.apply(new StreamingInserts( + getCreateDisposition(), schemaFunction).withTestServices(getBigQueryServices())); } else { - return rowsWithDestination.apply(new BatchLoads(this)); + return rowsWithDestination.apply(new BatchLoads(getWriteDisposition(), + getCreateDisposition(), getJsonTableRef(), getTableDescription(), schemaFunction) + .withTestServices(getBigQueryServices())); } } @@ -977,9 +1065,14 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull(DisplayData.item("table", getJsonTableRef()) - .withLabel("Table Reference")) - .addIfNotNull(DisplayData.item("schema", getJsonSchema()) - .withLabel("Table Schema")); + .withLabel("Table Reference")); + if (getSchemaFunction() instanceof ConstantSchemaFunction) { + ValueProvider jsonSchema = ((ConstantSchemaFunction) getSchemaFunction()) + .jsonSchema; + builder.addIfNotNull(DisplayData.item("schema", jsonSchema).withLabel("Table Schema")); + } else { + builder.add(DisplayData.item("schema", "Custom Schema Function").withLabel("Table Schema")); + } if (getTableFunction() != null) { builder.add(DisplayData.item("tableFn", getTableFunction().getClass()) @@ -997,12 +1090,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Table Description"), ""); } - /** Returns the table schema. */ - public TableSchema getSchema() { - return BigQueryHelpers.fromJsonString( - getJsonSchema() == null ? null : getJsonSchema().get(), TableSchema.class); - } - /** * Returns the table to write, or {@code null} if writing with {@code tableFunction}. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index a377af7fccc5a..83f48da896f1a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -22,8 +22,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; import java.io.IOException; import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -32,9 +35,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; /** * Creates any tables needed before performing streaming writes to the tables. This is a side-effect @@ -45,7 +49,7 @@ public class CreateTables PCollection>, PCollection>> { private final CreateDisposition createDisposition; private final BigQueryServices bqServices; - private final SerializableFunction schemaFunction; + private final SchemaFunction schemaFunction; /** * The list of tables created so far, so we don't try the creation each time. @@ -55,16 +59,14 @@ public class CreateTables private static Set createdTables = Collections.newSetFromMap(new ConcurrentHashMap()); - public CreateTables( - CreateDisposition createDisposition, - SerializableFunction schemaFunction) { + public CreateTables(CreateDisposition createDisposition, SchemaFunction schemaFunction) { this(createDisposition, new BigQueryServicesImpl(), schemaFunction); } private CreateTables( CreateDisposition createDisposition, BigQueryServices bqServices, - SerializableFunction schemaFunction) { + SchemaFunction schemaFunction) { this.createDisposition = createDisposition; this.bqServices = bqServices; this.schemaFunction = schemaFunction; @@ -77,20 +79,38 @@ CreateTables withTestServices(BigQueryServices bqServices) { @Override public PCollection> expand( PCollection> input) { + List>> sideInputs = Lists.newArrayList(); + if (schemaFunction.getSideInput() != null) { + sideInputs.add(schemaFunction.getSideInput()); + } return input.apply( ParDo.of( new DoFn, KV>() { + SchemaFunction schemaFunctionCopy; + + @StartBundle + public void startBundle(Context c) { + this.schemaFunctionCopy = SerializableUtils.clone(schemaFunction); + } + @ProcessElement public void processElement(ProcessContext context) throws InterruptedException, IOException { + if (schemaFunctionCopy.getSideInput() != null) { + schemaFunctionCopy.setSideInputValue( + context.sideInput(schemaFunctionCopy.getSideInput())); + } + BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - possibleCreateTable(options, context.element().getKey()); + possibleCreateTable(options, context.element().getKey(), schemaFunctionCopy); context.output(context.element()); } - })); + }) + .withSideInputs(sideInputs)); } - private void possibleCreateTable(BigQueryOptions options, TableDestination tableDestination) + private void possibleCreateTable(BigQueryOptions options, TableDestination tableDestination, + SchemaFunction schemaFunction) throws InterruptedException, IOException { String tableSpec = tableDestination.getTableSpec(); TableReference tableReference = tableDestination.getTableReference(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java new file mode 100644 index 0000000000000..84dd85574b273 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java @@ -0,0 +1,58 @@ +/* + * 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.bigquery; + +import com.google.api.services.bigquery.model.TableSchema; +import java.util.Map; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A user-suppled schema function, mapping a {@link TableDestination} to a {@link TableSchema}. + * + *

A function can declare that it wants access to a map-valued {@link PCollectionView} and use + * that map to assign {@link TableSchema}s to tables. This is used by + * {@link BigQueryIO.Write#withSchemaFromView}. In that case the map is assumed to map string + * tablespecs to json-formatted schemas. + */ +public abstract class SchemaFunction + implements SerializableFunction { + private PCollectionView> sideInput; + private Map materialized; + + public SchemaFunction() { + } + + public SchemaFunction withSideInput(PCollectionView> sideInput) { + this.sideInput = sideInput; + return this; + } + + public PCollectionView> getSideInput() { + return sideInput; + } + + public Map getSideInputValue() { + return materialized; + } + + void setSideInputValue(Map value) { + materialized = value; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index efd9c313614a2..3ad2d1167d425 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -19,42 +19,33 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; /** -* PTransform that performs streaming BigQuery write. To increase consistency, -* it leverages BigQuery best effort de-dup mechanism. + * PTransform that performs streaming BigQuery write. To increase consistency, it leverages + * BigQuery's best effort de-dup mechanism. */ -public class StreamingInserts extends - PTransform>, WriteResult> { - private final Write write; +public class StreamingInserts + extends PTransform>, WriteResult> { + private BigQueryServices bigQueryServices; + private final CreateDisposition createDisposition; + private final SchemaFunction schemaFunction; - private static class ConstantSchemaFunction implements - SerializableFunction { - private final @Nullable String jsonSchema; - - ConstantSchemaFunction(TableSchema schema) { - this.jsonSchema = BigQueryHelpers.toJsonString(schema); - } - - @Override - @Nullable - public TableSchema apply(TableDestination table) { - return BigQueryHelpers.fromJsonString(jsonSchema, TableSchema.class); - } + /** Constructor. */ + StreamingInserts(CreateDisposition createDisposition, SchemaFunction schemaFunction) { + this.createDisposition = createDisposition; + this.schemaFunction = schemaFunction; + this.bigQueryServices = new BigQueryServicesImpl(); } - /** Constructor. */ - StreamingInserts(Write write) { - this.write = write; + StreamingInserts withTestServices(BigQueryServices bigQueryServices) { + this.bigQueryServices = bigQueryServices; + return this; } @Override @@ -64,16 +55,12 @@ protected Coder getDefaultOutputCoder() { @Override public WriteResult expand(PCollection> input) { - // Since BigQueryIO.java does not yet have support for per-table schemas, inject a constant - // schema function here. If no schema is specified, this function will return null. - SerializableFunction schemaFunction = - new ConstantSchemaFunction(write.getSchema()); - - PCollection> writes = input - .apply("CreateTables", new CreateTables(write.getCreateDisposition(), schemaFunction) - .withTestServices(write.getBigQueryServices())); + PCollection> writes = + input.apply( + "CreateTables", + new CreateTables(createDisposition, schemaFunction) + .withTestServices(bigQueryServices)); - return writes.apply(new StreamingWriteTables() - .withTestServices(write.getBigQueryServices())); + return writes.apply(new StreamingWriteTables().withTestServices(bigQueryServices)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index f3368498a0e69..b53d780938090 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -39,7 +39,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.FileIOChannelFactory; import org.apache.beam.sdk.util.GcsIOChannelFactory; @@ -47,25 +46,25 @@ import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Writes partitions to BigQuery tables. * - *

The input is a list of files corresponding to each partition of a table. These files are - * load into a temporary table (or into the final table if there is only one partition). The output - * is a {@link KV} mapping each final table to a list of the temporary tables containing its data. + *

The input is a list of files corresponding to each partition of a table. These files are load + * into a temporary table (or into the final table if there is only one partition). The output is a + * {@link KV} mapping each final table to a list of the temporary tables containing its data. * *

In the case where all the data in the files fit into a single load job, this transform loads * the data directly into the final table, skipping temporary tables. In this case, the output * {@link KV} maps the final table to itself. */ -class WriteTables extends DoFn, List>, - KV> { +class WriteTables + extends DoFn, List>, KV> { private static final Logger LOG = LoggerFactory.getLogger(WriteTables.class); private final boolean singlePartition; @@ -74,7 +73,8 @@ class WriteTables extends DoFn, List>, private final String tempFilePrefix; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; - private final SerializableFunction schemaFunction; + private final SchemaFunction baseSchemaFunction; + private SchemaFunction schemaFunctionCopy; public WriteTables( boolean singlePartition, @@ -83,30 +83,39 @@ public WriteTables( String tempFilePrefix, WriteDisposition writeDisposition, CreateDisposition createDisposition, - SerializableFunction schemaFunction) { + SchemaFunction schemaFunction) { this.singlePartition = singlePartition; this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.tempFilePrefix = tempFilePrefix; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; - this.schemaFunction = schemaFunction; + this.baseSchemaFunction = schemaFunction; + } + + @StartBundle + public void startBundle(Context c) { + schemaFunctionCopy = SerializableUtils.clone(baseSchemaFunction); } @ProcessElement public void processElement(ProcessContext c) throws Exception { + if (schemaFunctionCopy.getSideInput() != null) { + schemaFunctionCopy.setSideInputValue(c.sideInput(schemaFunctionCopy.getSideInput())); + } + TableDestination tableDestination = c.element().getKey().getKey(); Integer partition = c.element().getKey().getShardNumber(); List partitionFiles = Lists.newArrayList(c.element().getValue()); - String jobIdPrefix = BigQueryHelpers.createJobId( - c.sideInput(jobIdToken), tableDestination, partition); + String jobIdPrefix = + BigQueryHelpers.createJobId(c.sideInput(jobIdToken), tableDestination, partition); TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { ref.setTableId(jobIdPrefix); } - TableSchema schema = (schemaFunction != null) ? schemaFunction.apply(tableDestination) : null; + TableSchema schema = schemaFunctionCopy.apply(tableDestination); load( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), @@ -131,22 +140,22 @@ private void load( List gcsUris, WriteDisposition writeDisposition, CreateDisposition createDisposition, - @Nullable String tableDescription) throws InterruptedException, IOException { - JobConfigurationLoad loadConfig = new JobConfigurationLoad() - .setDestinationTable(ref) - .setSchema(schema) - .setSourceUris(gcsUris) - .setWriteDisposition(writeDisposition.name()) - .setCreateDisposition(createDisposition.name()) - .setSourceFormat("NEWLINE_DELIMITED_JSON"); + @Nullable String tableDescription) + throws InterruptedException, IOException { + JobConfigurationLoad loadConfig = + new JobConfigurationLoad() + .setDestinationTable(ref) + .setSchema(schema) + .setSourceUris(gcsUris) + .setWriteDisposition(writeDisposition.name()) + .setCreateDisposition(createDisposition.name()) + .setSourceFormat("NEWLINE_DELIMITED_JSON"); String projectId = ref.getProjectId(); Job lastFailedLoadJob = null; for (int i = 0; i < Write.MAX_RETRY_JOBS; ++i) { String jobId = jobIdPrefix + "-" + i; - JobReference jobRef = new JobReference() - .setProjectId(projectId) - .setJobId(jobId); + JobReference jobRef = new JobReference().setProjectId(projectId).setJobId(jobId); jobService.startLoadJob(jobRef, loadConfig); Job loadJob = jobService.pollJob(jobRef, Write.LOAD_JOB_POLL_MAX_RETRIES); Status jobStatus = BigQueryHelpers.parseStatus(loadJob); @@ -157,31 +166,31 @@ private void load( } return; case UNKNOWN: - throw new RuntimeException(String.format( - "UNKNOWN status of load job [%s]: %s.", jobId, - BigQueryHelpers.jobToPrettyString(loadJob))); + throw new RuntimeException( + String.format( + "UNKNOWN status of load job [%s]: %s.", + jobId, BigQueryHelpers.jobToPrettyString(loadJob))); case FAILED: lastFailedLoadJob = loadJob; continue; default: - throw new IllegalStateException(String.format( - "Unexpected status [%s] of load job: %s.", - jobStatus, BigQueryHelpers.jobToPrettyString(loadJob))); + throw new IllegalStateException( + String.format( + "Unexpected status [%s] of load job: %s.", + jobStatus, BigQueryHelpers.jobToPrettyString(loadJob))); } } - throw new RuntimeException(String.format( - "Failed to create load job with id prefix %s, " - + "reached max retries: %d, last failed load job: %s.", - jobIdPrefix, - Write.MAX_RETRY_JOBS, - BigQueryHelpers.jobToPrettyString(lastFailedLoadJob))); + throw new RuntimeException( + String.format( + "Failed to create load job with id prefix %s, " + + "reached max retries: %d, last failed load job: %s.", + jobIdPrefix, + Write.MAX_RETRY_JOBS, + BigQueryHelpers.jobToPrettyString(lastFailedLoadJob))); } static void removeTemporaryFiles( - PipelineOptions options, - String tempFilePrefix, - Collection files) - throws IOException { + PipelineOptions options, String tempFilePrefix, Collection files) throws IOException { IOChannelFactory factory = IOChannelUtils.getFactory(tempFilePrefix); if (factory instanceof GcsIOChannelFactory) { GcsUtil gcsUtil = new GcsUtilFactory().create(options); @@ -203,8 +212,7 @@ static void removeTemporaryFiles( public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("tempFilePrefix", tempFilePrefix) - .withLabel("Temporary File Prefix")); + builder.addIfNotNull( + DisplayData.item("tempFilePrefix", tempFilePrefix).withLabel("Temporary File Prefix")); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index baa5621384611..d5bd66f0e91c9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -23,6 +23,7 @@ import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.toJsonString; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -77,6 +78,7 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.ConstantSchemaFunction; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; @@ -204,7 +206,11 @@ private void checkWriteObjectWithValidate( assertEquals(project, write.getTable().get().getProjectId()); assertEquals(dataset, write.getTable().get().getDatasetId()); assertEquals(table, write.getTable().get().getTableId()); - assertEquals(schema, write.getSchema()); + if (schema == null) { + assertNull(write.getSchemaFunction()); + } else { + assertEquals(schema, write.getSchemaFunction().apply(null)); + } assertEquals(createDisposition, write.getCreateDisposition()); assertEquals(writeDisposition, write.getWriteDisposition()); assertEquals(tableDescription, write.getTableDescription()); @@ -569,8 +575,6 @@ public Instant maxTimestamp() { return GlobalWindow.INSTANCE.maxTimestamp(); } - // The following methods are only needed due to BEAM-1022. Once this issue is fixed, we will - // no longer need these. @Override public boolean equals(Object other) { if (other instanceof PartitionedGlobalWindow) { @@ -633,7 +637,7 @@ public void testWriteWithDynamicTables(boolean streaming) throws Exception { // Create a windowing strategy that puts the input into five different windows depending on // record value. - WindowFn window = new PartitionedGlobalWindows( + WindowFn windowFn = new PartitionedGlobalWindows( new SerializableFunction() { @Override public String apply(Integer i) { @@ -642,24 +646,44 @@ public String apply(Integer i) { } ); + final Map targetTables = Maps.newHashMap(); + Map schemas = Maps.newHashMap(); + for (int i = 0; i < 5; i++) { + TableDestination destination = new TableDestination("project-id:dataset-id" + + ".table-id-" + i, ""); + targetTables.put(i, destination); + // Make sure each target table has its own custom table. + schemas.put(destination.getTableSpec(), + BigQueryHelpers.toJsonString(new TableSchema().setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER"), + new TableFieldSchema().setName("custom_" + i).setType("STRING"))))); + } + SerializableFunction, TableDestination> tableFunction = new SerializableFunction, TableDestination>() { @Override public TableDestination apply(ValueInSingleWindow input) { PartitionedGlobalWindow window = (PartitionedGlobalWindow) input.getWindow(); - // Check that we can access the element as well here. + // Check that we can access the element as well here and that it matches the window. checkArgument(window.value.equals(Integer.toString(input.getValue() % 5)), "Incorrect element"); - return new TableDestination("project-id:dataset-id.table-id-" + window.value, ""); + return targetTables.get(input.getValue() % 5); } }; Pipeline p = TestPipeline.create(bqOptions); - PCollection input = p.apply(Create.of(inserts)); + PCollection input = p.apply("CreateSource", Create.of(inserts)); if (streaming) { input = input.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } - input.apply(Window.into(window)) + + PCollectionView> schemasView = + p.apply("CreateSchemaMap", Create.of(schemas)) + .apply("ViewSchemaAsMap", View.asMap()); + + input.apply(Window.into(windowFn)) .apply(BigQueryIO.write() .to(tableFunction) .withFormatFunction(new SerializableFunction() { @@ -668,35 +692,27 @@ public TableRow apply(Integer i) { return new TableRow().set("name", "number" + i).set("number", i); }}) .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) - .withSchema(new TableSchema().setFields( - ImmutableList.of( - new TableFieldSchema().setName("name").setType("STRING"), - new TableFieldSchema().setName("number").setType("INTEGER")))) + .withSchemaFromView(schemasView) .withTestServices(fakeBqServices) .withoutValidation()); p.run(); - - assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-0"), - containsInAnyOrder( - new TableRow().set("name", "number0").set("number", 0), - new TableRow().set("name", "number5").set("number", 5))); - assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-1"), - containsInAnyOrder( - new TableRow().set("name", "number1").set("number", 1), - new TableRow().set("name", "number6").set("number", 6))); - assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-2"), - containsInAnyOrder( - new TableRow().set("name", "number2").set("number", 2), - new TableRow().set("name", "number7").set("number", 7))); - assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-3"), - containsInAnyOrder( - new TableRow().set("name", "number3").set("number", 3), - new TableRow().set("name", "number8").set("number", 8))); - assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-4"), - containsInAnyOrder( - new TableRow().set("name", "number4").set("number", 4), - new TableRow().set("name", "number9").set("number", 9))); + for (int i = 0; i < 5; ++i) { + String tableId = String.format("table-id-%d", i); + String tableSpec = String.format("project-id:dataset-id.%s", tableId); + + // Verify that table was created with the correct schema. + assertThat(BigQueryHelpers.toJsonString( + datasetService.getTable(new TableReference().setProjectId("project-id") + .setDatasetId("dataset-id").setTableId(tableId)).getSchema()), + equalTo(schemas.get(tableSpec))); + + // Verify that the table has the expected contents. + assertThat(datasetService.getAllRows("project-id", "dataset-id", tableId), + containsInAnyOrder( + new TableRow().set("name", String.format("number%d", i)).set("number", i), + new TableRow().set("name", String.format("number%d", i + 5)).set("number", i + 5))); + } } @Test @@ -1741,7 +1757,7 @@ public void testWriteTables() throws Exception { tempFilePrefix, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, - null); + ConstantSchemaFunction.fromJsonSchema(null)); DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); From ada84d76f1e1dc5fa3cd279c0b83db75ab784850 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Thu, 27 Apr 2017 23:44:52 -0700 Subject: [PATCH 2/6] Add DynamicDestinations - allow users to map schemas based on their own types. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 97 +++++- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 95 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 315 ++++++++++++------ .../sdk/io/gcp/bigquery/CreateTables.java | 86 +++-- .../io/gcp/bigquery/DynamicDestinations.java | 105 ++++++ .../sdk/io/gcp/bigquery/PrepareWrite.java | 33 +- .../sdk/io/gcp/bigquery/SchemaFunction.java | 58 ---- .../sdk/io/gcp/bigquery/StreamingInserts.java | 17 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 25 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 28 +- 10 files changed, 539 insertions(+), 320 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 47c598982a96e..4db14848543f5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -20,9 +20,11 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; import java.util.List; @@ -37,6 +39,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Distinct; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -49,6 +53,7 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -57,27 +62,28 @@ import org.apache.beam.sdk.values.TupleTagList; /** PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ -class BatchLoads extends PTransform>, WriteResult> { +class BatchLoads + extends PTransform>, WriteResult> { private BigQueryServices bigQueryServices; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; private final ValueProvider singletonJsonTableRef; private final String singletonTableDescription; - private final SchemaFunction schemaFunction; + private final DynamicDestinations dynamicDestinations; BatchLoads(WriteDisposition writeDisposition, CreateDisposition createDisposition, ValueProvider singletonJsonTableRef, String singletonTableDescription, - SchemaFunction schemaFunction) { + DynamicDestinations dynamicDestinations) { bigQueryServices = new BigQueryServicesImpl(); this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; this.singletonJsonTableRef = singletonJsonTableRef; this.singletonTableDescription = singletonTableDescription; - this.schemaFunction = schemaFunction; + this.dynamicDestinations = dynamicDestinations; } - BatchLoads withTestServices(BigQueryServices bigQueryServices) { + BatchLoads withTestServices(BigQueryServices bigQueryServices) { this.bigQueryServices = bigQueryServices; return this; } @@ -103,7 +109,7 @@ public void validate(PipelineOptions options) { } @Override - public WriteResult expand(PCollection> input) { + public WriteResult expand(PCollection> input) { Pipeline p = input.getPipeline(); BigQueryOptions options = p.getOptions().as(BigQueryOptions.class); @@ -138,16 +144,74 @@ public String apply(String input) { })) .apply(View.asSingleton()); - PCollection> inputInGlobalWindow = + PCollection> inputInGlobalWindow = input.apply( "rewindowIntoGlobal", - Window.>into(new GlobalWindows()) + Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); + final TupleTag> elementsOutputTag = new TupleTag<>(); + final TupleTag> tableSchemasTag = new TupleTag<>(); + + List> resolveSideInputs = Lists.newArrayList(); + if (dynamicDestinations.getSideInput() != null) { + resolveSideInputs.add(dynamicDestinations.getSideInput()); + } + PCollectionTuple resolvedInputTuple = inputInGlobalWindow.apply( + "resolveInput", + ParDo.of(new DoFn, KV>() { + DynamicDestinations dynamicDestinationsCopy; + + @StartBundle + public void startBundle(Context c) { + this.dynamicDestinationsCopy = SerializableUtils.clone(dynamicDestinations); + } + + @ProcessElement + public void processElement(ProcessContext context) { + if (dynamicDestinationsCopy.getSideInput() != null) { + dynamicDestinationsCopy.setSideInputValue( + context.sideInput(dynamicDestinationsCopy.getSideInput())); + } + // Convert to a TableDestination and output. + TableDestination tableDestination = + dynamicDestinationsCopy.getTable(context.element().getKey()); + TableReference tableReference = tableDestination.getTableReference(); + if (Strings.isNullOrEmpty(tableReference.getProjectId())) { + tableReference.setProjectId( + context.getPipelineOptions().as(BigQueryOptions.class).getProject()); + tableDestination = + new TableDestination( + tableReference, tableDestination.getTableDescription()); + } + context.output(elementsOutputTag, KV.of(tableDestination, context.element().getValue())); + + // Match the schema, and output the mapping. + TableSchema tableSchema = dynamicDestinationsCopy.getSchema(context.element().getKey()); + if (tableSchema != null) { + context.output(tableSchemasTag, KV.of(tableDestination.getTableSpec(), + BigQueryHelpers.toJsonString(tableSchema))); + } + } + }).withSideInputs(resolveSideInputs) + .withOutputTags(elementsOutputTag, TupleTagList.of(tableSchemasTag))); + + // We now have input in terms of TableDestinations. + PCollection> resolvedInput = + resolvedInputTuple.get(elementsOutputTag) + .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); + + // Produce the schema map to be used as a side input to following stages. + PCollectionView> schemaMapView = + resolvedInputTuple.get(tableSchemasTag) + .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) + .apply(Distinct.>create()) + .apply(View.asMap()); + // PCollection of filename, file byte size, and table destination. PCollection results = - inputInGlobalWindow + resolvedInput .apply("WriteBundlesToFiles", ParDo.of(new WriteBundlesToFiles(tempFilePrefix))) .setCoder(WriteBundlesToFiles.ResultCoder.of()); @@ -176,11 +240,8 @@ public String apply(String input) { .withSideInputs(resultsView) .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); - List> writeTablesSideInputs = Lists.newArrayList(); - writeTablesSideInputs.add(jobIdTokenView); - if (schemaFunction.getSideInput() != null) { - writeTablesSideInputs.add(schemaFunction.getSideInput()); - } + List> writeTablesSideInputs = + ImmutableList.of(jobIdTokenView, schemaMapView); Coder, List>> partitionsCoder = KvCoder.of( @@ -204,10 +265,10 @@ public String apply(String input) { false, bigQueryServices, jobIdTokenView, + schemaMapView, tempFilePrefix, WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - schemaFunction)) + CreateDisposition.CREATE_IF_NEEDED)) .withSideInputs(writeTablesSideInputs)); // This view maps each final table destination to the set of temporary partitioned tables @@ -241,10 +302,10 @@ public String apply(String input) { true, bigQueryServices, jobIdTokenView, + schemaMapView, tempFilePrefix, writeDisposition, - createDisposition, - schemaFunction)) + createDisposition)) .withSideInputs(writeTablesSideInputs)); return WriteResult.in(input.getPipeline()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index e04361c960ef1..d8b1136ea1f27 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -41,9 +41,7 @@ import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; -/** - * A set of helper functions and classes used by {@link BigQueryIO}. - */ +/** A set of helper functions and classes used by {@link BigQueryIO}. */ public class BigQueryHelpers { private static final String RESOURCE_NOT_FOUND_ERROR = "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline" @@ -55,9 +53,7 @@ public class BigQueryHelpers { + " an earlier stage of the pipeline, this validation can be disabled using" + " #withoutValidation."; - /** - * Status of a BigQuery job or request. - */ + /** Status of a BigQuery job or request. */ enum Status { SUCCEEDED, FAILED, @@ -65,20 +61,15 @@ enum Status { } @Nullable - /** - * Return a displayable string representation for a {@link TableReference}. - */ - static ValueProvider displayTable( - @Nullable ValueProvider table) { + /** Return a displayable string representation for a {@link TableReference}. */ + static ValueProvider displayTable(@Nullable ValueProvider table) { if (table == null) { return null; } return NestedValueProvider.of(table, new TableRefToTableSpec()); } - /** - * Returns a canonical string representation of the {@link TableReference}. - */ + /** Returns a canonical string representation of the {@link TableReference}. */ public static String toTableSpec(TableReference ref) { StringBuilder sb = new StringBuilder(); if (ref.getProjectId() != null) { @@ -100,8 +91,8 @@ static List getOrCreateMapListValue(Map> map, K key) { } /** - * Parse a table specification in the form - * {@code "[project_id]:[dataset_id].[table_id]"} or {@code "[dataset_id].[table_id]"}. + * Parse a table specification in the form {@code "[project_id]:[dataset_id].[table_id]"} or + * {@code "[dataset_id].[table_id]"}. * *

If the project id is omitted, the default project id is used. */ @@ -110,7 +101,8 @@ public static TableReference parseTableSpec(String tableSpec) { if (!match.matches()) { throw new IllegalArgumentException( "Table reference is not in [project_id]:[dataset_id].[table_id] " - + "format: " + tableSpec); + + "format: " + + tableSpec); } TableReference ref = new TableReference(); @@ -164,8 +156,7 @@ static T fromJsonString(String json, Class clazz) { return BigQueryIO.JSON_FACTORY.fromString(json, clazz); } catch (IOException e) { throw new RuntimeException( - String.format("Cannot deserialize %s from a JSON string: %s.", clazz, json), - e); + String.format("Cannot deserialize %s from a JSON string: %s.", clazz, json), e); } } @@ -178,9 +169,7 @@ static String randomUUIDString() { return UUID.randomUUID().toString().replaceAll("-", ""); } - static void verifyTableNotExistOrEmpty( - DatasetService datasetService, - TableReference tableRef) { + static void verifyTableNotExistOrEmpty(DatasetService datasetService, TableReference tableRef) { try { if (datasetService.getTable(tableRef) != null) { checkState( @@ -193,8 +182,7 @@ static void verifyTableNotExistOrEmpty( Thread.currentThread().interrupt(); } throw new RuntimeException( - "unable to confirm BigQuery table emptiness for table " - + toTableSpec(tableRef), e); + "unable to confirm BigQuery table emptiness for table " + toTableSpec(tableRef), e); } } @@ -206,12 +194,12 @@ static void verifyDatasetPresence(DatasetService datasetService, TableReference if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) { throw new IllegalArgumentException( String.format(RESOURCE_NOT_FOUND_ERROR, "dataset", toTableSpec(table)), e); - } else if (e instanceof RuntimeException) { + } else if (e instanceof RuntimeException) { throw (RuntimeException) e; } else { throw new RuntimeException( - String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "dataset", - toTableSpec(table)), + String.format( + UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "dataset", toTableSpec(table)), e); } } @@ -225,12 +213,13 @@ static void verifyTablePresence(DatasetService datasetService, TableReference ta if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) { throw new IllegalArgumentException( String.format(RESOURCE_NOT_FOUND_ERROR, "table", toTableSpec(table)), e); - } else if (e instanceof RuntimeException) { + } else if (e instanceof RuntimeException) { throw (RuntimeException) e; } else { throw new RuntimeException( - String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "table", - toTableSpec(table)), e); + String.format( + UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "table", toTableSpec(table)), + e); } } } @@ -248,8 +237,7 @@ static String createJobId(String prefix, TableDestination tableDestination, int } @VisibleForTesting - static class JsonSchemaToTableSchema - implements SerializableFunction { + static class JsonSchemaToTableSchema implements SerializableFunction { @Override public TableSchema apply(String from) { return fromJsonString(from, TableSchema.class); @@ -257,48 +245,49 @@ public TableSchema apply(String from) { } @VisibleForTesting - static class BeamJobUuidToBigQueryJobUuid - implements SerializableFunction { + static class BeamJobUuidToBigQueryJobUuid implements SerializableFunction { @Override public String apply(String from) { return "beam_job_" + from; } } - static class TableSchemaToJsonSchema - implements SerializableFunction { + static class TableSchemaToJsonSchema implements SerializableFunction { @Override public String apply(TableSchema from) { return toJsonString(from); } } - static class JsonTableRefToTableRef - implements SerializableFunction { + static class JsonTableRefToTableRef implements SerializableFunction { @Override public TableReference apply(String from) { return fromJsonString(from, TableReference.class); } } - static class TableRefToTableSpec - implements SerializableFunction { + static class JsonTableRefToTableSpec implements SerializableFunction { + @Override + public String apply(String from) { + return toTableSpec(fromJsonString(from, TableReference.class)); + } + } + + static class TableRefToTableSpec implements SerializableFunction { @Override public String apply(TableReference from) { return toTableSpec(from); } } - static class TableRefToJson - implements SerializableFunction { + static class TableRefToJson implements SerializableFunction { @Override public String apply(TableReference from) { return toJsonString(from); } } - static class TableRefToProjectId - implements SerializableFunction { + static class TableRefToProjectId implements SerializableFunction { @Override public String apply(TableReference from) { return from.getProjectId(); @@ -306,8 +295,7 @@ public String apply(TableReference from) { } @VisibleForTesting - static class TableSpecToTableRef - implements SerializableFunction { + static class TableSpecToTableRef implements SerializableFunction { @Override public TableReference apply(String from) { return parseTableSpec(from); @@ -315,8 +303,7 @@ public TableReference apply(String from) { } @VisibleForTesting - static class CreatePerBeamJobUuid - implements SerializableFunction { + static class CreatePerBeamJobUuid implements SerializableFunction { private final String stepUuid; CreatePerBeamJobUuid(String stepUuid) { @@ -330,8 +317,7 @@ public String apply(String jobUuid) { } @VisibleForTesting - static class CreateJsonTableRefFromUuid - implements SerializableFunction { + static class CreateJsonTableRefFromUuid implements SerializableFunction { private final String executingProject; CreateJsonTableRefFromUuid(String executingProject) { @@ -342,10 +328,11 @@ static class CreateJsonTableRefFromUuid public TableReference apply(String jobUuid) { String queryTempDatasetId = "temp_dataset_" + jobUuid; String queryTempTableId = "temp_table_" + jobUuid; - TableReference queryTempTableRef = new TableReference() - .setProjectId(executingProject) - .setDatasetId(queryTempDatasetId) - .setTableId(queryTempTableId); + TableReference queryTempTableRef = + new TableReference() + .setProjectId(executingProject) + .setDatasetId(queryTempDatasetId) + .setTableId(queryTempTableId); return queryTempTableRef; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 069e629dbb029..1c068b914bcbb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -30,7 +30,6 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -41,6 +40,7 @@ import java.util.regex.Pattern; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -50,6 +50,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.CreateJsonTableRefFromUuid; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.CreatePerBeamJobUuid; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableRef; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableSpec; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToJson; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef; @@ -72,6 +73,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -184,20 +186,20 @@ * {@link BigQueryIO.Write#withFormatFunction} to convert each element into a {@link TableRow} * object. * - *

Each table can be given it's own schema using {@link BigQueryIO.Write#withSchemaFunction}. - * {@link SchemaFunction} is a user-defined function that maps a destination table to a BigQuery - * schema. Note that this function must return a valid schema for all tables (unless the create - * disposition is CREATE_NEVER), otherwise the pipeline will fail to create tables. - * *

Per-table schemas can also be provided using {@link BigQueryIO.Write#withSchemaFromView}. * This allows you the schemas to be calculated based on a previous pipeline stage or statically * via a {@link org.apache.beam.sdk.transforms.Create} transform. This method expects to receive a * map-valued {@link PCollectionView}, mapping table specifications (project:dataset.table-id), * to JSON formatted {@link TableSchema} objects. All destination tables must be present in this * map, or the pipeline will fail to create tables. Care should be taken if the map value is based - * on a triggered aggregation over and unbounded {@link PCollection}. This method can also be useful - * when writing to a single table, as it allows a previous stage to calculate the schema (possibly - * based on the full collection of records being written to BigQuery). + * on a triggered aggregation over and unbounded {@link PCollection}; the side input will contain + * the entire history of all table schemas ever generated, which might blow up memory usage. + * This method can also be useful when writing to a single table, as it allows a previous stage + * to calculate the schema (possibly based on the full collection of records being written to + * BigQuery). + * + *

For the most general form of dynamic table destinations and schemas, look at + * {@link BigQueryIO.Write#to(DynamicDestinations)}. * *

Permissions

* @@ -698,11 +700,17 @@ public abstract static class Write extends PTransform, WriteRe static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; @Nullable abstract ValueProvider getJsonTableRef(); + + /** */ @Nullable abstract SerializableFunction, TableDestination> getTableFunction(); @Nullable abstract SerializableFunction getFormatFunction(); - /** Table schema. The schema is required only if the table does not exist. */ - @Nullable abstract SchemaFunction getSchemaFunction(); + /** */ + @Nullable abstract DynamicDestinations getDynamicDestinations(); + /** */ + @Nullable abstract PCollectionView> getSchemaFromView(); + /** */ + @Nullable abstract ValueProvider getJsonSchema(); abstract CreateDisposition getCreateDisposition(); abstract WriteDisposition getWriteDisposition(); /** Table description. Default is empty. */ @@ -720,7 +728,9 @@ abstract Builder setTableFunction( SerializableFunction, TableDestination> tableFunction); abstract Builder setFormatFunction( SerializableFunction formatFunction); - abstract Builder setSchemaFunction(SchemaFunction schemaFunction); + abstract Builder setDynamicDestinations(DynamicDestinations dynamicDestinations); + abstract Builder setSchemaFromView(PCollectionView> view); + abstract Builder setJsonSchema(ValueProvider jsonSchema); abstract Builder setCreateDisposition(CreateDisposition createDisposition); abstract Builder setWriteDisposition(WriteDisposition writeDisposition); abstract Builder setTableDescription(String tableDescription); @@ -797,13 +807,6 @@ public enum WriteDisposition { WRITE_EMPTY } - /** Ensures that methods of the to() family are called at most once. */ - private void ensureToNotCalledYet() { - checkState( - getJsonTableRef() == null && getTable() == null - && getTableFunction() == null, "to() already called"); - } - /** * Writes to the given table, specified in the format described in * {@link BigQueryHelpers#parseTableSpec}. @@ -819,14 +822,11 @@ public Write to(TableReference table) { /** Same as {@link #to(String)}, but with a {@link ValueProvider}. */ public Write to(ValueProvider tableSpec) { - ensureToNotCalledYet(); - String tableDescription = getTableDescription(); return toBuilder() .setJsonTableRef( NestedValueProvider.of( NestedValueProvider.of(tableSpec, new TableSpecToTableRef()), new TableRefToJson())) - .setTableFunction(new ConstantTableFunction(tableSpec, tableDescription)) .build(); } @@ -836,10 +836,16 @@ public Write to(ValueProvider tableSpec) { */ public Write to( SerializableFunction, TableDestination> tableFunction) { - ensureToNotCalledYet(); return toBuilder().setTableFunction(tableFunction).build(); } + /** + * Writes to the table and schema specified by the {@link DynamicDestinations} object. + */ + public Write to(DynamicDestinations dynamicDestinations) { + return toBuilder().setDynamicDestinations(dynamicDestinations).build(); + } + /** * Formats the user's type into a {@link TableRow} to be written to BigQuery. */ @@ -847,54 +853,149 @@ public Write withFormatFunction(SerializableFunction formatFunct return toBuilder().setFormatFunction(formatFunction).build(); } - static class ConstantTableFunction implements - SerializableFunction, TableDestination> { + /** + * Always returns a constant table destination. + */ + static class ConstantTableDestinations extends DynamicDestinations { private final ValueProvider tableSpec; private final String tableDescription; - ConstantTableFunction(ValueProvider tableSpec, String tableDescription) { + private ConstantTableDestinations(ValueProvider tableSpec, String tableDescription) { this.tableSpec = tableSpec; this.tableDescription = tableDescription; } + static ConstantTableDestinations fromTableSpec( + ValueProvider tableSpec, String tableDescription) { + return new ConstantTableDestinations(tableSpec, tableDescription); + } + + static ConstantTableDestinations fromJsonTableRef( + ValueProvider jsonTableRef, String tableDescription) { + return new ConstantTableDestinations( + NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableSpec()), tableDescription); + } + @Override - public TableDestination apply(ValueInSingleWindow value) { + public TableDestination getDestination(ValueInSingleWindow element) { return new TableDestination(tableSpec.get(), tableDescription); } + + @Override + public TableSchema getSchema(TableDestination destination) { + return null; + } + + @Override + public TableDestination getTable(TableDestination destination) { + return destination; + } + + @Override + public Coder getDestinationCoder() { + return TableDestinationCoder.of(); + } + } + + /** + * Returns a tables based on a user-supplied function. + */ + static class TableFunctionDestinations extends DynamicDestinations { + private final SerializableFunction, TableDestination> tableFunction; + + private TableFunctionDestinations( + SerializableFunction, TableDestination> tableFunction) { + this.tableFunction = tableFunction; + } + + @Override + public TableDestination getDestination(ValueInSingleWindow element) { + return tableFunction.apply(element); + } + + @Override + public TableSchema getSchema(TableDestination destination) { + return null; + } + + @Override + public TableDestination getTable(TableDestination destination) { + return destination; + } + + @Override + public Coder getDestinationCoder() { + return TableDestinationCoder.of(); + } } /** - * A function that returns the same schema for every table. + * Delegates all calls to an inner instance of {@link DynamicDestinations}. This allows + * subclasses to modify another instance of {@link DynamicDestinations} by subclassing and + * overriding just the methods they want to alter. */ - static class ConstantSchemaFunction extends SchemaFunction { + static class DelegatingDynamicDestinations + extends DynamicDestinations { + private final DynamicDestinations inner; + DelegatingDynamicDestinations(DynamicDestinations inner) { + this.inner = inner; + } + @Override + public DestinationT getDestination(ValueInSingleWindow element) { + return inner.getDestination(element); + } + + @Override + public TableSchema getSchema(DestinationT destination) { + return inner.getSchema(destination); + } + + @Override + public TableDestination getTable(DestinationT destination) { + return inner.getTable(destination); + } + + @Override + public Coder getDestinationCoder() { + return inner.getDestinationCoder(); + } + } + /** + * Returns the same schema for every table. + */ + static class ConstantSchemaDestinations + extends DelegatingDynamicDestinations { private final @Nullable ValueProvider jsonSchema; - private ConstantSchemaFunction(@Nullable ValueProvider jsonSchema) { + ConstantSchemaDestinations(DynamicDestinations inner, + ValueProvider jsonSchema) { + super(inner); this.jsonSchema = jsonSchema; } - public static ConstantSchemaFunction fromTableSchema(ValueProvider tableSchema) { - if (tableSchema.isAccessible()) { - // TableSchema isn't Serializable, so convert to JSON. - return new ConstantSchemaFunction(StaticValueProvider.of( - BigQueryHelpers.toJsonString(tableSchema.get()))); - } else { - return new ConstantSchemaFunction( - NestedValueProvider.of(tableSchema, new TableSchemaToJsonSchema())); - } + @Override + public TableSchema getSchema(TableDestination destination) { + return BigQueryHelpers.fromJsonString(jsonSchema.get(), TableSchema.class); } + } - public static ConstantSchemaFunction fromJsonSchema(ValueProvider jsonSchema) { - return new ConstantSchemaFunction(jsonSchema); + /** + * Takes in a side input mapping tablespec to json table schema, and always returns the + * matching schema from the side input. + */ + static class SchemaFromViewDestinations + extends DelegatingDynamicDestinations { + SchemaFromViewDestinations(DynamicDestinations inner, + PCollectionView> schemaView) { + super(inner); + withSideInput(schemaView); } @Override - public TableSchema apply(TableDestination table) { - if (jsonSchema == null) { - return null; - } else { - return BigQueryHelpers.fromJsonString(jsonSchema.get(), TableSchema.class); - } + public TableSchema getSchema(TableDestination destination) { + Map mapValue = getSideInputValue(); + return BigQueryHelpers.fromJsonString(mapValue.get(destination.getTableSpec()), + TableSchema.class); } } @@ -906,15 +1007,14 @@ public TableSchema apply(TableDestination table) { * {@link CreateDisposition#CREATE_IF_NEEDED}. */ public Write withSchema(TableSchema schema) { - return withSchemaFunction(ConstantSchemaFunction.fromTableSchema( - StaticValueProvider.of(schema))); + return withJsonSchema(StaticValueProvider.of(BigQueryHelpers.toJsonString(schema))); } /** * Same as {@link #withSchema(TableSchema)} but using a deferred {@link ValueProvider}. */ public Write withSchema(ValueProvider schema) { - return withSchemaFunction(ConstantSchemaFunction.fromTableSchema(schema)); + return withJsonSchema(NestedValueProvider.of(schema, new TableSchemaToJsonSchema())); } /** @@ -922,27 +1022,14 @@ public Write withSchema(ValueProvider schema) { * {@link TableSchema}. */ public Write withJsonSchema(String jsonSchema) { - return withSchemaFunction(ConstantSchemaFunction.fromJsonSchema( - StaticValueProvider.of(jsonSchema))); + return withJsonSchema(StaticValueProvider.of(jsonSchema)); } /** - * Same as {@link #withSchema(String)} but using a deferred {@link ValueProvider}. + * Same as {@link #withJsonSchema(String)} but using a deferred {@link ValueProvider}. */ public Write withJsonSchema(ValueProvider jsonSchema) { - return withSchemaFunction(ConstantSchemaFunction.fromJsonSchema(jsonSchema)); - } - - /** - * Use the specified {@link SchemaFunction} to assign schemas for tables. - * - *

This function is ignored if the create disposition is set to - * {@link CreateDisposition#CREATE_NEVER}. - */ - public Write withSchemaFunction(SchemaFunction schemaFunction) { - return toBuilder() - .setSchemaFunction(schemaFunction) - .build(); + return toBuilder().setJsonSchema(jsonSchema).build(); } /** @@ -952,13 +1039,7 @@ public Write withSchemaFunction(SchemaFunction schemaFunction) { * JSON-formatted {@link TableSchema}s. */ public Write withSchemaFromView(PCollectionView> view) { - return withSchemaFunction(new SchemaFunction() { - @Override - public TableSchema apply(TableDestination input) { - return BigQueryHelpers.fromJsonString(getSideInputValue().get(input.getTableSpec()), - TableSchema.class); - } - }.withSideInput(view)); + return toBuilder().setSchemaFromView(view).build(); } /** Specifies whether the table should be created if it does not exist. */ @@ -991,7 +1072,8 @@ public void validate(PipelineOptions pipelineOptions) { BigQueryOptions options = pipelineOptions.as(BigQueryOptions.class); // We must have a destination to write to! - checkState(getTableFunction() != null, + checkState(getTableFunction() != null || getJsonTableRef() != null + || getDynamicDestinations() != null, "must set the table reference of a BigQueryIO.Write transform"); checkArgument(getFormatFunction() != null, @@ -1001,14 +1083,27 @@ public void validate(PipelineOptions pipelineOptions) { // Require a schema if creating one or more tables. checkArgument( getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED - || getSchemaFunction() != null, + || getJsonSchema() != null || getDynamicDestinations() != null + || getSchemaFromView() != null, "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided."); + checkArgument(getJsonTableRef() == null || getTableFunction() == null, + "Cannot specify both jsonTableRef and tableFunction"); + checkArgument(getJsonTableRef() == null || getDynamicDestinations() == null, + "Cannot specify both jsonTableRef and dynamicDestinations"); + checkArgument(getTableFunction() == null || getDynamicDestinations() == null, + "Cannot specify both tableFunction and dynamicDestinations"); + + checkArgument(getJsonSchema() == null || getDynamicDestinations() == null, + "Cannot specify both jsonSchema and dynamicDestinations."); + checkArgument(getJsonSchema() == null || getSchemaFromView() == null, + "Cannot specify both jsonSchema and schemaFromView."); + checkArgument(getSchemaFromView() == null || getDynamicDestinations() == null, + "Cannot specify both schemaFromView and dynamicDestinations."); + // The user specified a table. - if (getJsonTableRef() != null && getValidate()) { + if (getJsonTableRef() != null && getJsonTableRef().isAccessible() && getValidate()) { TableReference table = getTableWithDefaultProject(options).get(); - // TODO: This seems wrong - what if the ValueProvider is not accessible? - DatasetService datasetService = getBigQueryServices().getDatasetService(options); // Check for destination table presence and emptiness for early failure notification. // Note that a presence check can fail when the table or dataset is created by an earlier @@ -1026,30 +1121,60 @@ public void validate(PipelineOptions pipelineOptions) { @Override public WriteResult expand(PCollection input) { - validate(input.getPipeline().getOptions()); - PCollection> rowsWithDestination = - input.apply("PrepareWrite", new PrepareWrite( - getTableFunction(), getFormatFunction())) - .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); + DynamicDestinations dynamicDestinations = getDynamicDestinations(); + if (dynamicDestinations == null) { + if (getJsonTableRef() != null) { + dynamicDestinations = ConstantTableDestinations.fromJsonTableRef( + getJsonTableRef(), getTableDescription()); + } else if (getTableFunction() != null) { + dynamicDestinations = new TableFunctionDestinations(getTableFunction()); + } + + // Wrap with a DynamicDestinations class that will provide a schema. There might be no + // schema provided if the create disposition is CREATE_NEVER. + if (getJsonSchema() != null) { + dynamicDestinations = new ConstantSchemaDestinations( + dynamicDestinations, getJsonSchema()); + } else if (getSchemaFromView() != null) { + dynamicDestinations = new SchemaFromViewDestinations( + dynamicDestinations, getSchemaFromView()); + } + } + return expandTyped(input, dynamicDestinations); + } - // The transforms always expect to get a non-null schema function. - SchemaFunction schemaFunction = MoreObjects.firstNonNull( - this.getSchemaFunction(), schemaFunction = ConstantSchemaFunction.fromJsonSchema(null)); + private WriteResult expandTyped( + PCollection input, DynamicDestinations dynamicDestinations) { + Coder destinationCoder = dynamicDestinations.getDestinationCoder(); + if (destinationCoder == null) { + try { + // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. + destinationCoder = input.getPipeline().getCoderRegistry().getDefaultCoder( + new TypeDescriptor() { + }); + } catch (CannotProvideCoderException e) { + throw new RuntimeException(e); + } + } + PCollection> rowsWithDestination = + input.apply("PrepareWrite", new PrepareWrite<>( + dynamicDestinations, getFormatFunction())) + .setCoder(KvCoder.of(destinationCoder, TableRowJsonCoder.of())); - // When writing an Unbounded PCollection, or when a tablespec function is defined, we use - // StreamingInserts and BigQuery's streaming import API. + // When writing an Unbounded PCollection, we use StreamingInserts and BigQuery's streaming + // import API. if (input.isBounded() == IsBounded.UNBOUNDED) { checkArgument( getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + " PCollection."); - return rowsWithDestination.apply(new StreamingInserts( - getCreateDisposition(), schemaFunction).withTestServices(getBigQueryServices())); + return rowsWithDestination.apply(new StreamingInserts( + getCreateDisposition(), dynamicDestinations).withTestServices(getBigQueryServices())); } else { - return rowsWithDestination.apply(new BatchLoads(getWriteDisposition(), - getCreateDisposition(), getJsonTableRef(), getTableDescription(), schemaFunction) + return rowsWithDestination.apply(new BatchLoads(getWriteDisposition(), + getCreateDisposition(), getJsonTableRef(), getTableDescription(), dynamicDestinations) .withTestServices(getBigQueryServices())); } } @@ -1066,10 +1191,8 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull(DisplayData.item("table", getJsonTableRef()) .withLabel("Table Reference")); - if (getSchemaFunction() instanceof ConstantSchemaFunction) { - ValueProvider jsonSchema = ((ConstantSchemaFunction) getSchemaFunction()) - .jsonSchema; - builder.addIfNotNull(DisplayData.item("schema", jsonSchema).withLabel("Table Schema")); + if (getJsonSchema() != null) { + builder.addIfNotNull(DisplayData.item("schema", getJsonSchema()).withLabel("Table Schema")); } else { builder.add(DisplayData.item("schema", "Custom Schema Function").withLabel("Table Schema")); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 83f48da896f1a..5fbcb388c6854 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -22,11 +22,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import java.io.IOException; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -44,12 +44,12 @@ * Creates any tables needed before performing streaming writes to the tables. This is a side-effect * {@link DoFn}, and returns the original collection unchanged. */ -public class CreateTables +public class CreateTables extends PTransform< - PCollection>, PCollection>> { + PCollection>, PCollection>> { private final CreateDisposition createDisposition; private final BigQueryServices bqServices; - private final SchemaFunction schemaFunction; + private final DynamicDestinations dynamicDestinations; /** * The list of tables created so far, so we don't try the creation each time. @@ -59,58 +59,73 @@ public class CreateTables private static Set createdTables = Collections.newSetFromMap(new ConcurrentHashMap()); - public CreateTables(CreateDisposition createDisposition, SchemaFunction schemaFunction) { - this(createDisposition, new BigQueryServicesImpl(), schemaFunction); + public CreateTables( + CreateDisposition createDisposition, + DynamicDestinations dynamicDestinations) { + this(createDisposition, new BigQueryServicesImpl(), dynamicDestinations); } private CreateTables( CreateDisposition createDisposition, BigQueryServices bqServices, - SchemaFunction schemaFunction) { + DynamicDestinations dynamicDestinations) { this.createDisposition = createDisposition; this.bqServices = bqServices; - this.schemaFunction = schemaFunction; + this.dynamicDestinations = dynamicDestinations; } - CreateTables withTestServices(BigQueryServices bqServices) { - return new CreateTables(createDisposition, bqServices, schemaFunction); + CreateTables withTestServices(BigQueryServices bqServices) { + return new CreateTables(createDisposition, bqServices, dynamicDestinations); } @Override public PCollection> expand( - PCollection> input) { - List>> sideInputs = Lists.newArrayList(); - if (schemaFunction.getSideInput() != null) { - sideInputs.add(schemaFunction.getSideInput()); + PCollection> input) { + List> sideInputs = Lists.newArrayList(); + if (dynamicDestinations.getSideInput() != null) { + sideInputs.add(dynamicDestinations.getSideInput()); } return input.apply( ParDo.of( - new DoFn, KV>() { - SchemaFunction schemaFunctionCopy; + new DoFn, KV>() { + DynamicDestinations dynamicDestinationsCopy; - @StartBundle - public void startBundle(Context c) { - this.schemaFunctionCopy = SerializableUtils.clone(schemaFunction); - } + @StartBundle + public void startBundle(Context c) { + this.dynamicDestinationsCopy = SerializableUtils.clone(dynamicDestinations); + } - @ProcessElement - public void processElement(ProcessContext context) - throws InterruptedException, IOException { - if (schemaFunctionCopy.getSideInput() != null) { - schemaFunctionCopy.setSideInputValue( - context.sideInput(schemaFunctionCopy.getSideInput())); - } + @ProcessElement + public void processElement(ProcessContext context) + throws InterruptedException, IOException { + if (dynamicDestinationsCopy.getSideInput() != null) { + dynamicDestinationsCopy.setSideInputValue( + context.sideInput(dynamicDestinationsCopy.getSideInput())); + } - BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - possibleCreateTable(options, context.element().getKey(), schemaFunctionCopy); - context.output(context.element()); - } - }) - .withSideInputs(sideInputs)); + TableDestination tableDestination = + dynamicDestinationsCopy.getTable(context.element().getKey()); + TableReference tableReference = tableDestination.getTableReference(); + if (Strings.isNullOrEmpty(tableReference.getProjectId())) { + tableReference.setProjectId( + context.getPipelineOptions().as(BigQueryOptions.class).getProject()); + tableDestination = + new TableDestination( + tableReference, tableDestination.getTableDescription()); + } + TableSchema tableSchema = + dynamicDestinationsCopy.getSchema(context.element().getKey()); + BigQueryOptions options = + context.getPipelineOptions().as(BigQueryOptions.class); + possibleCreateTable(options, tableDestination, tableSchema); + context.output(KV.of(tableDestination, context.element().getValue())); + } + }) + .withSideInputs(sideInputs)); } - private void possibleCreateTable(BigQueryOptions options, TableDestination tableDestination, - SchemaFunction schemaFunction) + private void possibleCreateTable( + BigQueryOptions options, TableDestination tableDestination, TableSchema tableSchema) throws InterruptedException, IOException { String tableSpec = tableDestination.getTableSpec(); TableReference tableReference = tableDestination.getTableReference(); @@ -122,7 +137,6 @@ private void possibleCreateTable(BigQueryOptions options, TableDestination table // every thread from attempting a create and overwhelming our BigQuery quota. DatasetService datasetService = bqServices.getDatasetService(options); if (!createdTables.contains(tableSpec)) { - TableSchema tableSchema = schemaFunction.apply(tableDestination); if (datasetService.getTable(tableReference) == null) { datasetService.createTable( new Table() diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java new file mode 100644 index 0000000000000..4e7a21d663c04 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java @@ -0,0 +1,105 @@ +/* + * 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.bigquery; + +import com.google.api.services.bigquery.model.TableSchema; +import java.io.Serializable; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.ValueInSingleWindow; + +/** + * This class provides the most general way of specifying dynamic BigQuery table destinations. + * Destinations can be extracted from the input element, and stored as a custom type. Mappings + * are provided to convert the destination into a BigQuery table reference and a BigQuery schema. + * The class can read a side input from another PCollection while performing these mappings. + * + *

For example, consider a PCollection of events, each containing a user-id field. You want to + * write each user's events to a separate table with a separate schema per user. Since the user-id + * field is a string, you will represent the destination as a string. + *

{@code
+ *events.apply(BigQueryIO.write()
+ *  .to(new DynamicDestinations() {
+ *        public String getDestination(ValueInSingleWindow element) {
+ *          return element.getValue().getUserId();
+ *        }
+ *        public TableDestination getTable(String user) {
+ *          return new TableDestination(tableForUser(user), "Table for user " + user);
+ *        }
+ *        public TableSchema getSchema(String user) {
+ *          return tableSchemaForUser(user);
+ *        }
+ *      })
+ *  .withFormatFunction(new SerializableFunction() {
+ *     public TableRow apply(UserEvent event) {
+ *       return convertUserEventToTableRow(event);
+ *     }
+ *   }));
+ *}
+ */ +public abstract class DynamicDestinations implements Serializable { + private PCollectionView sideInput; + private Object materialized; + + public DynamicDestinations withSideInput(PCollectionView> sideInput) { + this.sideInput = sideInput; + return this; + } + + /** + * Returns an object that represents at a high level which table is being written to. + */ + public abstract DestinationT getDestination(ValueInSingleWindow element); + + /** + * Returns the coder for {@link DestinationT}. If this is not overridden, then + * {@link BigQueryIO} will look in the coder registry for a suitable coder. + */ + public @Nullable Coder getDestinationCoder() { + return null; + } + + /** + * Returns a {@link TableDestination} object for the destination. + */ + public abstract TableDestination getTable(DestinationT destination); + + /** + * Returns the table schema for the destination. + */ + public abstract TableSchema getSchema(DestinationT destination); + + public PCollectionView getSideInput() { + return (PCollectionView) sideInput; + } + + /** + * Returns the materialized value of the side input. Can be called by concrete + * {@link DynamicDestinations} instances in {@link #getSchema} or {@link #getTable}. + */ + public SideInputT getSideInputValue() { + return (SideInputT) materialized; + } + + void setSideInputValue(SideInputT value) { + materialized = value; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java index 73d8eb77813bc..be64958fceeaf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java @@ -17,9 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Strings; import java.io.IOException; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -35,29 +33,28 @@ * which tables each element is written to, and format the element into a {@link TableRow} using the * user-supplied format function. */ -public class PrepareWrite - extends PTransform, PCollection>> { - private SerializableFunction, TableDestination> tableFunction; +public class PrepareWrite + extends PTransform, PCollection>> { + private DynamicDestinations dynamicDestinations; private SerializableFunction formatFunction; public PrepareWrite( - SerializableFunction, TableDestination> tableFunction, + DynamicDestinations dynamicDestinations, SerializableFunction formatFunction) { - this.tableFunction = tableFunction; + this.dynamicDestinations = dynamicDestinations; this.formatFunction = formatFunction; } @Override - public PCollection> expand(PCollection input) { + public PCollection> expand(PCollection input) { return input.apply( ParDo.of( - new DoFn>() { + new DoFn>() { @ProcessElement public void processElement(ProcessContext context, BoundedWindow window) throws IOException { - TableDestination tableDestination = - tableSpecFromWindowedValue( - context.getPipelineOptions().as(BigQueryOptions.class), + DestinationT tableDestination = + dynamicDestinations.getDestination( ValueInSingleWindow.of( context.element(), context.timestamp(), window, context.pane())); TableRow tableRow = formatFunction.apply(context.element()); @@ -65,16 +62,4 @@ public void processElement(ProcessContext context, BoundedWindow window) } })); } - - private TableDestination tableSpecFromWindowedValue( - BigQueryOptions options, ValueInSingleWindow value) { - TableDestination tableDestination = tableFunction.apply(value); - TableReference tableReference = tableDestination.getTableReference(); - if (Strings.isNullOrEmpty(tableReference.getProjectId())) { - tableReference.setProjectId(options.getProject()); - tableDestination = - new TableDestination(tableReference, tableDestination.getTableDescription()); - } - return tableDestination; - } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java deleted file mode 100644 index 84dd85574b273..0000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaFunction.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.sdk.io.gcp.bigquery; - -import com.google.api.services.bigquery.model.TableSchema; -import java.util.Map; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.PCollectionView; - -/** - * A user-suppled schema function, mapping a {@link TableDestination} to a {@link TableSchema}. - * - *

A function can declare that it wants access to a map-valued {@link PCollectionView} and use - * that map to assign {@link TableSchema}s to tables. This is used by - * {@link BigQueryIO.Write#withSchemaFromView}. In that case the map is assumed to map string - * tablespecs to json-formatted schemas. - */ -public abstract class SchemaFunction - implements SerializableFunction { - private PCollectionView> sideInput; - private Map materialized; - - public SchemaFunction() { - } - - public SchemaFunction withSideInput(PCollectionView> sideInput) { - this.sideInput = sideInput; - return this; - } - - public PCollectionView> getSideInput() { - return sideInput; - } - - public Map getSideInputValue() { - return materialized; - } - - void setSideInputValue(Map value) { - materialized = value; - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 3ad2d1167d425..343c2116631d9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -30,20 +30,21 @@ * PTransform that performs streaming BigQuery write. To increase consistency, it leverages * BigQuery's best effort de-dup mechanism. */ -public class StreamingInserts - extends PTransform>, WriteResult> { +public class StreamingInserts + extends PTransform>, WriteResult> { private BigQueryServices bigQueryServices; private final CreateDisposition createDisposition; - private final SchemaFunction schemaFunction; + private final DynamicDestinations dynamicDestinations; /** Constructor. */ - StreamingInserts(CreateDisposition createDisposition, SchemaFunction schemaFunction) { + StreamingInserts(CreateDisposition createDisposition, + DynamicDestinations dynamicDestinations) { this.createDisposition = createDisposition; - this.schemaFunction = schemaFunction; + this.dynamicDestinations = dynamicDestinations; this.bigQueryServices = new BigQueryServicesImpl(); } - StreamingInserts withTestServices(BigQueryServices bigQueryServices) { + StreamingInserts withTestServices(BigQueryServices bigQueryServices) { this.bigQueryServices = bigQueryServices; return this; } @@ -54,11 +55,11 @@ protected Coder getDefaultOutputCoder() { } @Override - public WriteResult expand(PCollection> input) { + public WriteResult expand(PCollection> input) { PCollection> writes = input.apply( "CreateTables", - new CreateTables(createDisposition, schemaFunction) + new CreateTables(createDisposition, dynamicDestinations) .withTestServices(bigQueryServices)); return writes.apply(new StreamingWriteTables().withTestServices(bigQueryServices)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index b53d780938090..a1c909f2066bd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -29,6 +29,7 @@ import java.nio.file.Paths; import java.util.Collection; import java.util.List; +import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.slf4j.Logger; @@ -70,39 +70,31 @@ class WriteTables private final boolean singlePartition; private final BigQueryServices bqServices; private final PCollectionView jobIdToken; + private final PCollectionView> schemaMapView; private final String tempFilePrefix; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; - private final SchemaFunction baseSchemaFunction; - private SchemaFunction schemaFunctionCopy; public WriteTables( boolean singlePartition, BigQueryServices bqServices, PCollectionView jobIdToken, + PCollectionView> schemaMapView, String tempFilePrefix, WriteDisposition writeDisposition, - CreateDisposition createDisposition, - SchemaFunction schemaFunction) { + CreateDisposition createDisposition) { this.singlePartition = singlePartition; this.bqServices = bqServices; this.jobIdToken = jobIdToken; + this.schemaMapView = schemaMapView; this.tempFilePrefix = tempFilePrefix; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; - this.baseSchemaFunction = schemaFunction; - } - - @StartBundle - public void startBundle(Context c) { - schemaFunctionCopy = SerializableUtils.clone(baseSchemaFunction); } @ProcessElement public void processElement(ProcessContext c) throws Exception { - if (schemaFunctionCopy.getSideInput() != null) { - schemaFunctionCopy.setSideInputValue(c.sideInput(schemaFunctionCopy.getSideInput())); - } + Map schemaMap = c.sideInput(schemaMapView); TableDestination tableDestination = c.element().getKey().getKey(); Integer partition = c.element().getKey().getShardNumber(); @@ -110,12 +102,15 @@ public void processElement(ProcessContext c) throws Exception { String jobIdPrefix = BigQueryHelpers.createJobId(c.sideInput(jobIdToken), tableDestination, partition); + TableSchema schema = + BigQueryHelpers.fromJsonString( + schemaMap.get(tableDestination.getTableSpec()), TableSchema.class); + TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { ref.setTableId(jobIdPrefix); } - TableSchema schema = schemaFunctionCopy.apply(tableDestination); load( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index d5bd66f0e91c9..e7b5244b088b2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -43,6 +43,7 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -78,7 +79,6 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.ConstantSchemaFunction; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; @@ -207,9 +207,11 @@ private void checkWriteObjectWithValidate( assertEquals(dataset, write.getTable().get().getDatasetId()); assertEquals(table, write.getTable().get().getTableId()); if (schema == null) { - assertNull(write.getSchemaFunction()); + assertNull(write.getJsonSchema()); + assertNull(write.getSchemaFromView()); } else { - assertEquals(schema, write.getSchemaFunction().apply(null)); + assertEquals(schema, BigQueryHelpers.fromJsonString( + write.getJsonSchema().get(), TableSchema.class)); } assertEquals(createDisposition, write.getCreateDisposition()); assertEquals(writeDisposition, write.getWriteDisposition()); @@ -1746,22 +1748,26 @@ public void testWriteTables() throws Exception { } } - PCollection jobIdTokenCollection = p.apply("CreateJobId", Create.of("jobId")); - PCollectionView jobIdTokenView = - jobIdTokenCollection.apply(View.asSingleton()); + PCollectionView jobIdTokenView = p + .apply("CreateJobId", Create.of("jobId")) + .apply(View.asSingleton()); + PCollectionView> schemaMapView = p.apply("CreateEmptySchema", Create.empty( + new TypeDescriptor>() {})) + .apply(View.asMap()); WriteTables writeTables = new WriteTables( false, fakeBqServices, jobIdTokenView, + schemaMapView, tempFilePrefix, WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - ConstantSchemaFunction.fromJsonSchema(null)); + CreateDisposition.CREATE_IF_NEEDED); DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); + tester.setSideInput(schemaMapView, GlobalWindow.INSTANCE, ImmutableMap.of()); for (KV, List> partition : partitions) { tester.processElement(partition); } @@ -1876,9 +1882,9 @@ public void processElement(ProcessContext c) { KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of())); - PCollection jobIdTokenCollection = p.apply("CreateJobId", Create.of("jobId")); - PCollectionView jobIdTokenView = - jobIdTokenCollection.apply(View.asSingleton()); + PCollectionView jobIdTokenView = p + .apply("CreateJobId", Create.of("jobId")) + .apply(View.asSingleton()); WriteRename writeRename = new WriteRename( fakeBqServices, From f31511f5a94a17f0f8d02d1907b9eb057a9f1d83 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Sat, 29 Apr 2017 06:54:14 -0700 Subject: [PATCH 3/6] Address code-review comments. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 143 ++-- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 747 ++++++++---------- .../sdk/io/gcp/bigquery/CalculateSchemas.java | 86 ++ .../sdk/io/gcp/bigquery/CreateTables.java | 23 +- .../io/gcp/bigquery/DynamicDestinations.java | 102 ++- .../bigquery/DynamicDestinationsHelpers.java | 190 +++++ .../sdk/io/gcp/bigquery/StreamingInserts.java | 2 +- .../io/gcp/bigquery/StreamingWriteTables.java | 8 + .../gcp/bigquery/TableDestinationCoder.java | 2 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 56 +- .../sdk/io/gcp/bigquery/WritePartition.java | 69 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 56 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 187 +++-- 13 files changed, 977 insertions(+), 694 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 4db14848543f5..a2f31b4b8cdf1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -20,11 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; -import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; import java.util.List; @@ -33,14 +30,12 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Distinct; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -53,7 +48,6 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.gcsfs.GcsPath; -import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -67,20 +61,20 @@ class BatchLoads private BigQueryServices bigQueryServices; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; - private final ValueProvider singletonJsonTableRef; - private final String singletonTableDescription; + private final boolean singletonTable; private final DynamicDestinations dynamicDestinations; + private final Coder destinationCoder; BatchLoads(WriteDisposition writeDisposition, CreateDisposition createDisposition, - ValueProvider singletonJsonTableRef, - String singletonTableDescription, - DynamicDestinations dynamicDestinations) { + boolean singletonTable, + DynamicDestinations dynamicDestinations, + Coder destinationCoder) { bigQueryServices = new BigQueryServicesImpl(); this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; - this.singletonJsonTableRef = singletonJsonTableRef; - this.singletonTableDescription = singletonTableDescription; + this.singletonTable = singletonTable; this.dynamicDestinations = dynamicDestinations; + this.destinationCoder = destinationCoder; } BatchLoads withTestServices(BigQueryServices bigQueryServices) { @@ -95,7 +89,7 @@ public void validate(PipelineOptions options) { checkArgument( !Strings.isNullOrEmpty(tempLocation), "BigQueryIO.Write needs a GCS temp location to store temp files."); - if (write.getBigQueryServices() == null) { + if (bigQueryServices == null) { try { GcsPath.fromUri(tempLocation); } catch (IllegalArgumentException e) { @@ -150,80 +144,33 @@ public String apply(String input) { Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); + PCollectionView> schemasView = + inputInGlobalWindow.apply(new CalculateSchemas<>(dynamicDestinations)); - final TupleTag> elementsOutputTag = new TupleTag<>(); - final TupleTag> tableSchemasTag = new TupleTag<>(); - - List> resolveSideInputs = Lists.newArrayList(); + List> writeBundlesToFilesSideInputs = Lists.newArrayList(); if (dynamicDestinations.getSideInput() != null) { - resolveSideInputs.add(dynamicDestinations.getSideInput()); + writeBundlesToFilesSideInputs.add(dynamicDestinations.getSideInput()); } - PCollectionTuple resolvedInputTuple = inputInGlobalWindow.apply( - "resolveInput", - ParDo.of(new DoFn, KV>() { - DynamicDestinations dynamicDestinationsCopy; - - @StartBundle - public void startBundle(Context c) { - this.dynamicDestinationsCopy = SerializableUtils.clone(dynamicDestinations); - } - - @ProcessElement - public void processElement(ProcessContext context) { - if (dynamicDestinationsCopy.getSideInput() != null) { - dynamicDestinationsCopy.setSideInputValue( - context.sideInput(dynamicDestinationsCopy.getSideInput())); - } - // Convert to a TableDestination and output. - TableDestination tableDestination = - dynamicDestinationsCopy.getTable(context.element().getKey()); - TableReference tableReference = tableDestination.getTableReference(); - if (Strings.isNullOrEmpty(tableReference.getProjectId())) { - tableReference.setProjectId( - context.getPipelineOptions().as(BigQueryOptions.class).getProject()); - tableDestination = - new TableDestination( - tableReference, tableDestination.getTableDescription()); - } - context.output(elementsOutputTag, KV.of(tableDestination, context.element().getValue())); - - // Match the schema, and output the mapping. - TableSchema tableSchema = dynamicDestinationsCopy.getSchema(context.element().getKey()); - if (tableSchema != null) { - context.output(tableSchemasTag, KV.of(tableDestination.getTableSpec(), - BigQueryHelpers.toJsonString(tableSchema))); - } - } - }).withSideInputs(resolveSideInputs) - .withOutputTags(elementsOutputTag, TupleTagList.of(tableSchemasTag))); - - // We now have input in terms of TableDestinations. - PCollection> resolvedInput = - resolvedInputTuple.get(elementsOutputTag) - .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); - - // Produce the schema map to be used as a side input to following stages. - PCollectionView> schemaMapView = - resolvedInputTuple.get(tableSchemasTag) - .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .apply(Distinct.>create()) - .apply(View.asMap()); // PCollection of filename, file byte size, and table destination. - PCollection results = - resolvedInput - .apply("WriteBundlesToFiles", ParDo.of(new WriteBundlesToFiles(tempFilePrefix))) - .setCoder(WriteBundlesToFiles.ResultCoder.of()); - TupleTag, List>> multiPartitionsTag = - new TupleTag, List>>("multiPartitionsTag") {}; - TupleTag, List>> singlePartitionTag = - new TupleTag, List>>("singlePartitionTag") {}; + PCollection> results = + inputInGlobalWindow + .apply("WriteBundlesToFiles", ParDo.of( + new WriteBundlesToFiles(tempFilePrefix)) + .withSideInputs(writeBundlesToFilesSideInputs)) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + + TupleTag, List>> multiPartitionsTag = + new TupleTag, List>>("multiPartitionsTag") {}; + TupleTag, List>> singlePartitionTag = + new TupleTag, List>>("singlePartitionTag") {}; // Turn the list of files and record counts in a PCollectionView that can be used as a // side input. - PCollectionView> resultsView = - results.apply("ResultsView", View.asIterable()); + PCollectionView>> resultsView = + results.apply("ResultsView", + View.>asIterable()); // This transform will look at the set of files written for each table, and if any table has // too many files or bytes, will partition that table's files into multiple partitions for // loading. @@ -231,9 +178,8 @@ public void processElement(ProcessContext context) { singleton.apply( "WritePartition", ParDo.of( - new WritePartition( - singletonJsonTableRef, - singletonTableDescription, + new WritePartition<>( + singletonTable, resultsView, multiPartitionsTag, singlePartitionTag)) @@ -241,11 +187,16 @@ public void processElement(ProcessContext context) { .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); List> writeTablesSideInputs = - ImmutableList.of(jobIdTokenView, schemaMapView); + Lists.newArrayList(jobIdTokenView, schemasView); + if (dynamicDestinations.getSideInput() != null) { + writeTablesSideInputs.add(dynamicDestinations.getSideInput()); + } + + Coder, List>> partitionsCoder = + KvCoder.of( + ShardedKeyCoder.of(NullableCoder.of(destinationCoder)), + ListCoder.of(StringUtf8Coder.of())); - Coder, List>> partitionsCoder = - KvCoder.of( - ShardedKeyCoder.of(TableDestinationCoder.of()), ListCoder.of(StringUtf8Coder.of())); // If WriteBundlesToFiles produced more than MAX_NUM_FILES files or MAX_SIZE_BYTES bytes, then // the import needs to be split into multiple partitions, and those partitions will be // specified in multiPartitionsTag. @@ -257,18 +208,19 @@ public void processElement(ProcessContext context) { // reexecution of the WritePartitions step once WriteTables has begun. .apply( "MultiPartitionsReshuffle", - Reshuffle., List>of()) + Reshuffle., List>of()) .apply( "MultiPartitionsWriteTables", ParDo.of( - new WriteTables( + new WriteTables<>( false, bigQueryServices, jobIdTokenView, - schemaMapView, + schemasView, tempFilePrefix, WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED)) + CreateDisposition.CREATE_IF_NEEDED, + dynamicDestinations)) .withSideInputs(writeTablesSideInputs)); // This view maps each final table destination to the set of temporary partitioned tables @@ -294,18 +246,19 @@ public void processElement(ProcessContext context) { // Reshuffle will distribute this among multiple workers, and also guard against // reexecution of the WritePartitions step once WriteTables has begun. .apply( - "SinglePartitionsReshuffle", Reshuffle., List>of()) + "SinglePartitionsReshuffle", Reshuffle., List>of()) .apply( "SinglePartitionWriteTables", ParDo.of( - new WriteTables( + new WriteTables<>( true, bigQueryServices, jobIdTokenView, - schemaMapView, + schemasView, tempFilePrefix, writeDisposition, - createDisposition)) + createDisposition, + dynamicDestinations)) .withSideInputs(writeTablesSideInputs)); return WriteResult.in(input.getPipeline()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 1c068b914bcbb..ad92bca947796 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -30,9 +30,12 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Predicates; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -50,12 +53,14 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.CreateJsonTableRefFromUuid; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.CreatePerBeamJobUuid; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableRef; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableSpec; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToJson; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantSchemaDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.SchemaFromViewDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.TableFunctionDestinations; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; @@ -73,41 +78,40 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * {@link PTransform}s for reading and writing - * BigQuery tables. + * {@link PTransform}s for reading and writing BigQuery tables. * *

Table References

* *

A fully-qualified BigQuery table name consists of three components: + * *

    - *
  • {@code projectId}: the Cloud project id (defaults to - * {@link GcpOptions#getProject()}). - *
  • {@code datasetId}: the BigQuery dataset id, unique within a project. - *
  • {@code tableId}: a table id, unique within a dataset. + *
  • {@code projectId}: the Cloud project id (defaults to {@link GcpOptions#getProject()}). + *
  • {@code datasetId}: the BigQuery dataset id, unique within a project. + *
  • {@code tableId}: a table id, unique within a dataset. *
* - *

BigQuery table references are stored as a {@link TableReference}, which comes - * from the - * BigQuery Java Client API. - * Tables can be referred to as Strings, with or without the {@code projectId}. - * A helper function is provided ({@link BigQueryHelpers#parseTableSpec(String)}) - * that parses the following string forms into a {@link TableReference}: + *

BigQuery table references are stored as a {@link TableReference}, which comes from the BigQuery Java Client API. Tables + * can be referred to as Strings, with or without the {@code projectId}. A helper function is + * provided ({@link BigQueryHelpers#parseTableSpec(String)}) that parses the following string forms + * into a {@link TableReference}: * *

    - *
  • [{@code project_id}]:[{@code dataset_id}].[{@code table_id}] - *
  • [{@code dataset_id}].[{@code table_id}] + *
  • [{@code project_id}]:[{@code dataset_id}].[{@code table_id}] + *
  • [{@code dataset_id}].[{@code table_id}] *
* *

Reading

* - *

To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. - * This produces a {@link PCollection} of {@link TableRow TableRows} as output: + *

To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. This produces a + * {@link PCollection} of {@link TableRow TableRows} as output: + * *

{@code
  * PCollection weatherData = pipeline.apply(
  *     BigQueryIO.read().from("clouddataflow-readonly:samples.weather_stations"));
@@ -129,12 +133,12 @@
  *
  * 

Writing

* - *

To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. - * This consumes either a {@link PCollection} of {@link TableRow TableRows} as input when using - * {@link BigQueryIO#writeTableRows()} or of a user-defined type when using - * {@link BigQueryIO#write()}. When using a user-defined type, a function must be provided to - * turn this type into a {@link TableRow} using - * {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. + *

To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. This consumes + * either a {@link PCollection} of {@link TableRow TableRows} as input when using {@link + * BigQueryIO#writeTableRows()} or of a user-defined type when using {@link BigQueryIO#write()}. + * When using a user-defined type, a function must be provided to turn this type into a {@link + * TableRow} using {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. + * *

{@code
  * PCollection quotes = ...
  *
@@ -149,19 +153,18 @@
  *     .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
  * }
* - *

See {@link BigQueryIO.Write} for details on how to specify if a write should - * append to an existing table, replace the table, or verify that the table is - * empty. Note that the dataset being written to must already exist. Unbounded PCollections can only - * be written using {@link Write.WriteDisposition#WRITE_EMPTY} or - * {@link Write.WriteDisposition#WRITE_APPEND}. + *

See {@link BigQueryIO.Write} for details on how to specify if a write should append to an + * existing table, replace the table, or verify that the table is empty. Note that the dataset being + * written to must already exist. Unbounded PCollections can only be written using {@link + * Write.WriteDisposition#WRITE_EMPTY} or {@link Write.WriteDisposition#WRITE_APPEND}. * *

Sharding BigQuery output tables

* - *

A common use case is to dynamically generate BigQuery table names based on - * the current window or the current value. To support this, - * {@link BigQueryIO.Write#to(SerializableFunction)} - * accepts a function mapping the current element to a tablespec. For example, - * here's code that outputs daily tables to BigQuery: + *

A common use case is to dynamically generate BigQuery table names based on the current window + * or the current value. To support this, {@link BigQueryIO.Write#to(SerializableFunction)} accepts + * a function mapping the current element to a tablespec. For example, here's code that outputs + * daily tables to BigQuery: + * *

{@code
  * PCollection quotes = ...
  * quotes.apply(Window.into(CalendarWindows.days(1)))
@@ -179,33 +182,31 @@
  * }
* *

Note that this also allows the table to be a function of the element as well as the current - * pane, in the case of triggered windows. In this case it might be convenient to call - * {@link BigQueryIO#write()} directly instead of using the {@link BigQueryIO#writeTableRows()} - * helper. This will allow the mapping function to access the element of the user-defined type. - * In this case, a formatting function must be specified using - * {@link BigQueryIO.Write#withFormatFunction} to convert each element into a {@link TableRow} - * object. + * pane, in the case of triggered windows. In this case it might be convenient to call {@link + * BigQueryIO#write()} directly instead of using the {@link BigQueryIO#writeTableRows()} helper. + * This will allow the mapping function to access the element of the user-defined type. In this + * case, a formatting function must be specified using {@link BigQueryIO.Write#withFormatFunction} + * to convert each element into a {@link TableRow} object. * - *

Per-table schemas can also be provided using {@link BigQueryIO.Write#withSchemaFromView}. - * This allows you the schemas to be calculated based on a previous pipeline stage or statically - * via a {@link org.apache.beam.sdk.transforms.Create} transform. This method expects to receive a - * map-valued {@link PCollectionView}, mapping table specifications (project:dataset.table-id), - * to JSON formatted {@link TableSchema} objects. All destination tables must be present in this - * map, or the pipeline will fail to create tables. Care should be taken if the map value is based - * on a triggered aggregation over and unbounded {@link PCollection}; the side input will contain - * the entire history of all table schemas ever generated, which might blow up memory usage. - * This method can also be useful when writing to a single table, as it allows a previous stage - * to calculate the schema (possibly based on the full collection of records being written to - * BigQuery). + *

Per-table schemas can also be provided using {@link BigQueryIO.Write#withSchemaFromView}. This + * allows you the schemas to be calculated based on a previous pipeline stage or statically via a + * {@link org.apache.beam.sdk.transforms.Create} transform. This method expects to receive a + * map-valued {@link PCollectionView}, mapping table specifications (project:dataset.table-id), to + * JSON formatted {@link TableSchema} objects. All destination tables must be present in this map, + * or the pipeline will fail to create tables. Care should be taken if the map value is based on a + * triggered aggregation over and unbounded {@link PCollection}; the side input will contain the + * entire history of all table schemas ever generated, which might blow up memory usage. This method + * can also be useful when writing to a single table, as it allows a previous stage to calculate the + * schema (possibly based on the full collection of records being written to BigQuery). * - *

For the most general form of dynamic table destinations and schemas, look at - * {@link BigQueryIO.Write#to(DynamicDestinations)}. + *

For the most general form of dynamic table destinations and schemas, look at {@link + * BigQueryIO.Write#to(DynamicDestinations)}. * *

Permissions

* *

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

Please see BigQuery Access Control * for security and permission related information specific to BigQuery. @@ -213,28 +214,20 @@ public class BigQueryIO { private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); - /** - * Singleton instance of the JSON factory used to read and write JSON - * formatted rows. - */ + /** Singleton instance of the JSON factory used to read and write JSON formatted rows. */ static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); /** - * Project IDs must contain 6-63 lowercase letters, digits, or dashes. - * IDs must start with a letter and may not end with a dash. - * This regex isn't exact - this allows for patterns that would be rejected by - * the service, but this is sufficient for basic parsing of table references. + * Project IDs must contain 6-63 lowercase letters, digits, or dashes. IDs must start with a + * letter and may not end with a dash. This regex isn't exact - this allows for patterns that + * would be rejected by the service, but this is sufficient for basic parsing of table references. */ private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]"; - /** - * Regular expression that matches Dataset IDs. - */ + /** Regular expression that matches Dataset IDs. */ private static final String DATASET_REGEXP = "[-\\w.]{1,1024}"; - /** - * Regular expression that matches Table IDs. - */ + /** Regular expression that matches Table IDs. */ private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}"; /** @@ -242,29 +235,31 @@ public class BigQueryIO { * {@code "[dataset_id].[table_id]"}. */ private static final String DATASET_TABLE_REGEXP = - String.format("((?%s):)?(?%s)\\.(?%s)", PROJECT_ID_REGEXP, - DATASET_REGEXP, TABLE_REGEXP); + String.format( + "((?%s):)?(?%s)\\.(?
%s)", + PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP); static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP); /** - * A formatting function that maps a TableRow to itself. This allows sending a - * {@code PCollection} directly to BigQueryIO.Write. + * A formatting function that maps a TableRow to itself. This allows sending a {@code + * PCollection} directly to BigQueryIO.Write. */ - static final SerializableFunction IDENTITY_FORMATTER = + static final SerializableFunction IDENTITY_FORMATTER = new SerializableFunction() { - @Override - public TableRow apply(TableRow input) { - return input; - } - }; + @Override + public TableRow apply(TableRow input) { + return input; + } + }; /** - * A {@link PTransform} that reads from a BigQuery table and returns a - * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table. + * A {@link PTransform} that reads from a BigQuery table and returns a {@link PCollection} of + * {@link TableRow TableRows} containing each of the rows of the table. + * + *

Each {@link TableRow} contains values indexed by column name. Here is a sample processing + * function that processes a "line" column from rows: * - *

Each {@link TableRow} contains values indexed by column name. Here is a - * sample processing function that processes a "line" column from rows: *

{@code
    * static class ExtractWordsFn extends DoFn {
    *   public void processElement(ProcessContext c) {
@@ -277,7 +272,8 @@ public TableRow apply(TableRow input) {
    *       }
    *     }
    *   }
-   * }}
+ * } + * } */ public static Read read() { return new AutoValue_BigQueryIO_Read.Builder() @@ -289,11 +285,20 @@ public static Read read() { /** Implementation of {@link #read}. */ @AutoValue public abstract static class Read extends PTransform> { - @Nullable abstract ValueProvider getJsonTableRef(); - @Nullable abstract ValueProvider getQuery(); + @Nullable + abstract ValueProvider getJsonTableRef(); + + @Nullable + abstract ValueProvider getQuery(); + abstract boolean getValidate(); - @Nullable abstract Boolean getFlattenResults(); - @Nullable abstract Boolean getUseLegacySql(); + + @Nullable + abstract Boolean getFlattenResults(); + + @Nullable + abstract Boolean getUseLegacySql(); + abstract BigQueryServices getBigQueryServices(); abstract Builder toBuilder(); @@ -301,10 +306,15 @@ public abstract static class Read extends PTransform jsonTableRef); + abstract Builder setQuery(ValueProvider query); + abstract Builder setValidate(boolean validate); + abstract Builder setFlattenResults(Boolean flattenResults); + abstract Builder setUseLegacySql(Boolean useLegacySql); + abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); abstract Read build(); @@ -317,71 +327,64 @@ private void ensureFromNotCalledYet() { } /** - * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or - * {@code "[dataset_id].[table_id]"} for tables within the current project. + * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or {@code + * "[dataset_id].[table_id]"} for tables within the current project. */ public Read from(String tableSpec) { return from(StaticValueProvider.of(tableSpec)); } - /** - * Same as {@code from(String)}, but with a {@link ValueProvider}. - */ + /** Same as {@code from(String)}, but with a {@link ValueProvider}. */ public Read from(ValueProvider tableSpec) { ensureFromNotCalledYet(); return toBuilder() .setJsonTableRef( NestedValueProvider.of( NestedValueProvider.of(tableSpec, new TableSpecToTableRef()), - new TableRefToJson())).build(); + new TableRefToJson())) + .build(); } /** * Reads results received after executing the given query. * - *

By default, the query results will be flattened -- see - * "flattenResults" in the - * Jobs documentation for more information. To disable flattening, use - * {@link BigQueryIO.Read#withoutResultFlattening}. + *

By default, the query results will be flattened -- see "flattenResults" in the Jobs documentation for + * more information. To disable flattening, use {@link BigQueryIO.Read#withoutResultFlattening}. * - *

By default, the query will use BigQuery's legacy SQL dialect. To use the BigQuery - * Standard SQL dialect, use {@link BigQueryIO.Read#usingStandardSql}. + *

By default, the query will use BigQuery's legacy SQL dialect. To use the BigQuery Standard + * SQL dialect, use {@link BigQueryIO.Read#usingStandardSql}. */ public Read fromQuery(String query) { return fromQuery(StaticValueProvider.of(query)); } - /** - * Same as {@code fromQuery(String)}, but with a {@link ValueProvider}. - */ + /** Same as {@code fromQuery(String)}, but with a {@link ValueProvider}. */ public Read fromQuery(ValueProvider query) { ensureFromNotCalledYet(); return toBuilder().setQuery(query).setFlattenResults(true).setUseLegacySql(true).build(); } - /** - * Read from table specified by a {@link TableReference}. - */ + /** Read from table specified by a {@link TableReference}. */ public Read from(TableReference table) { return from(StaticValueProvider.of(BigQueryHelpers.toTableSpec(table))); } private static final String QUERY_VALIDATION_FAILURE_ERROR = "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the" - + " pipeline, This validation can be disabled using #withoutValidation."; + + " pipeline, This validation can be disabled using #withoutValidation."; /** - * Disable validation that the table exists or the query succeeds prior to pipeline - * submission. Basic validation (such as ensuring that a query or table is specified) still - * occurs. + * Disable validation that the table exists or the query succeeds prior to pipeline submission. + * Basic validation (such as ensuring that a query or table is specified) still occurs. */ public Read withoutValidation() { return toBuilder().setValidate(false).build(); } /** - * Disable - * flattening of query results. + * Disable flattening of + * query results. * *

Only valid when a query is used ({@link #fromQuery}). Setting this option when reading * from a table will cause an error during validation. @@ -460,8 +463,10 @@ public void validate(PipelineOptions options) { // Note that a table or query check can fail if the table or dataset are created by // earlier stages of the pipeline or if a query depends on earlier stages of a pipeline. // For these cases the withoutValidation method can be used to disable the check. - if (getValidate() && table != null && table.isAccessible() && table.get().getProjectId() - != null) { + if (getValidate() + && table != null + && table.isAccessible() + && table.get().getProjectId() != null) { checkState(table.isAccessible(), "Cannot call validate if table is dynamically set."); // Check for source table presence for early failure notification. DatasetService datasetService = getBigQueryServices().getDatasetService(bqOptions); @@ -488,10 +493,11 @@ public void validate(PipelineOptions options) { public PCollection expand(PBegin input) { String stepUuid = BigQueryHelpers.randomUUIDString(); BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); - ValueProvider jobUuid = NestedValueProvider.of( - StaticValueProvider.of(bqOptions.getJobName()), new CreatePerBeamJobUuid(stepUuid)); - final ValueProvider jobIdToken = NestedValueProvider.of( - jobUuid, new BeamJobUuidToBigQueryJobUuid()); + ValueProvider jobUuid = + NestedValueProvider.of( + StaticValueProvider.of(bqOptions.getJobName()), new CreatePerBeamJobUuid(stepUuid)); + final ValueProvider jobIdToken = + NestedValueProvider.of(jobUuid, new BeamJobUuidToBigQueryJobUuid()); BoundedSource source; @@ -512,8 +518,7 @@ public PCollection expand(PBegin input) { BigQueryQuerySource.create( jobIdToken, getQuery(), - NestedValueProvider.of( - jobUuid, new CreateJsonTableRefFromUuid(executingProject)), + NestedValueProvider.of(jobUuid, new CreateJsonTableRefFromUuid(executingProject)), getFlattenResults(), getUseLegacySql(), extractDestinationDir, @@ -555,7 +560,8 @@ void cleanup(PipelineOptions options) throws Exception { } } }; - return input.getPipeline() + return input + .getPipeline() .apply(org.apache.beam.sdk.io.Read.from(source)) .setCoder(getDefaultOutputCoder()) .apply(new PassThroughThenCleanup(cleanupOperation)); @@ -570,30 +576,28 @@ protected Coder getDefaultOutputCoder() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("table", BigQueryHelpers.displayTable(getTableProvider())) - .withLabel("Table")) - .addIfNotNull(DisplayData.item("query", getQuery()) - .withLabel("Query")) - .addIfNotNull(DisplayData.item("flattenResults", getFlattenResults()) - .withLabel("Flatten Query Results")) - .addIfNotNull(DisplayData.item("useLegacySql", getUseLegacySql()) - .withLabel("Use Legacy SQL Dialect")) - .addIfNotDefault(DisplayData.item("validation", getValidate()) - .withLabel("Validation Enabled"), - true); + .addIfNotNull( + DisplayData.item("table", BigQueryHelpers.displayTable(getTableProvider())) + .withLabel("Table")) + .addIfNotNull(DisplayData.item("query", getQuery()).withLabel("Query")) + .addIfNotNull( + DisplayData.item("flattenResults", getFlattenResults()) + .withLabel("Flatten Query Results")) + .addIfNotNull( + DisplayData.item("useLegacySql", getUseLegacySql()) + .withLabel("Use Legacy SQL Dialect")) + .addIfNotDefault( + DisplayData.item("validation", getValidate()).withLabel("Validation Enabled"), true); } - /** - * Returns the table to read, or {@code null} if reading from a query instead. - */ + /** Returns the table to read, or {@code null} if reading from a query instead. */ @Nullable public ValueProvider getTableProvider() { return getJsonTableRef() == null - ? null : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); + ? null + : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); } - /** - * Returns the table to read, or {@code null} if reading from a query instead. - */ + /** Returns the table to read, or {@code null} if reading from a query instead. */ @Nullable public TableReference getTable() { ValueProvider provider = getTableProvider(); @@ -614,10 +618,12 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac JobStatistics jobStats = extractJob.getStatistics(); List counts = jobStats.getExtract().getDestinationUriFileCounts(); if (counts.size() != 1) { - String errorMessage = (counts.size() == 0 - ? "No destination uri file count received." - : String.format("More than one destination uri file count received. First two are %s, %s", - counts.get(0), counts.get(1))); + String errorMessage = + (counts.size() == 0 + ? "No destination uri file count received." + : String.format( + "More than one destination uri file count received. First two are %s, %s", + counts.get(0), counts.get(1))); throw new RuntimeException(errorMessage); } long filesCount = counts.get(0); @@ -636,23 +642,24 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac /** * A {@link PTransform} that writes a {@link PCollection} to a BigQuery table. A formatting - * function must be provided to convert each input element into a {@link TableRow} using - * {@link Write#withFormatFunction(SerializableFunction)}. + * function must be provided to convert each input element into a {@link TableRow} using {@link + * Write#withFormatFunction(SerializableFunction)}. * *

In BigQuery, each table has an encosing dataset. The dataset being written must already * exist. * - *

By default, tables will be created if they do not exist, which corresponds to a - * {@link Write.CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of - * BigQuery's Jobs API. A schema must be provided (via {@link Write#withSchema(TableSchema)}), - * or else the transform may fail at runtime with an {@link IllegalArgumentException}. + *

By default, tables will be created if they do not exist, which corresponds to a {@link + * Write.CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's + * Jobs API. A schema must be provided (via {@link Write#withSchema(TableSchema)}), or else the + * transform may fail at runtime with an {@link IllegalArgumentException}. * - *

By default, writes require an empty table, which corresponds to - * a {@link Write.WriteDisposition#WRITE_EMPTY} disposition that matches the - * default of BigQuery's Jobs API. + *

By default, writes require an empty table, which corresponds to a {@link + * Write.WriteDisposition#WRITE_EMPTY} disposition that matches the default of BigQuery's Jobs + * API. + * + *

Here is a sample transform that produces TableRow values containing "word" and "count" + * columns: * - *

Here is a sample transform that produces TableRow values containing - * "word" and "count" columns: *

{@code
    * static class FormatCountsFn extends DoFn, TableRow> {
    *   public void processElement(ProcessContext c) {
@@ -661,7 +668,8 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac
    *         .set("count", c.element().getValue().intValue());
    *     c.output(row);
    *   }
-   * }}
+ * } + * } */ public static Write write() { return new AutoValue_BigQueryIO_Write.Builder() @@ -674,8 +682,8 @@ public static Write write() { } /** - * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows} - * to a BigQuery table. + * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows} to + * a BigQuery table. */ public static Write writeTableRows() { return BigQueryIO.write().withFormatFunction(IDENTITY_FORMATTER); @@ -699,24 +707,32 @@ public abstract static class Write extends PTransform, WriteRe // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery job finishes. static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; - @Nullable abstract ValueProvider getJsonTableRef(); - - /** */ - @Nullable abstract SerializableFunction, TableDestination> - getTableFunction(); - @Nullable abstract SerializableFunction getFormatFunction(); - /** */ - @Nullable abstract DynamicDestinations getDynamicDestinations(); - /** */ - @Nullable abstract PCollectionView> getSchemaFromView(); - /** */ - @Nullable abstract ValueProvider getJsonSchema(); + @Nullable + abstract ValueProvider getJsonTableRef(); + + @Nullable + abstract SerializableFunction, TableDestination> getTableFunction(); + + @Nullable + abstract SerializableFunction getFormatFunction(); + + @Nullable + abstract DynamicDestinations getDynamicDestinations(); + + @Nullable + abstract PCollectionView> getSchemaFromView(); + + @Nullable + abstract ValueProvider getJsonSchema(); + abstract CreateDisposition getCreateDisposition(); + abstract WriteDisposition getWriteDisposition(); /** Table description. Default is empty. */ abstract String getTableDescription(); /** An option to indicate if table validation is desired. Default is true. */ abstract boolean getValidate(); + abstract BigQueryServices getBigQueryServices(); abstract Builder toBuilder(); @@ -724,17 +740,26 @@ public abstract static class Write extends PTransform, WriteRe @AutoValue.Builder abstract static class Builder { abstract Builder setJsonTableRef(ValueProvider jsonTableRef); + abstract Builder setTableFunction( SerializableFunction, TableDestination> tableFunction); - abstract Builder setFormatFunction( - SerializableFunction formatFunction); + + abstract Builder setFormatFunction(SerializableFunction formatFunction); + abstract Builder setDynamicDestinations(DynamicDestinations dynamicDestinations); + abstract Builder setSchemaFromView(PCollectionView> view); + abstract Builder setJsonSchema(ValueProvider jsonSchema); + abstract Builder setCreateDisposition(CreateDisposition createDisposition); + abstract Builder setWriteDisposition(WriteDisposition writeDisposition); + abstract Builder setTableDescription(String tableDescription); + abstract Builder setValidate(boolean validate); + abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); abstract Write build(); @@ -743,8 +768,9 @@ abstract Builder setFormatFunction( /** * An enumeration type for the BigQuery create disposition strings. * - * @see - * configuration.query.createDisposition in the BigQuery Jobs API + * @see + * configuration.query.createDisposition in the BigQuery Jobs API */ public enum CreateDisposition { /** @@ -755,17 +781,15 @@ public enum CreateDisposition { CREATE_NEVER, /** - * Specifies that tables should be created if needed. This is the default - * behavior. + * Specifies that tables should be created if needed. This is the default behavior. * - *

Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}. - * This precondition is checked before starting a job. The schema is - * not required to match an existing table's schema. + *

Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}. This + * precondition is checked before starting a job. The schema is not required to match an + * existing table's schema. * - *

When this transformation is executed, if the output table does not - * exist, the table is created from the provided schema. Note that even if - * the table exists, it may be recreated if necessary when paired with a - * {@link WriteDisposition#WRITE_TRUNCATE}. + *

When this transformation is executed, if the output table does not exist, the table is + * created from the provided schema. Note that even if the table exists, it may be recreated + * if necessary when paired with a {@link WriteDisposition#WRITE_TRUNCATE}. */ CREATE_IF_NEEDED } @@ -773,43 +797,39 @@ public enum CreateDisposition { /** * An enumeration type for the BigQuery write disposition strings. * - * @see - * configuration.query.writeDisposition in the BigQuery Jobs API + * @see + * configuration.query.writeDisposition in the BigQuery Jobs API */ public enum WriteDisposition { /** * Specifies that write should replace a table. * - *

The replacement may occur in multiple steps - for instance by first - * removing the existing table, then creating a replacement, then filling - * it in. This is not an atomic operation, and external programs may - * see the table in any of these intermediate steps. + *

The replacement may occur in multiple steps - for instance by first removing the + * existing table, then creating a replacement, then filling it in. This is not an atomic + * operation, and external programs may see the table in any of these intermediate steps. */ WRITE_TRUNCATE, - /** - * Specifies that rows may be appended to an existing table. - */ + /** Specifies that rows may be appended to an existing table. */ WRITE_APPEND, /** - * Specifies that the output table must be empty. This is the default - * behavior. + * Specifies that the output table must be empty. This is the default behavior. * *

If the output table is not empty, the write fails at runtime. * - *

This check may occur long before data is written, and does not - * guarantee exclusive access to the table. If two programs are run - * concurrently, each specifying the same output table and - * a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is possible - * for both to succeed. + *

This check may occur long before data is written, and does not guarantee exclusive + * access to the table. If two programs are run concurrently, each specifying the same output + * table and a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is + * possible for both to succeed. */ WRITE_EMPTY } /** - * Writes to the given table, specified in the format described in - * {@link BigQueryHelpers#parseTableSpec}. + * Writes to the given table, specified in the format described in {@link + * BigQueryHelpers#parseTableSpec}. */ public Write to(String tableSpec) { return to(StaticValueProvider.of(tableSpec)); @@ -831,203 +851,48 @@ public Write to(ValueProvider tableSpec) { } /** - * Writes to table specified by the specified table function. The table is a function of - * {@link ValueInSingleWindow}, so can be determined by the value or by the window. + * Writes to table specified by the specified table function. The table is a function of {@link + * ValueInSingleWindow}, so can be determined by the value or by the window. */ public Write to( SerializableFunction, TableDestination> tableFunction) { return toBuilder().setTableFunction(tableFunction).build(); } - /** - * Writes to the table and schema specified by the {@link DynamicDestinations} object. - */ + /** Writes to the table and schema specified by the {@link DynamicDestinations} object. */ public Write to(DynamicDestinations dynamicDestinations) { - return toBuilder().setDynamicDestinations(dynamicDestinations).build(); + return toBuilder().setDynamicDestinations(dynamicDestinations).build(); } - /** - * Formats the user's type into a {@link TableRow} to be written to BigQuery. - */ + /** Formats the user's type into a {@link TableRow} to be written to BigQuery. */ public Write withFormatFunction(SerializableFunction formatFunction) { return toBuilder().setFormatFunction(formatFunction).build(); } - /** - * Always returns a constant table destination. - */ - static class ConstantTableDestinations extends DynamicDestinations { - private final ValueProvider tableSpec; - private final String tableDescription; - - private ConstantTableDestinations(ValueProvider tableSpec, String tableDescription) { - this.tableSpec = tableSpec; - this.tableDescription = tableDescription; - } - - static ConstantTableDestinations fromTableSpec( - ValueProvider tableSpec, String tableDescription) { - return new ConstantTableDestinations(tableSpec, tableDescription); - } - - static ConstantTableDestinations fromJsonTableRef( - ValueProvider jsonTableRef, String tableDescription) { - return new ConstantTableDestinations( - NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableSpec()), tableDescription); - } - - @Override - public TableDestination getDestination(ValueInSingleWindow element) { - return new TableDestination(tableSpec.get(), tableDescription); - } - - @Override - public TableSchema getSchema(TableDestination destination) { - return null; - } - - @Override - public TableDestination getTable(TableDestination destination) { - return destination; - } - - @Override - public Coder getDestinationCoder() { - return TableDestinationCoder.of(); - } - } - - /** - * Returns a tables based on a user-supplied function. - */ - static class TableFunctionDestinations extends DynamicDestinations { - private final SerializableFunction, TableDestination> tableFunction; - - private TableFunctionDestinations( - SerializableFunction, TableDestination> tableFunction) { - this.tableFunction = tableFunction; - } - - @Override - public TableDestination getDestination(ValueInSingleWindow element) { - return tableFunction.apply(element); - } - - @Override - public TableSchema getSchema(TableDestination destination) { - return null; - } - - @Override - public TableDestination getTable(TableDestination destination) { - return destination; - } - - @Override - public Coder getDestinationCoder() { - return TableDestinationCoder.of(); - } - } - - /** - * Delegates all calls to an inner instance of {@link DynamicDestinations}. This allows - * subclasses to modify another instance of {@link DynamicDestinations} by subclassing and - * overriding just the methods they want to alter. - */ - static class DelegatingDynamicDestinations - extends DynamicDestinations { - private final DynamicDestinations inner; - DelegatingDynamicDestinations(DynamicDestinations inner) { - this.inner = inner; - } - @Override - public DestinationT getDestination(ValueInSingleWindow element) { - return inner.getDestination(element); - } - - @Override - public TableSchema getSchema(DestinationT destination) { - return inner.getSchema(destination); - } - - @Override - public TableDestination getTable(DestinationT destination) { - return inner.getTable(destination); - } - - @Override - public Coder getDestinationCoder() { - return inner.getDestinationCoder(); - } - } - /** - * Returns the same schema for every table. - */ - static class ConstantSchemaDestinations - extends DelegatingDynamicDestinations { - private final @Nullable ValueProvider jsonSchema; - - ConstantSchemaDestinations(DynamicDestinations inner, - ValueProvider jsonSchema) { - super(inner); - this.jsonSchema = jsonSchema; - } - - @Override - public TableSchema getSchema(TableDestination destination) { - return BigQueryHelpers.fromJsonString(jsonSchema.get(), TableSchema.class); - } - } - - /** - * Takes in a side input mapping tablespec to json table schema, and always returns the - * matching schema from the side input. - */ - static class SchemaFromViewDestinations - extends DelegatingDynamicDestinations { - SchemaFromViewDestinations(DynamicDestinations inner, - PCollectionView> schemaView) { - super(inner); - withSideInput(schemaView); - } - - @Override - public TableSchema getSchema(TableDestination destination) { - Map mapValue = getSideInputValue(); - return BigQueryHelpers.fromJsonString(mapValue.get(destination.getTableSpec()), - TableSchema.class); - } - } - /** * Uses the specified schema for rows to be written. * - *

The schema is required only if writing to a table that does not already - * exist, and {@link CreateDisposition} is set to - * {@link CreateDisposition#CREATE_IF_NEEDED}. + *

The schema is required only if writing to a table that does not already exist, and + * {@link CreateDisposition} is set to {@link CreateDisposition#CREATE_IF_NEEDED}. */ public Write withSchema(TableSchema schema) { return withJsonSchema(StaticValueProvider.of(BigQueryHelpers.toJsonString(schema))); } - /** - * Same as {@link #withSchema(TableSchema)} but using a deferred {@link ValueProvider}. - */ + /** Same as {@link #withSchema(TableSchema)} but using a deferred {@link ValueProvider}. */ public Write withSchema(ValueProvider schema) { return withJsonSchema(NestedValueProvider.of(schema, new TableSchemaToJsonSchema())); } /** - * Similar to {@link #withSchema(TableSchema)} but takes in a JSON-serialized - * {@link TableSchema}. + * Similar to {@link #withSchema(TableSchema)} but takes in a JSON-serialized {@link + * TableSchema}. */ public Write withJsonSchema(String jsonSchema) { return withJsonSchema(StaticValueProvider.of(jsonSchema)); } - /** - * Same as {@link #withJsonSchema(String)} but using a deferred {@link ValueProvider}. - */ + /** Same as {@link #withJsonSchema(String)} but using a deferred {@link ValueProvider}. */ public Write withJsonSchema(ValueProvider jsonSchema) { return toBuilder().setJsonSchema(jsonSchema).build(); } @@ -1036,7 +901,8 @@ public Write withJsonSchema(ValueProvider jsonSchema) { * Allows the schemas for each table to be computed within the pipeline itself. * *

The input is a map-valued {@link PCollectionView} mapping string tablespecs to - * JSON-formatted {@link TableSchema}s. + * JSON-formatted {@link TableSchema}s. Tablespecs must be in the same format as taken by + * {@link #to(String)}. */ public Write withSchemaFromView(PCollectionView> view) { return toBuilder().setSchemaFromView(view).build(); @@ -1072,34 +938,41 @@ public void validate(PipelineOptions pipelineOptions) { BigQueryOptions options = pipelineOptions.as(BigQueryOptions.class); // We must have a destination to write to! - checkState(getTableFunction() != null || getJsonTableRef() != null + checkState( + getTableFunction() != null + || getJsonTableRef() != null || getDynamicDestinations() != null, "must set the table reference of a BigQueryIO.Write transform"); - checkArgument(getFormatFunction() != null, - "A function must be provided to convert type into a TableRow. " - + "use BigQueryIO.Write.withFormatFunction to provide a formatting function."); + checkArgument( + getFormatFunction() != null, + "A function must be provided to convert type into a TableRow. " + + "use BigQueryIO.Write.withFormatFunction to provide a formatting function."); // Require a schema if creating one or more tables. checkArgument( getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED - || getJsonSchema() != null || getDynamicDestinations() != null - || getSchemaFromView() != null, + || getJsonSchema() != null + || getDynamicDestinations() != null + || getSchemaFromView() != null, "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided."); - checkArgument(getJsonTableRef() == null || getTableFunction() == null, - "Cannot specify both jsonTableRef and tableFunction"); - checkArgument(getJsonTableRef() == null || getDynamicDestinations() == null, - "Cannot specify both jsonTableRef and dynamicDestinations"); - checkArgument(getTableFunction() == null || getDynamicDestinations() == null, - "Cannot specify both tableFunction and dynamicDestinations"); + List allToArgs = Lists.newArrayList(getJsonTableRef(), getTableFunction(), + getDynamicDestinations()); + checkArgument( + 1 + == Iterables.size( + Iterables.filter(allToArgs, Predicates.notNull())), + "Exactly one of jsonTableRef, tableFunction, or " + "dynamicDestinations must be set"); - checkArgument(getJsonSchema() == null || getDynamicDestinations() == null, - "Cannot specify both jsonSchema and dynamicDestinations."); - checkArgument(getJsonSchema() == null || getSchemaFromView() == null, - "Cannot specify both jsonSchema and schemaFromView."); - checkArgument(getSchemaFromView() == null || getDynamicDestinations() == null, - "Cannot specify both schemaFromView and dynamicDestinations."); + List allSchemaArgs = Lists.newArrayList(getJsonSchema(), getSchemaFromView(), + getDynamicDestinations()); + checkArgument( + 2 + > Iterables.size( + Iterables.filter(allSchemaArgs, Predicates.notNull())), + "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may " + + "be set"); // The user specified a table. if (getJsonTableRef() != null && getJsonTableRef().isAccessible() && getValidate()) { @@ -1126,8 +999,9 @@ public WriteResult expand(PCollection input) { DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { if (getJsonTableRef() != null) { - dynamicDestinations = ConstantTableDestinations.fromJsonTableRef( - getJsonTableRef(), getTableDescription()); + dynamicDestinations = + DynamicDestinationsHelpers.ConstantTableDestinations.fromJsonTableRef( + getJsonTableRef(), getTableDescription()); } else if (getTableFunction() != null) { dynamicDestinations = new TableFunctionDestinations(getTableFunction()); } @@ -1135,11 +1009,11 @@ public WriteResult expand(PCollection input) { // Wrap with a DynamicDestinations class that will provide a schema. There might be no // schema provided if the create disposition is CREATE_NEVER. if (getJsonSchema() != null) { - dynamicDestinations = new ConstantSchemaDestinations( - dynamicDestinations, getJsonSchema()); + dynamicDestinations = + new ConstantSchemaDestinations(dynamicDestinations, getJsonSchema()); } else if (getSchemaFromView() != null) { - dynamicDestinations = new SchemaFromViewDestinations( - dynamicDestinations, getSchemaFromView()); + dynamicDestinations = + new SchemaFromViewDestinations(dynamicDestinations, getSchemaFromView()); } } return expandTyped(input, dynamicDestinations); @@ -1147,20 +1021,17 @@ public WriteResult expand(PCollection input) { private WriteResult expandTyped( PCollection input, DynamicDestinations dynamicDestinations) { - Coder destinationCoder = dynamicDestinations.getDestinationCoder(); - if (destinationCoder == null) { - try { - // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. - destinationCoder = input.getPipeline().getCoderRegistry().getDefaultCoder( - new TypeDescriptor() { - }); - } catch (CannotProvideCoderException e) { + Coder destinationCoder = null; + try { + destinationCoder = dynamicDestinations.getDestinationCoderWithDefault( + input.getPipeline().getCoderRegistry()); + } catch (CannotProvideCoderException e) { throw new RuntimeException(e); - } } + PCollection> rowsWithDestination = - input.apply("PrepareWrite", new PrepareWrite<>( - dynamicDestinations, getFormatFunction())) + input + .apply("PrepareWrite", new PrepareWrite<>(dynamicDestinations, getFormatFunction())) .setCoder(KvCoder.of(destinationCoder, TableRowJsonCoder.of())); // When writing an Unbounded PCollection, we use StreamingInserts and BigQuery's streaming @@ -1170,12 +1041,18 @@ dynamicDestinations, getFormatFunction())) getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + " PCollection."); - return rowsWithDestination.apply(new StreamingInserts( - getCreateDisposition(), dynamicDestinations).withTestServices(getBigQueryServices())); + return rowsWithDestination.apply( + new StreamingInserts<>(getCreateDisposition(), dynamicDestinations) + .setTestServices(getBigQueryServices())); } else { - return rowsWithDestination.apply(new BatchLoads(getWriteDisposition(), - getCreateDisposition(), getJsonTableRef(), getTableDescription(), dynamicDestinations) - .withTestServices(getBigQueryServices())); + return rowsWithDestination.apply( + new BatchLoads<>( + getWriteDisposition(), + getCreateDisposition(), + getJsonTableRef() != null, + dynamicDestinations, + destinationCoder) + .withTestServices(getBigQueryServices())); } } @@ -1188,9 +1065,8 @@ protected Coder getDefaultOutputCoder() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("table", getJsonTableRef()) - .withLabel("Table Reference")); + builder.addIfNotNull( + DisplayData.item("table", getJsonTableRef()).withLabel("Table Reference")); if (getJsonSchema() != null) { builder.addIfNotNull(DisplayData.item("schema", getJsonSchema()).withLabel("Table Schema")); } else { @@ -1198,19 +1074,24 @@ public void populateDisplayData(DisplayData.Builder builder) { } if (getTableFunction() != null) { - builder.add(DisplayData.item("tableFn", getTableFunction().getClass()) - .withLabel("Table Reference Function")); + builder.add( + DisplayData.item("tableFn", getTableFunction().getClass()) + .withLabel("Table Reference Function")); } builder - .add(DisplayData.item("createDisposition", getCreateDisposition().toString()) - .withLabel("Table CreateDisposition")) - .add(DisplayData.item("writeDisposition", getWriteDisposition().toString()) - .withLabel("Table WriteDisposition")) - .addIfNotDefault(DisplayData.item("validation", getValidate()) - .withLabel("Validation Enabled"), true) - .addIfNotDefault(DisplayData.item("tableDescription", getTableDescription()) - .withLabel("Table Description"), ""); + .add( + DisplayData.item("createDisposition", getCreateDisposition().toString()) + .withLabel("Table CreateDisposition")) + .add( + DisplayData.item("writeDisposition", getWriteDisposition().toString()) + .withLabel("Table WriteDisposition")) + .addIfNotDefault( + DisplayData.item("validation", getValidate()).withLabel("Validation Enabled"), true) + .addIfNotDefault( + DisplayData.item("tableDescription", getTableDescription()) + .withLabel("Table Description"), + ""); } /** @@ -1218,15 +1099,18 @@ public void populateDisplayData(DisplayData.Builder builder) { * *

If the table's project is not specified, use the executing project. */ - @Nullable ValueProvider getTableWithDefaultProject(BigQueryOptions bqOptions) { + @Nullable + ValueProvider getTableWithDefaultProject(BigQueryOptions bqOptions) { ValueProvider table = getTable(); if (table == null) { return table; } if (!table.isAccessible()) { - LOG.info("Using a dynamic value for table input. This must contain a project" - + " in the table reference: {}", table); + LOG.info( + "Using a dynamic value for table input. This must contain a project" + + " in the table reference: {}", + table); return table; } if (Strings.isNullOrEmpty(table.get().getProjectId())) { @@ -1234,8 +1118,9 @@ public void populateDisplayData(DisplayData.Builder builder) { // the default project. TableReference tableRef = table.get(); tableRef.setProjectId(bqOptions.getProject()); - return NestedValueProvider.of(StaticValueProvider.of( - BigQueryHelpers.toJsonString(tableRef)), new JsonTableRefToTableRef()); + return NestedValueProvider.of( + StaticValueProvider.of(BigQueryHelpers.toJsonString(tableRef)), + new JsonTableRefToTableRef()); } return table; } @@ -1243,16 +1128,13 @@ public void populateDisplayData(DisplayData.Builder builder) { /** Returns the table reference, or {@code null}. */ @Nullable public ValueProvider getTable() { - return getJsonTableRef() == null ? null : - NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); + return getJsonTableRef() == null + ? null + : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); } - - } - /** - * Clear the cached map of created tables. Used for testing. - */ + /** Clear the cached map of created tables. Used for testing. */ @VisibleForTesting static void clearCreatedTables() { CreateTables.clearCreatedTables(); @@ -1262,5 +1144,4 @@ static void clearCreatedTables() { /** Disallow construction of utility class. */ private BigQueryIO() {} - } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java new file mode 100644 index 0000000000000..071dbf342072e --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java @@ -0,0 +1,86 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +/** Created by relax on 4/30/17. */ +/* + * 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. + */ + +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.transforms.Distinct; +import org.apache.beam.sdk.transforms.DoFn; + +import org.apache.beam.sdk.transforms.Keys; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Compute the mapping of destinations to json-formatted schema objects. */ +class CalculateSchemas + extends PTransform< + PCollection>, PCollectionView>> { + private static final Logger LOG = LoggerFactory.getLogger(CalculateSchemas.class); + + private final DynamicDestinations dynamicDestinations; + + public CalculateSchemas(DynamicDestinations dynamicDestinations) { + this.dynamicDestinations = dynamicDestinations; + } + + @Override + public PCollectionView> expand( + PCollection> input) { + List> sideInputs = Lists.newArrayList(); + if (dynamicDestinations.getSideInput() != null) { + sideInputs.add(dynamicDestinations.getSideInput()); + } + return input + .apply("Keys", Keys.create()) + .apply("Distinct Keys", Distinct.create()) + .apply( + "GetSchemas", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + // If the DynamicDestinations class wants to read a side input, give it the + // value. + DynamicDestinations.SideInputAccessor sideInputAccessor = + new DynamicDestinations.SideInputAccessor( + c, dynamicDestinations.getSideInput()); + TableSchema tableSchema = dynamicDestinations.getSchema( + c.element(), sideInputAccessor); + if (tableSchema != null) { + // If the createDisposition is CREATE_NEVER, then there's no need for a + // schema, and getSchema might return null. In this case, we simply + // leave it out of the map. + c.output(KV.of(c.element(), BigQueryHelpers.toJsonString(tableSchema))); + } + } + }) + .withSideInputs(sideInputs)) + .apply("asMap", View.asMap()); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 5fbcb388c6854..77cb9d9f24cb5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -35,7 +35,6 @@ 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.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -75,7 +74,7 @@ private CreateTables( } CreateTables withTestServices(BigQueryServices bqServices) { - return new CreateTables(createDisposition, bqServices, dynamicDestinations); + return new CreateTables(createDisposition, bqServices, dynamicDestinations); } @Override @@ -88,23 +87,16 @@ public PCollection> expand( return input.apply( ParDo.of( new DoFn, KV>() { - DynamicDestinations dynamicDestinationsCopy; - - @StartBundle - public void startBundle(Context c) { - this.dynamicDestinationsCopy = SerializableUtils.clone(dynamicDestinations); - } - @ProcessElement public void processElement(ProcessContext context) throws InterruptedException, IOException { - if (dynamicDestinationsCopy.getSideInput() != null) { - dynamicDestinationsCopy.setSideInputValue( - context.sideInput(dynamicDestinationsCopy.getSideInput())); - } + // If a side input is needed to produce the table name, set it. + DynamicDestinations.SideInputAccessor sideInputAccessor = + new DynamicDestinations.SideInputAccessor( + context, dynamicDestinations.getSideInput()); TableDestination tableDestination = - dynamicDestinationsCopy.getTable(context.element().getKey()); + dynamicDestinations.getTable(context.element().getKey(), sideInputAccessor); TableReference tableReference = tableDestination.getTableReference(); if (Strings.isNullOrEmpty(tableReference.getProjectId())) { tableReference.setProjectId( @@ -114,7 +106,8 @@ public void processElement(ProcessContext context) tableReference, tableDestination.getTableDescription()); } TableSchema tableSchema = - dynamicDestinationsCopy.getSchema(context.element().getKey()); + dynamicDestinations.getSchema(context.element().getKey(), + sideInputAccessor); BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); possibleCreateTable(options, tableDestination, tableSchema); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java index 4e7a21d663c04..c2a570405ec9f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java @@ -20,10 +20,15 @@ import com.google.api.services.bigquery.model.TableSchema; import java.io.Serializable; -import java.util.Map; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; /** @@ -35,8 +40,8 @@ *

For example, consider a PCollection of events, each containing a user-id field. You want to * write each user's events to a separate table with a separate schema per user. Since the user-id * field is a string, you will represent the destination as a string. - *

{@code
- *events.apply(BigQueryIO.write()
+ * 
{@code
+ * events.apply(BigQueryIO.write()
  *  .to(new DynamicDestinations() {
  *        public String getDestination(ValueInSingleWindow element) {
  *          return element.getValue().getUserId();
@@ -53,13 +58,47 @@
  *       return convertUserEventToTableRow(event);
  *     }
  *   }));
- *}
+ * }
+ * + *

An instance of {@link DynamicDestinations} can also request a side input value that can be + * examined from inside {@link #getTable} and {@link #getSchema}. The side input is requested by + * calling {@link #setSideInputRequired} on the base class. The value can be examined via the + * {@link DynamicDestinations.SideInputAccessor} class. + * + *

{@code DestinationT} is expected to provide proper hash and equality members. Ideally it will + * be a compact type with an efficient coder, as these objects may be used as a key in a + * {@link org.apache.beam.sdk.transforms.GroupByKey}. */ public abstract class DynamicDestinations implements Serializable { private PCollectionView sideInput; - private Object materialized; - public DynamicDestinations withSideInput(PCollectionView> sideInput) { + /** + * Returns the materialized value of the side input. Can be used by concrete + * {@link DynamicDestinations} instances in {@link #getSchema} or {@link #getTable}. + */ + public static class SideInputAccessor { + private ProcessContext processContext; + private PCollectionView sideInput; + SideInputAccessor(ProcessContext processContext, PCollectionView sideInput) { + this.processContext = processContext; + this.sideInput = sideInput; + } + + public SideInputT getSideInputValue() throws IllegalStateException { + if (sideInput == null) { + return null; + } + @SuppressWarnings("unchecked") + SideInputT materialized = (SideInputT) processContext.sideInput(sideInput); + return materialized; + } + } + + /** + * Specifies that this object needs access to a side input. This side input must be globally + * windowed, as it will be accessed from the global window. + */ + public DynamicDestinations setSideInputRequired(PCollectionView sideInput) { this.sideInput = sideInput; return this; } @@ -71,35 +110,58 @@ public DynamicDestinations withSideInput(PCollectionView> si /** * Returns the coder for {@link DestinationT}. If this is not overridden, then - * {@link BigQueryIO} will look in the coder registry for a suitable coder. + * {@link BigQueryIO} will look in the coder registry for a suitable coder. This must be a + * deterministic coder, as {@link DestinationT} will be used as a key type in a + * {@link org.apache.beam.sdk.transforms.GroupByKey}. */ - public @Nullable Coder getDestinationCoder() { + @Nullable + public Coder getDestinationCoder() { return null; } /** * Returns a {@link TableDestination} object for the destination. */ - public abstract TableDestination getTable(DestinationT destination); + public abstract TableDestination getTable(DestinationT destination, + SideInputAccessor sideInputAccessor); /** * Returns the table schema for the destination. */ - public abstract TableSchema getSchema(DestinationT destination); - - public PCollectionView getSideInput() { - return (PCollectionView) sideInput; - } + public abstract TableSchema getSchema(DestinationT destination, + SideInputAccessor sideInputAccessor); /** - * Returns the materialized value of the side input. Can be called by concrete - * {@link DynamicDestinations} instances in {@link #getSchema} or {@link #getTable}. + * This returns the unmaterialized side input used by this transform. */ - public SideInputT getSideInputValue() { - return (SideInputT) materialized; + PCollectionView getSideInput() { + @SuppressWarnings("unchecked") + PCollectionView sideInputTyped = (PCollectionView) sideInput; + return sideInputTyped; } - void setSideInputValue(SideInputT value) { - materialized = value; + // Gets the destination coder. If the user does not provide one, try to find one in the coder + // registry. If no coder can be found, throws CannotProvideCoderException. + Coder getDestinationCoderWithDefault(CoderRegistry registry) + throws CannotProvideCoderException { + Coder destinationCoder = getDestinationCoder(); + // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. + // We must first use reflection to figure out what the type parameter is. + Type superclass = getClass().getGenericSuperclass(); + while (destinationCoder == null) { + if (superclass instanceof ParameterizedType) { + ParameterizedType parameterized = (ParameterizedType) superclass; + if (parameterized.getRawType() == DynamicDestinations.class) { + // DestinationT is the second parameter. + Type parameter = parameterized.getActualTypeArguments()[1]; + @SuppressWarnings("unchecked") + Class parameterClass = (Class) parameter; + destinationCoder = registry.getDefaultCoder(parameterClass); + break; + } + } + superclass = ((Class) superclass).getGenericSuperclass(); + } + return destinationCoder; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java new file mode 100644 index 0000000000000..1ca729653213c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -0,0 +1,190 @@ +/* + * 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.bigquery; + +import com.google.api.services.bigquery.model.TableSchema; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableSpec; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.ValueInSingleWindow; + +/** + * Contains some useful helper instances of {@link DynamicDestinations}. + */ +class DynamicDestinationsHelpers { + /** + * Always returns a constant table destination. + */ + static class ConstantTableDestinations extends DynamicDestinations { + private final ValueProvider tableSpec; + private final String tableDescription; + + ConstantTableDestinations(ValueProvider tableSpec, String tableDescription) { + this.tableSpec = tableSpec; + this.tableDescription = tableDescription; + } + + static ConstantTableDestinations fromTableSpec( + ValueProvider tableSpec, String tableDescription) { + return new ConstantTableDestinations(tableSpec, tableDescription); + } + + static ConstantTableDestinations fromJsonTableRef( + ValueProvider jsonTableRef, String tableDescription) { + return new ConstantTableDestinations( + NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableSpec()), tableDescription); + } + + @Override + public TableDestination getDestination(ValueInSingleWindow element) { + return new TableDestination(tableSpec.get(), tableDescription); + } + + @Override + public TableSchema getSchema(TableDestination destination, + SideInputAccessor sideInputAccessor) { + return null; + } + + @Override + public TableDestination getTable(TableDestination destination, + SideInputAccessor sideInputAccessor) { + return destination; + } + + @Override + public Coder getDestinationCoder() { + return TableDestinationCoder.of(); + } + } + + /** + * Returns a tables based on a user-supplied function. + */ + static class TableFunctionDestinations extends DynamicDestinations { + private final SerializableFunction, TableDestination> tableFunction; + + TableFunctionDestinations( + SerializableFunction, TableDestination> tableFunction) { + this.tableFunction = tableFunction; + } + + @Override + public TableDestination getDestination(ValueInSingleWindow element) { + return tableFunction.apply(element); + } + + @Override + public TableSchema getSchema(TableDestination destination, + SideInputAccessor sideInputAccessor) { + return null; + } + + @Override + public TableDestination getTable(TableDestination destination, + SideInputAccessor sideInputAccessor) { + return destination; + } + + @Override + public Coder getDestinationCoder() { + return TableDestinationCoder.of(); + } + } + + /** + * Delegates all calls to an inner instance of {@link DynamicDestinations}. This allows + * subclasses to modify another instance of {@link DynamicDestinations} by subclassing and + * overriding just the methods they want to alter. + */ + static class DelegatingDynamicDestinations + extends DynamicDestinations { + private final DynamicDestinations inner; + DelegatingDynamicDestinations(DynamicDestinations inner) { + this.inner = inner; + } + @Override + public DestinationT getDestination(ValueInSingleWindow element) { + return inner.getDestination(element); + } + + @Override + public TableSchema getSchema(DestinationT destination, SideInputAccessor sideInputAccessor) { + return inner.getSchema(destination, sideInputAccessor); + } + + @Override + public TableDestination getTable(DestinationT destination, + SideInputAccessor sideInputAccessor) { + return inner.getTable(destination, sideInputAccessor); + } + + @Override + public Coder getDestinationCoder() { + return inner.getDestinationCoder(); + } + } + + /** + * Returns the same schema for every table. + */ + static class ConstantSchemaDestinations + extends DelegatingDynamicDestinations { + private final @Nullable + ValueProvider jsonSchema; + + ConstantSchemaDestinations(DynamicDestinations inner, + ValueProvider jsonSchema) { + super(inner); + this.jsonSchema = jsonSchema; + } + + @Override + public TableSchema getSchema(TableDestination destination, + SideInputAccessor sideInputAccessor) { + return BigQueryHelpers.fromJsonString(jsonSchema.get(), TableSchema.class); + } + } + + /** + * Takes in a side input mapping tablespec to json table schema, and always returns the + * matching schema from the side input. + */ + static class SchemaFromViewDestinations + extends DelegatingDynamicDestinations { + SchemaFromViewDestinations(DynamicDestinations inner, + PCollectionView> schemaView) { + super(inner); + setSideInputRequired(schemaView); + } + + @Override + public TableSchema getSchema(TableDestination destination, + SideInputAccessor sideInputAccessor) { + Map mapValue = sideInputAccessor.getSideInputValue(); + return BigQueryHelpers.fromJsonString(mapValue.get(destination.getTableSpec()), + TableSchema.class); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 343c2116631d9..bd87456b57374 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -44,7 +44,7 @@ public class StreamingInserts this.bigQueryServices = new BigQueryServicesImpl(); } - StreamingInserts withTestServices(BigQueryServices bigQueryServices) { + StreamingInserts setTestServices(BigQueryServices bigQueryServices) { this.bigQueryServices = bigQueryServices; return this; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java index 4d130b6066345..20b47e10f9a0e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java @@ -22,6 +22,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -77,6 +80,11 @@ public WriteResult expand(PCollection> input) { tagged .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of())) .apply(Reshuffle., TableRowInfo>of()) + // Put in the global window to ensure that DynamicDestinations side inputs are accessed + // correctly. + .apply("GlobalWindow", + Window., TableRowInfo>>into(new GlobalWindows()) + .triggering(DefaultTrigger.of()).discardingFiredPanes()) .apply("StreamingWrite", ParDo.of( new StreamingWriteFn(bigQueryServices))); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java index c41880450124f..8a06d1397849f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -41,7 +41,7 @@ public void encode(TableDestination value, OutputStream outStream, Context conte throw new CoderException("cannot encode a null value"); } stringCoder.encode(value.getTableSpec(), outStream, context.nested()); - stringCoder.encode(value.getTableDescription(), outStream, context); + stringCoder.encode(value.getTableDescription(), outStream, context.nested()); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index 5f890678c6455..4f609b22033e8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -26,6 +26,7 @@ import java.io.Serializable; import java.util.Map; import java.util.UUID; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -37,64 +38,66 @@ import org.slf4j.LoggerFactory; /** - * Writes each bundle of {@link TableRow} elements out to a separate file using - * {@link TableRowWriter}. + * Writes each bundle of {@link TableRow} elements out to a separate file using {@link + * TableRowWriter}. */ -class WriteBundlesToFiles extends DoFn, WriteBundlesToFiles.Result> { +class WriteBundlesToFiles + extends DoFn, WriteBundlesToFiles.Result> { private static final Logger LOG = LoggerFactory.getLogger(WriteBundlesToFiles.class); // Map from tablespec to a writer for that table. - private transient Map writers; + private transient Map writers; private final String tempFilePrefix; /** * The result of the {@link WriteBundlesToFiles} transform. Corresponds to a single output file, * and encapsulates the table it is destined to as well as the file byte size. */ - public static final class Result implements Serializable { + public static final class Result implements Serializable { private static final long serialVersionUID = 1L; public final String filename; public final Long fileByteSize; - public final TableDestination tableDestination; + public final DestinationT destination; - public Result(String filename, Long fileByteSize, TableDestination tableDestination) { + public Result(String filename, Long fileByteSize, DestinationT destination) { this.filename = filename; this.fileByteSize = fileByteSize; - this.tableDestination = tableDestination; + this.destination = destination; } } - /** - * a coder for the {@link Result} class. - */ - public static class ResultCoder extends CustomCoder { - private static final ResultCoder INSTANCE = new ResultCoder(); + /** a coder for the {@link Result} class. */ + public static class ResultCoder extends CustomCoder> { private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); private static final VarLongCoder longCoder = VarLongCoder.of(); - private static final TableDestinationCoder tableDestinationCoder = TableDestinationCoder.of(); + private final Coder destinationCoder; - public static ResultCoder of() { - return INSTANCE; + public static ResultCoder of( + Coder destinationCoder) { + return new ResultCoder<>(destinationCoder); + } + + ResultCoder(Coder destinationCoder) { + this.destinationCoder = destinationCoder; } @Override - public void encode(Result value, OutputStream outStream, Context context) + public void encode(Result value, OutputStream outStream, Context context) throws IOException { if (value == null) { throw new CoderException("cannot encode a null value"); } stringCoder.encode(value.filename, outStream, context.nested()); longCoder.encode(value.fileByteSize, outStream, context.nested()); - tableDestinationCoder.encode(value.tableDestination, outStream, context.nested()); + destinationCoder.encode(value.destination, outStream, context.nested()); } @Override - public Result decode(InputStream inStream, Context context) - throws IOException { + public Result decode(InputStream inStream, Context context) throws IOException { String filename = stringCoder.decode(inStream, context.nested()); long fileByteSize = longCoder.decode(inStream, context.nested()); - TableDestination tableDestination = tableDestinationCoder.decode(inStream, context.nested()); - return new Result(filename, fileByteSize, tableDestination); + DestinationT destination = destinationCoder.decode(inStream, context.nested()); + return new Result<>(filename, fileByteSize, destination); } @Override @@ -138,9 +141,9 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - for (Map.Entry entry : writers.entrySet()) { + for (Map.Entry entry : writers.entrySet()) { TableRowWriter.Result result = entry.getValue().close(); - c.output(new Result(result.resourceId.toString(), result.byteSize, entry.getKey())); + c.output(new Result<>(result.resourceId.toString(), result.byteSize, entry.getKey())); } writers.clear(); } @@ -149,8 +152,7 @@ public void finishBundle(Context c) throws Exception { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("tempFilePrefix", tempFilePrefix) - .withLabel("Temporary File Prefix")); + builder.addIfNotNull( + DisplayData.item("tempFilePrefix", tempFilePrefix).withLabel("Temporary File Prefix")); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java index 0ae1768ee1636..4136fa06084a5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.gcp.bigquery; -import com.google.api.services.bigquery.model.TableReference; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.util.List; @@ -27,7 +26,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; -import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; @@ -37,12 +35,12 @@ * Partitions temporary files based on number of files and file sizes. Output key is a pair of * tablespec and the list of files corresponding to each partition of that table. */ -class WritePartition extends DoFn, List>> { - private final ValueProvider singletonOutputJsonTableRef; - private final String singletonOutputTableDescription; - private final PCollectionView> resultsView; - private TupleTag, List>> multiPartitionsTag; - private TupleTag, List>> singlePartitionTag; +class WritePartition + extends DoFn, List>> { + private final boolean singletonTable; + private final PCollectionView>> resultsView; + private TupleTag, List>> multiPartitionsTag; + private TupleTag, List>> singlePartitionTag; private static class PartitionData { private int numFiles = 0; @@ -66,7 +64,7 @@ void addBytes(long numBytes) { } List getFilenames() { - return filenames; + return filenames; } void addFilename(String filename) { @@ -98,18 +96,16 @@ PartitionData getLatestPartition() { } void addPartition(PartitionData partition) { - partitions.add(partition); + partitions.add(partition); } } WritePartition( - ValueProvider singletonOutputJsonTableRef, - String singletonOutputTableDescription, - PCollectionView> resultsView, - TupleTag, List>> multiPartitionsTag, - TupleTag, List>> singlePartitionTag) { - this.singletonOutputJsonTableRef = singletonOutputJsonTableRef; - this.singletonOutputTableDescription = singletonOutputTableDescription; + boolean singletonTable, + PCollectionView>> resultsView, + TupleTag, List>> multiPartitionsTag, + TupleTag, List>> singlePartitionTag) { + this.singletonTable = singletonTable; this.resultsView = resultsView; this.multiPartitionsTag = multiPartitionsTag; this.singlePartitionTag = singlePartitionTag; @@ -117,30 +113,31 @@ void addPartition(PartitionData partition) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - List results = Lists.newArrayList(c.sideInput(resultsView)); + List> results = + Lists.newArrayList(c.sideInput(resultsView)); // If there are no elements to write _and_ the user specified a constant output table, then // generate an empty table of that name. - if (results.isEmpty() && singletonOutputJsonTableRef != null) { - TableReference singletonTable = BigQueryHelpers.fromJsonString( - singletonOutputJsonTableRef.get(), TableReference.class); - if (singletonTable != null) { + if (results.isEmpty() && singletonTable) { TableRowWriter writer = new TableRowWriter(c.element()); writer.open(UUID.randomUUID().toString()); TableRowWriter.Result writerResult = writer.close(); - results.add(new Result(writerResult.resourceId.toString(), writerResult.byteSize, - new TableDestination(singletonTable, singletonOutputTableDescription))); - } + // Return a null destination in this case - the constant DynamicDestinations class will + // resolve it to the singleton output table. + results.add( + new Result( + writerResult.resourceId.toString(), + writerResult.byteSize, + null)); } - - Map currentResults = Maps.newHashMap(); - for (WriteBundlesToFiles.Result fileResult : results) { - TableDestination tableDestination = fileResult.tableDestination; - DestinationData destinationData = currentResults.get(tableDestination); + Map currentResults = Maps.newHashMap(); + for (WriteBundlesToFiles.Result fileResult : results) { + DestinationT destination = fileResult.destination; + DestinationData destinationData = currentResults.get(destination); if (destinationData == null) { destinationData = new DestinationData(); - currentResults.put(tableDestination, destinationData); + currentResults.put(destination, destinationData); } PartitionData latestPartition = destinationData.getLatestPartition(); @@ -156,18 +153,18 @@ public void processElement(ProcessContext c) throws Exception { // Now that we've figured out which tables and partitions to write out, emit this information // to the next stage. - for (Map.Entry entry : currentResults.entrySet()) { - TableDestination tableDestination = entry.getKey(); + for (Map.Entry entry : currentResults.entrySet()) { + DestinationT destination = entry.getKey(); DestinationData destinationData = entry.getValue(); // In the fast-path case where we only output one table, the transform loads it directly // to the final table. In this case, we output on a special TupleTag so the enclosing // transform knows to skip the rename step. - TupleTag, List>> outputTag = + TupleTag, List>> outputTag = (destinationData.getPartitions().size() == 1) ? singlePartitionTag : multiPartitionsTag; for (int i = 0; i < destinationData.getPartitions().size(); ++i) { PartitionData partitionData = destinationData.getPartitions().get(i); - c.output(outputTag, KV.of(ShardedKey.of(tableDestination, i + 1), - partitionData.getFilenames())); + c.output( + outputTag, KV.of(ShardedKey.of(destination, i + 1), partitionData.getFilenames())); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index a1c909f2066bd..7d7e7c700181a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -23,6 +23,7 @@ import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import java.io.IOException; import java.nio.file.Files; @@ -55,73 +56,88 @@ /** * Writes partitions to BigQuery tables. * - *

The input is a list of files corresponding to each partition of a table. These files are load - * into a temporary table (or into the final table if there is only one partition). The output is a - * {@link KV} mapping each final table to a list of the temporary tables containing its data. + *

The input is a list of files corresponding to each partition of a table. loadThese files are + * loaded into a temporary table (or into the final table if there is only one partition). The + * output is a {@link KV} mapping each final table to a list of the temporary tables containing its + * data. * *

In the case where all the data in the files fit into a single load job, this transform loads * the data directly into the final table, skipping temporary tables. In this case, the output * {@link KV} maps the final table to itself. */ -class WriteTables - extends DoFn, List>, KV> { +class WriteTables + extends DoFn, List>, KV> { private static final Logger LOG = LoggerFactory.getLogger(WriteTables.class); private final boolean singlePartition; private final BigQueryServices bqServices; private final PCollectionView jobIdToken; - private final PCollectionView> schemaMapView; + private final PCollectionView> schemasView; private final String tempFilePrefix; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; + private final DynamicDestinations dynamicDestinations; public WriteTables( boolean singlePartition, BigQueryServices bqServices, PCollectionView jobIdToken, - PCollectionView> schemaMapView, + PCollectionView> schemasView, String tempFilePrefix, WriteDisposition writeDisposition, - CreateDisposition createDisposition) { + CreateDisposition createDisposition, + DynamicDestinations dynamicDestinations) { this.singlePartition = singlePartition; this.bqServices = bqServices; this.jobIdToken = jobIdToken; - this.schemaMapView = schemaMapView; + this.schemasView = schemasView; this.tempFilePrefix = tempFilePrefix; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; + this.dynamicDestinations = dynamicDestinations; } @ProcessElement public void processElement(ProcessContext c) throws Exception { - Map schemaMap = c.sideInput(schemaMapView); + // If the DynamicDestinations class wants to read a side input, give it the value. + DynamicDestinations.SideInputAccessor sideInputAccessor = + new DynamicDestinations.SideInputAccessor( + c, dynamicDestinations.getSideInput()); + + DestinationT destination = c.element().getKey().getKey(); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(schemasView).get(destination), TableSchema.class); + TableDestination tableDestination = + dynamicDestinations.getTable(destination, sideInputAccessor); + TableReference tableReference = tableDestination.getTableReference(); + if (Strings.isNullOrEmpty(tableReference.getProjectId())) { + tableReference.setProjectId( + c.getPipelineOptions().as(BigQueryOptions.class).getProject()); + tableDestination = new TableDestination( + tableReference, tableDestination.getTableDescription()); + } - TableDestination tableDestination = c.element().getKey().getKey(); Integer partition = c.element().getKey().getShardNumber(); List partitionFiles = Lists.newArrayList(c.element().getValue()); String jobIdPrefix = BigQueryHelpers.createJobId(c.sideInput(jobIdToken), tableDestination, partition); - TableSchema schema = - BigQueryHelpers.fromJsonString( - schemaMap.get(tableDestination.getTableSpec()), TableSchema.class); - - TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { - ref.setTableId(jobIdPrefix); + tableReference.setTableId(jobIdPrefix); } load( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, - ref, - schema, + tableReference, + tableSchema, partitionFiles, writeDisposition, createDisposition, tableDestination.getTableDescription()); - c.output(KV.of(tableDestination, BigQueryHelpers.toJsonString(ref))); + c.output(KV.of(tableDestination, BigQueryHelpers.toJsonString(tableReference))); removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partitionFiles); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index e7b5244b088b2..02c9d0523f6d0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -66,6 +66,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; @@ -449,6 +451,94 @@ public void processElement(ProcessContext c) throws Exception { p.run(); } + // Create an intermediate type to ensure that coder inference up the inheritance tree is tested. + abstract static class StringIntegerDestinations extends DynamicDestinations { + } + + @Test + public void testWriteDynamicDestinationsBatch() throws Exception { + writeDynamicDestinations(false); + } + + @Test + public void testWriteDynamicDestinationsStreaming() throws Exception { + writeDynamicDestinations(true); + } + + public void writeDynamicDestinations(boolean streaming) throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("project-id"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); + + datasetService.createDataset("project-id", "dataset-id", "", ""); + + final Pattern userPattern = Pattern.compile("([a-z]+)([0-9]+)"); + Pipeline p = TestPipeline.create(bqOptions); + PCollection users = p.apply(Create.of("bill1", "sam2", "laurence3") + .withCoder(StringUtf8Coder.of())); + if (streaming) { + users = users.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + } + users.apply(BigQueryIO.write() + .withTestServices(fakeBqServices) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withFormatFunction(new SerializableFunction() { + @Override + public TableRow apply(String user) { + Matcher matcher = userPattern.matcher(user); + if (matcher.matches()) { + return new TableRow().set("name", matcher.group(1)) + .set("id", Integer.valueOf(matcher.group(2))); + } + throw new RuntimeException("Unmatching element " + user); + } + }) + .to(new StringIntegerDestinations() { + @Override + public Integer getDestination(ValueInSingleWindow element) { + Matcher matcher = userPattern.matcher(element.getValue()); + if (matcher.matches()) { + // Since we name tables by userid, we can simply store an Integer to represent + // a table. + return Integer.valueOf(matcher.group(2)); + } + throw new RuntimeException("Unmatching destination " + element.getValue()); + } + + @Override + public TableDestination getTable(Integer userId, SideInputAccessor sideInput) { + // Each user in it's own table. + return new TableDestination("dataset-id.userid-" + userId, + "table for userid " + userId); + } + + @Override + public TableSchema getSchema(Integer userId, SideInputAccessor sideInput) { + return new TableSchema().setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("id").setType("INTEGER"))); + } + }) + .withoutValidation()); + p.run(); + + File tempDir = new File(bqOptions.getTempLocation()); + testNumFiles(tempDir, 0); + + assertThat(datasetService.getAllRows("project-id", "dataset-id", "userid-1"), + containsInAnyOrder(new TableRow().set("name", "bill").set("id", 1))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "userid-2"), + containsInAnyOrder(new TableRow().set("name", "sam").set("id", 2))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "userid-3"), + containsInAnyOrder(new TableRow().set("name", "laurence").set("id", 3))); + } + @Test public void testWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); @@ -1596,72 +1686,59 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file // In the case where a static destination is specified (i.e. not through a dynamic table // function) and there is no input data, WritePartition will generate an empty table. This // code is to test that path. - TableReference singletonReference = new TableReference() - .setProjectId("projectid") - .setDatasetId("dataset") - .setTableId("table"); - String singletonDescription = "singleton"; boolean isSingleton = numTables == 1 && numFilesPerTable == 0; - List> expectedPartitions = Lists.newArrayList(); + List> expectedPartitions = Lists.newArrayList(); if (isSingleton) { - expectedPartitions.add(ShardedKey.of( - new TableDestination(singletonReference, singletonDescription), 1)); + expectedPartitions.add(ShardedKey.of(null, 1)); } else { for (int i = 0; i < numTables; ++i) { for (int j = 1; j <= expectedNumPartitionsPerTable; ++j) { String tableName = String.format("project-id:dataset-id.tables%05d", i); - TableDestination destination = new TableDestination(tableName, tableName); - expectedPartitions.add(ShardedKey.of(destination, j)); + expectedPartitions.add(ShardedKey.of(tableName, j)); } } } - List files = Lists.newArrayList(); - Map> filenamesPerTable = Maps.newHashMap(); + List> files = Lists.newArrayList(); + Map> filenamesPerTable = Maps.newHashMap(); for (int i = 0; i < numTables; ++i) { String tableName = String.format("project-id:dataset-id.tables%05d", i); - TableDestination destination = new TableDestination(tableName, tableName); - List filenames = filenamesPerTable.get(destination); + List filenames = filenamesPerTable.get(tableName); if (filenames == null) { filenames = Lists.newArrayList(); - filenamesPerTable.put(destination, filenames); + filenamesPerTable.put(tableName, filenames); } for (int j = 0; j < numFilesPerTable; ++j) { String fileName = String.format("%s_files%05d", tableName, j); filenames.add(fileName); - files.add(new Result(fileName, fileSize, destination)); + files.add(new Result<>(fileName, fileSize, tableName)); } } - TupleTag, List>> multiPartitionsTag = - new TupleTag, List>>("multiPartitionsTag") {}; - TupleTag, List>> singlePartitionTag = - new TupleTag, List>>("singlePartitionTag") {}; + TupleTag, List>> multiPartitionsTag = + new TupleTag, List>>("multiPartitionsTag") {}; + TupleTag, List>> singlePartitionTag = + new TupleTag, List>>("singlePartitionTag") {}; - PCollection filesPCollection = - p.apply(Create.of(files).withType(new TypeDescriptor() {})); - PCollectionView> resultsView = + PCollection> filesPCollection = + p.apply(Create.of(files) + .withCoder(WriteBundlesToFiles.ResultCoder.of(StringUtf8Coder.of()))); + PCollectionView>> resultsView = PCollectionViews.iterableView( filesPCollection, WindowingStrategy.globalDefault(), - WriteBundlesToFiles.ResultCoder.of()); + WriteBundlesToFiles.ResultCoder.of(StringUtf8Coder.of())); - ValueProvider singletonTable = null; - if (isSingleton) { - singletonTable = StaticValueProvider.of(BigQueryHelpers.toJsonString(singletonReference)); - } - WritePartition writePartition = - new WritePartition(singletonTable, - "singleton", resultsView, - multiPartitionsTag, singlePartitionTag); + WritePartition writePartition = + new WritePartition<>(isSingleton, resultsView, multiPartitionsTag, singlePartitionTag); - DoFnTester, List>> tester = + DoFnTester, List>> tester = DoFnTester.of(writePartition); tester.setSideInput(resultsView, GlobalWindow.INSTANCE, files); tester.processElement(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); - List, List>> partitions; + List, List>> partitions; if (expectedNumPartitionsPerTable > 1) { partitions = tester.takeOutputElements(multiPartitionsTag); } else { @@ -1669,10 +1746,10 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file } - List> partitionsResult = Lists.newArrayList(); - Map> filesPerTableResult = Maps.newHashMap(); - for (KV, List> partition : partitions) { - TableDestination table = partition.getKey().getKey(); + List> partitionsResult = Lists.newArrayList(); + Map> filesPerTableResult = Maps.newHashMap(); + for (KV, List> partition : partitions) { + String table = partition.getKey().getKey(); partitionsResult.add(partition.getKey()); List tableFilesResult = filesPerTableResult.get(table); if (tableFilesResult == null) { @@ -1696,6 +1773,22 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file } } + static class IdentityDynamicTables extends DynamicDestinations { + @Override + public String getDestination(ValueInSingleWindow element) { + throw new UnsupportedOperationException("getDestination not expected in this test."); + } + + @Override + public TableDestination getTable(String destination, SideInputAccessor sideInputAccessor) { + return new TableDestination(destination, destination); + } + + @Override + public TableSchema getSchema(String destination, SideInputAccessor sideInputAccessor) { + throw new UnsupportedOperationException("getSchema not expected in this test."); + } + } @Test public void testWriteTables() throws Exception { p.enableAbandonedNodeEnforcement(false); @@ -1714,7 +1807,7 @@ public void testWriteTables() throws Exception { Path baseDir = Files.createTempDirectory(tempFolder, "testWriteTables"); - List, List>> partitions = Lists.newArrayList(); + List, List>> partitions = Lists.newArrayList(); for (int i = 0; i < numTables; ++i) { String tableName = String.format("project-id:dataset-id.table%05d", i); TableDestination tableDestination = new TableDestination(tableName, tableName); @@ -1733,7 +1826,7 @@ public void testWriteTables() throws Exception { } filesPerPartition.add(filename); } - partitions.add(KV.of(ShardedKey.of(tableDestination, j), + partitions.add(KV.of(ShardedKey.of(tableDestination.getTableSpec(), j), filesPerPartition)); List expectedTables = expectedTempTables.get(tableDestination); @@ -1752,28 +1845,30 @@ public void testWriteTables() throws Exception { .apply("CreateJobId", Create.of("jobId")) .apply(View.asSingleton()); - PCollectionView> schemaMapView = p.apply("CreateEmptySchema", Create.empty( + PCollectionView> schemaMapView = p.apply("CreateEmptySchema", Create + .empty( new TypeDescriptor>() {})) .apply(View.asMap()); - WriteTables writeTables = new WriteTables( + WriteTables writeTables = new WriteTables<>( false, fakeBqServices, jobIdTokenView, schemaMapView, tempFilePrefix, WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED); + CreateDisposition.CREATE_IF_NEEDED, + new IdentityDynamicTables()); - DoFnTester, List>, + DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); tester.setSideInput(schemaMapView, GlobalWindow.INSTANCE, ImmutableMap.of()); - for (KV, List> partition : partitions) { + for (KV, List> partition : partitions) { tester.processElement(partition); } Map> tempTablesResult = Maps.newHashMap(); - for (KV element : tester.takeOutputElements()) { + for (KV element : tester.takeOutputElements()) { List tables = tempTablesResult.get(element.getKey()); if (tables == null) { tables = Lists.newArrayList(); From 581d8a687bdc22b8c877e60b107a4d2c616d6df6 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 2 May 2017 18:29:15 -0700 Subject: [PATCH 4/6] Undo some formatting changes --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 284 ++++++++---------- .../sdk/io/gcp/bigquery/CalculateSchemas.java | 5 +- 2 files changed, 126 insertions(+), 163 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index ad92bca947796..07348dfec7ac9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -83,17 +83,17 @@ import org.slf4j.LoggerFactory; /** - * {@link PTransform}s for reading and writing BigQuery tables. + * {@link PTransform}s for reading and writing + * BigQuery tables. * *

Table References

* *

A fully-qualified BigQuery table name consists of three components: - * *

    - *
  • {@code projectId}: the Cloud project id (defaults to {@link GcpOptions#getProject()}). - *
  • {@code datasetId}: the BigQuery dataset id, unique within a project. - *
  • {@code tableId}: a table id, unique within a dataset. + * < li>{@code projectId}: the Cloud project id (defaults to + * {@link GcpOptions#getProject()}). + *
  • {@code datasetId}: the BigQuery dataset id, unique within a project. + *
  • {@code tableId}: a table id, unique within a dataset. *
* *

BigQuery table references are stored as a {@link TableReference}, which comes from the - *

  • [{@code project_id}]:[{@code dataset_id}].[{@code table_id}] - *
  • [{@code dataset_id}].[{@code table_id}] + *
  • [{@code project_id}]:[{@code dataset_id}].[{@code table_id}] + *
  • [{@code dataset_id}].[{@code table_id}] * * *

    Reading

    * - *

    To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. This produces a - * {@link PCollection} of {@link TableRow TableRows} as output: + *

    To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. + * This produces a {@link PCollection} of {@link TableRow TableRows} as output: * *

    {@code
      * PCollection weatherData = pipeline.apply(
    @@ -133,11 +133,12 @@
      *
      * 

    Writing

    * - *

    To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. This consumes - * either a {@link PCollection} of {@link TableRow TableRows} as input when using {@link - * BigQueryIO#writeTableRows()} or of a user-defined type when using {@link BigQueryIO#write()}. - * When using a user-defined type, a function must be provided to turn this type into a {@link - * TableRow} using {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. + *

    To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. + * This consumes either a {@link PCollection} of {@link TableRow TableRows} as input when using + * {@link BigQueryIO#writeTableRows()} or of a user-defined type when using + * {@link BigQueryIO#write()}. When using a user-defined type, a function must be provided to + * turn this type into a {@link TableRow} using + * {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. * *

    {@code
      * PCollection quotes = ...
    @@ -199,8 +200,8 @@
      * can also be useful when writing to a single table, as it allows a previous stage to calculate the
      * schema (possibly based on the full collection of records being written to BigQuery).
      *
    - * 

    For the most general form of dynamic table destinations and schemas, look at {@link - * BigQueryIO.Write#to(DynamicDestinations)}. + *

    For the most general form of dynamic table destinations and schemas, look at + * {@link BigQueryIO.Write#to(DynamicDestinations)}. * *

    Permissions

    * @@ -214,20 +215,27 @@ public class BigQueryIO { private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); - /** Singleton instance of the JSON factory used to read and write JSON formatted rows. */ + /** + * Singleton instance of the JSON factory used to read and write JSON formatted rows. + */ static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); /** - * Project IDs must contain 6-63 lowercase letters, digits, or dashes. IDs must start with a - * letter and may not end with a dash. This regex isn't exact - this allows for patterns that - * would be rejected by the service, but this is sufficient for basic parsing of table references. + * Project IDs must contain 6-63 lowercase letters, digits, or dashes. + * IDs must start with a letter and may not end with a dash. + * This regex isn't exact - this allows for patterns that would be rejected by + * the service, but this is sufficient for basic parsing of table references. */ private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]"; - /** Regular expression that matches Dataset IDs. */ + /** + * Regular expression that matches Dataset IDs. + */ private static final String DATASET_REGEXP = "[-\\w.]{1,1024}"; - /** Regular expression that matches Table IDs. */ + /** + * Regular expression that matches Table IDs. + */ private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}"; /** @@ -235,15 +243,14 @@ public class BigQueryIO { * {@code "[dataset_id].[table_id]"}. */ private static final String DATASET_TABLE_REGEXP = - String.format( - "((?%s):)?(?%s)\\.(?
  • %s)", - PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP); + String.format("((?%s):)?(?%s)\\.(?
    %s)", PROJECT_ID_REGEXP, + DATASET_REGEXP, TABLE_REGEXP); static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP); /** - * A formatting function that maps a TableRow to itself. This allows sending a {@code - * PCollection} directly to BigQueryIO.Write. + * A formatting function that maps a TableRow to itself. This allows sending a + * {@code PCollection} directly to BigQueryIO.Write. */ static final SerializableFunction IDENTITY_FORMATTER = new SerializableFunction() { @@ -254,11 +261,11 @@ public TableRow apply(TableRow input) { }; /** - * A {@link PTransform} that reads from a BigQuery table and returns a {@link PCollection} of - * {@link TableRow TableRows} containing each of the rows of the table. + * A {@link PTransform} that reads from a BigQuery table and returns a + * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table. * - *

    Each {@link TableRow} contains values indexed by column name. Here is a sample processing - * function that processes a "line" column from rows: + *

    Each {@link TableRow} contains values indexed by column name. Here is a + * sample processing function that processes a "line" column from rows: * *

    {@code
        * static class ExtractWordsFn extends DoFn {
    @@ -285,38 +292,22 @@ public static Read read() {
       /** Implementation of {@link #read}. */
       @AutoValue
       public abstract static class Read extends PTransform> {
    -    @Nullable
    -    abstract ValueProvider getJsonTableRef();
    -
    -    @Nullable
    -    abstract ValueProvider getQuery();
    -
    +    @Nullable abstract ValueProvider getJsonTableRef();
    +    @Nullable abstract ValueProvider getQuery();
         abstract boolean getValidate();
    -
    -    @Nullable
    -    abstract Boolean getFlattenResults();
    -
    -    @Nullable
    -    abstract Boolean getUseLegacySql();
    -
    +    @Nullable abstract Boolean getFlattenResults();
    +    @Nullable abstract Boolean getUseLegacySql();
         abstract BigQueryServices getBigQueryServices();
    -
         abstract Builder toBuilder();
     
         @AutoValue.Builder
         abstract static class Builder {
           abstract Builder setJsonTableRef(ValueProvider jsonTableRef);
    -
           abstract Builder setQuery(ValueProvider query);
    -
           abstract Builder setValidate(boolean validate);
    -
           abstract Builder setFlattenResults(Boolean flattenResults);
    -
           abstract Builder setUseLegacySql(Boolean useLegacySql);
    -
           abstract Builder setBigQueryServices(BigQueryServices bigQueryServices);
    -
           abstract Read build();
         }
     
    @@ -327,8 +318,8 @@ private void ensureFromNotCalledYet() {
         }
     
         /**
    -     * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or {@code
    -     * "[dataset_id].[table_id]"} for tables within the current project.
    +     * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or
    +     * {@code "[dataset_id].[table_id]"} for tables within the current project.
          */
         public Read from(String tableSpec) {
           return from(StaticValueProvider.of(tableSpec));
    @@ -359,20 +350,24 @@ public Read fromQuery(String query) {
           return fromQuery(StaticValueProvider.of(query));
         }
     
    -    /** Same as {@code fromQuery(String)}, but with a {@link ValueProvider}. */
    +    /**
    +     * Same as {@code fromQuery(String)}, but with a {@link ValueProvider}.
    +     */
         public Read fromQuery(ValueProvider query) {
           ensureFromNotCalledYet();
           return toBuilder().setQuery(query).setFlattenResults(true).setUseLegacySql(true).build();
         }
     
    -    /** Read from table specified by a {@link TableReference}. */
    +    /**
    +     * Read from table specified by a {@link TableReference}.
    +     */
         public Read from(TableReference table) {
           return from(StaticValueProvider.of(BigQueryHelpers.toTableSpec(table)));
         }
     
         private static final String QUERY_VALIDATION_FAILURE_ERROR =
             "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
    -            + " pipeline, This validation can be disabled using #withoutValidation.";
    +        + " pipeline, This validation can be disabled using #withoutValidation.";
     
         /**
          * Disable validation that the table exists or the query succeeds prior to pipeline submission.
    @@ -463,9 +458,7 @@ public void validate(PipelineOptions options) {
           // Note that a table or query check can fail if the table or dataset are created by
           // earlier stages of the pipeline or if a query depends on earlier stages of a pipeline.
           // For these cases the withoutValidation method can be used to disable the check.
    -      if (getValidate()
    -          && table != null
    -          && table.isAccessible()
    +      if (getValidate() && table != null && table.isAccessible()
               && table.get().getProjectId() != null) {
             checkState(table.isAccessible(), "Cannot call validate if table is dynamically set.");
             // Check for source table presence for early failure notification.
    @@ -493,11 +486,10 @@ public void validate(PipelineOptions options) {
         public PCollection expand(PBegin input) {
           String stepUuid = BigQueryHelpers.randomUUIDString();
           BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
    -      ValueProvider jobUuid =
    -          NestedValueProvider.of(
    +      ValueProvider jobUuid = NestedValueProvider.of(
                   StaticValueProvider.of(bqOptions.getJobName()), new CreatePerBeamJobUuid(stepUuid));
    -      final ValueProvider jobIdToken =
    -          NestedValueProvider.of(jobUuid, new BeamJobUuidToBigQueryJobUuid());
    +      final ValueProvider jobIdToken = NestedValueProvider.of(
    +          jobUuid, new BeamJobUuidToBigQueryJobUuid());
     
           BoundedSource source;
     
    @@ -518,7 +510,8 @@ public PCollection expand(PBegin input) {
                 BigQueryQuerySource.create(
                     jobIdToken,
                     getQuery(),
    -                NestedValueProvider.of(jobUuid, new CreateJsonTableRefFromUuid(executingProject)),
    +                NestedValueProvider.of(
    +                    jobUuid, new CreateJsonTableRefFromUuid(executingProject)),
                     getFlattenResults(),
                     getUseLegacySql(),
                     extractDestinationDir,
    @@ -560,8 +553,7 @@ void cleanup(PipelineOptions options) throws Exception {
                   }
                 }
               };
    -      return input
    -          .getPipeline()
    +      return input.getPipeline()
               .apply(org.apache.beam.sdk.io.Read.from(source))
               .setCoder(getDefaultOutputCoder())
               .apply(new PassThroughThenCleanup(cleanupOperation));
    @@ -576,26 +568,26 @@ protected Coder getDefaultOutputCoder() {
         public void populateDisplayData(DisplayData.Builder builder) {
           super.populateDisplayData(builder);
           builder
    -          .addIfNotNull(
    -              DisplayData.item("table", BigQueryHelpers.displayTable(getTableProvider()))
    +          .addIfNotNull(DisplayData.item("table", BigQueryHelpers.displayTable(getTableProvider()))
                       .withLabel("Table"))
    -          .addIfNotNull(DisplayData.item("query", getQuery()).withLabel("Query"))
    -          .addIfNotNull(
    -              DisplayData.item("flattenResults", getFlattenResults())
    +          .addIfNotNull(DisplayData.item("query", getQuery())
    +              .withLabel("Query"))
    +          .addIfNotNull(DisplayData.item("flattenResults", getFlattenResults())
                       .withLabel("Flatten Query Results"))
    -          .addIfNotNull(
    -              DisplayData.item("useLegacySql", getUseLegacySql())
    +          .addIfNotNull(DisplayData.item("useLegacySql", getUseLegacySql())
                       .withLabel("Use Legacy SQL Dialect"))
    -          .addIfNotDefault(
    -              DisplayData.item("validation", getValidate()).withLabel("Validation Enabled"), true);
    +          .addIfNotDefault(DisplayData.item("validation", getValidate())
    +              .withLabel("Validation Enabled"),
    +              true);
         }
     
    -    /** Returns the table to read, or {@code null} if reading from a query instead. */
    +    /**
    +     * Returns the table to read, or {@code null} if reading from a query instead.
    +     */
         @Nullable
         public ValueProvider getTableProvider() {
           return getJsonTableRef() == null
    -          ? null
    -          : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef());
    +          ? null : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef());
         }
         /** Returns the table to read, or {@code null} if reading from a query instead. */
         @Nullable
    @@ -618,8 +610,7 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac
         JobStatistics jobStats = extractJob.getStatistics();
         List counts = jobStats.getExtract().getDestinationUriFileCounts();
         if (counts.size() != 1) {
    -      String errorMessage =
    -          (counts.size() == 0
    +      String errorMessage = (counts.size() == 0
                   ? "No destination uri file count received."
                   : String.format(
                       "More than one destination uri file count received. First two are %s, %s",
    @@ -642,20 +633,20 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac
     
       /**
        * A {@link PTransform} that writes a {@link PCollection} to a BigQuery table. A formatting
    -   * function must be provided to convert each input element into a {@link TableRow} using {@link
    -   * Write#withFormatFunction(SerializableFunction)}.
    +   * function must be provided to convert each input element into a {@link TableRow} using
    +   * {@link Write#withFormatFunction(SerializableFunction)}.
        *
        * 

    In BigQuery, each table has an encosing dataset. The dataset being written must already * exist. * - *

    By default, tables will be created if they do not exist, which corresponds to a {@link - * Write.CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's - * Jobs API. A schema must be provided (via {@link Write#withSchema(TableSchema)}), or else the - * transform may fail at runtime with an {@link IllegalArgumentException}. + *

    By default, tables will be created if they do not exist, which corresponds to a + * {@link Write.CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of + * BigQuery's Jobs API. A schema must be provided (via {@link Write#withSchema(TableSchema)}), + * or else the transform may fail at runtime with an {@link IllegalArgumentException}. * - *

    By default, writes require an empty table, which corresponds to a {@link - * Write.WriteDisposition#WRITE_EMPTY} disposition that matches the default of BigQuery's Jobs - * API. + *

    By default, writes require an empty table, which corresponds to + * a {@link Write.WriteDisposition#WRITE_EMPTY} disposition that matches the default of + * BigQuery's Jobs API. * *

    Here is a sample transform that produces TableRow values containing "word" and "count" * columns: @@ -707,32 +698,19 @@ public abstract static class Write extends PTransform, WriteRe // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery job finishes. static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; - @Nullable - abstract ValueProvider getJsonTableRef(); - - @Nullable - abstract SerializableFunction, TableDestination> getTableFunction(); - - @Nullable - abstract SerializableFunction getFormatFunction(); - - @Nullable - abstract DynamicDestinations getDynamicDestinations(); - - @Nullable - abstract PCollectionView> getSchemaFromView(); - - @Nullable - abstract ValueProvider getJsonSchema(); - + @Nullable abstract ValueProvider getJsonTableRef(); + @Nullable abstract SerializableFunction, TableDestination> + getTableFunction(); + @Nullable abstract SerializableFunction getFormatFunction(); + @Nullable abstract DynamicDestinations getDynamicDestinations(); + @Nullable abstract PCollectionView> getSchemaFromView(); + @Nullable abstract ValueProvider getJsonSchema(); abstract CreateDisposition getCreateDisposition(); - abstract WriteDisposition getWriteDisposition(); /** Table description. Default is empty. */ abstract String getTableDescription(); /** An option to indicate if table validation is desired. Default is true. */ abstract boolean getValidate(); - abstract BigQueryServices getBigQueryServices(); abstract Builder toBuilder(); @@ -740,26 +718,16 @@ public abstract static class Write extends PTransform, WriteRe @AutoValue.Builder abstract static class Builder { abstract Builder setJsonTableRef(ValueProvider jsonTableRef); - abstract Builder setTableFunction( SerializableFunction, TableDestination> tableFunction); - abstract Builder setFormatFunction(SerializableFunction formatFunction); - abstract Builder setDynamicDestinations(DynamicDestinations dynamicDestinations); - abstract Builder setSchemaFromView(PCollectionView> view); - abstract Builder setJsonSchema(ValueProvider jsonSchema); - abstract Builder setCreateDisposition(CreateDisposition createDisposition); - abstract Builder setWriteDisposition(WriteDisposition writeDisposition); - abstract Builder setTableDescription(String tableDescription); - abstract Builder setValidate(boolean validate); - abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); abstract Write build(); @@ -768,8 +736,8 @@ abstract Builder setTableFunction( /** * An enumeration type for the BigQuery create disposition strings. * - * @see + * @see + * * configuration.query.createDisposition in the BigQuery Jobs API */ public enum CreateDisposition { @@ -783,8 +751,8 @@ public enum CreateDisposition { /** * Specifies that tables should be created if needed. This is the default behavior. * - *

    Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}. This - * precondition is checked before starting a job. The schema is not required to match an + *

    Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}. + * This precondition is checked before starting a job. The schema is not required to match an * existing table's schema. * *

    When this transformation is executed, if the output table does not exist, the table is @@ -851,20 +819,24 @@ public Write to(ValueProvider tableSpec) { } /** - * Writes to table specified by the specified table function. The table is a function of {@link - * ValueInSingleWindow}, so can be determined by the value or by the window. + * Writes to table specified by the specified table function. The table is a function of + * {@link ValueInSingleWindow}, so can be determined by the value or by the window. */ public Write to( SerializableFunction, TableDestination> tableFunction) { return toBuilder().setTableFunction(tableFunction).build(); } - /** Writes to the table and schema specified by the {@link DynamicDestinations} object. */ + /** + * Writes to the table and schema specified by the {@link DynamicDestinations} object. + */ public Write to(DynamicDestinations dynamicDestinations) { return toBuilder().setDynamicDestinations(dynamicDestinations).build(); } - /** Formats the user's type into a {@link TableRow} to be written to BigQuery. */ + /** + * Formats the user's type into a {@link TableRow} to be written to BigQuery. + */ public Write withFormatFunction(SerializableFunction formatFunction) { return toBuilder().setFormatFunction(formatFunction).build(); } @@ -892,7 +864,9 @@ public Write withJsonSchema(String jsonSchema) { return withJsonSchema(StaticValueProvider.of(jsonSchema)); } - /** Same as {@link #withJsonSchema(String)} but using a deferred {@link ValueProvider}. */ + /** + * Same as {@link #withJsonSchema(String)} but using a deferred {@link ValueProvider}. + */ public Write withJsonSchema(ValueProvider jsonSchema) { return toBuilder().setJsonSchema(jsonSchema).build(); } @@ -939,19 +913,16 @@ public void validate(PipelineOptions pipelineOptions) { // We must have a destination to write to! checkState( - getTableFunction() != null - || getJsonTableRef() != null + getTableFunction() != null || getJsonTableRef() != null || getDynamicDestinations() != null, "must set the table reference of a BigQueryIO.Write transform"); - checkArgument( - getFormatFunction() != null, + checkArgument(getFormatFunction() != null, "A function must be provided to convert type into a TableRow. " + "use BigQueryIO.Write.withFormatFunction to provide a formatting function."); // Require a schema if creating one or more tables. - checkArgument( - getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED + checkArgument(getCreateDisposition() != CreateDisposition.CREATE_IF_NEEDED || getJsonSchema() != null || getDynamicDestinations() != null || getSchemaFromView() != null, @@ -1065,8 +1036,8 @@ protected Coder getDefaultOutputCoder() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.addIfNotNull( - DisplayData.item("table", getJsonTableRef()).withLabel("Table Reference")); + builder.addIfNotNull(DisplayData.item("table", getJsonTableRef()) + .withLabel("Table Reference")); if (getJsonSchema() != null) { builder.addIfNotNull(DisplayData.item("schema", getJsonSchema()).withLabel("Table Schema")); } else { @@ -1074,24 +1045,19 @@ public void populateDisplayData(DisplayData.Builder builder) { } if (getTableFunction() != null) { - builder.add( - DisplayData.item("tableFn", getTableFunction().getClass()) + builder.add(DisplayData.item("tableFn", getTableFunction().getClass()) .withLabel("Table Reference Function")); } builder - .add( - DisplayData.item("createDisposition", getCreateDisposition().toString()) + .add(DisplayData.item("createDisposition", getCreateDisposition().toString()) .withLabel("Table CreateDisposition")) - .add( - DisplayData.item("writeDisposition", getWriteDisposition().toString()) + .add(DisplayData.item("writeDisposition", getWriteDisposition().toString()) .withLabel("Table WriteDisposition")) - .addIfNotDefault( - DisplayData.item("validation", getValidate()).withLabel("Validation Enabled"), true) - .addIfNotDefault( - DisplayData.item("tableDescription", getTableDescription()) - .withLabel("Table Description"), - ""); + .addIfNotDefault(DisplayData.item("validation", getValidate()) + .withLabel("Validation Enabled"), true) + .addIfNotDefault(DisplayData.item("tableDescription", getTableDescription()) + .withLabel("Table Description"), ""); } /** @@ -1107,10 +1073,8 @@ ValueProvider getTableWithDefaultProject(BigQueryOptions bqOptio } if (!table.isAccessible()) { - LOG.info( - "Using a dynamic value for table input. This must contain a project" - + " in the table reference: {}", - table); + LOG.info("Using a dynamic value for table input. This must contain a project" + + " in the table reference: {}", table); return table; } if (Strings.isNullOrEmpty(table.get().getProjectId())) { @@ -1118,9 +1082,8 @@ ValueProvider getTableWithDefaultProject(BigQueryOptions bqOptio // the default project. TableReference tableRef = table.get(); tableRef.setProjectId(bqOptions.getProject()); - return NestedValueProvider.of( - StaticValueProvider.of(BigQueryHelpers.toJsonString(tableRef)), - new JsonTableRefToTableRef()); + return NestedValueProvider.of(StaticValueProvider.of( + BigQueryHelpers.toJsonString(tableRef)), new JsonTableRefToTableRef()); } return table; } @@ -1128,13 +1091,14 @@ ValueProvider getTableWithDefaultProject(BigQueryOptions bqOptio /** Returns the table reference, or {@code null}. */ @Nullable public ValueProvider getTable() { - return getJsonTableRef() == null - ? null - : NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); + return getJsonTableRef() == null ? null : + NestedValueProvider.of(getJsonTableRef(), new JsonTableRefToTableRef()); } } - /** Clear the cached map of created tables. Used for testing. */ + /** + * Clear the cached map of created tables. Used for testing. + */ @VisibleForTesting static void clearCreatedTables() { CreateTables.clearCreatedTables(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java index 071dbf342072e..c55f91e4b6038 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java @@ -1,6 +1,3 @@ -package org.apache.beam.sdk.io.gcp.bigquery; - -/** Created by relax on 4/30/17. */ /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -19,6 +16,8 @@ * limitations under the License. */ +package org.apache.beam.sdk.io.gcp.bigquery; + import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.Lists; From f6e6e01395a5a3321a6a8afe74d1b85a90fe693c Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 2 May 2017 20:32:04 -0700 Subject: [PATCH 5/6] Address final code review comments. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 18 ++----- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 35 ++++++------- .../sdk/io/gcp/bigquery/CalculateSchemas.java | 10 ++-- .../sdk/io/gcp/bigquery/CreateTables.java | 9 ++-- .../io/gcp/bigquery/DynamicDestinations.java | 50 ++++++++----------- .../bigquery/DynamicDestinationsHelpers.java | 17 +++++-- .../sdk/io/gcp/bigquery/StreamingInserts.java | 3 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 4 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 29 +++++------ 9 files changed, 78 insertions(+), 97 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index a2f31b4b8cdf1..4e14696fc90ac 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -61,6 +61,8 @@ class BatchLoads private BigQueryServices bigQueryServices; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; + // Indicates that we are writing to a constant single table. If this is the case, we will create + // the table, even if there is no data in it. private final boolean singletonTable; private final DynamicDestinations dynamicDestinations; private final Coder destinationCoder; @@ -77,9 +79,8 @@ class BatchLoads this.destinationCoder = destinationCoder; } - BatchLoads withTestServices(BigQueryServices bigQueryServices) { + void setTestServices(BigQueryServices bigQueryServices) { this.bigQueryServices = bigQueryServices; - return this; } @Override @@ -147,18 +148,11 @@ public String apply(String input) { PCollectionView> schemasView = inputInGlobalWindow.apply(new CalculateSchemas<>(dynamicDestinations)); - List> writeBundlesToFilesSideInputs = Lists.newArrayList(); - if (dynamicDestinations.getSideInput() != null) { - writeBundlesToFilesSideInputs.add(dynamicDestinations.getSideInput()); - } - // PCollection of filename, file byte size, and table destination. - PCollection> results = inputInGlobalWindow .apply("WriteBundlesToFiles", ParDo.of( - new WriteBundlesToFiles(tempFilePrefix)) - .withSideInputs(writeBundlesToFilesSideInputs)) + new WriteBundlesToFiles(tempFilePrefix))) .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); TupleTag, List>> multiPartitionsTag = @@ -188,9 +182,7 @@ public String apply(String input) { List> writeTablesSideInputs = Lists.newArrayList(jobIdTokenView, schemasView); - if (dynamicDestinations.getSideInput() != null) { - writeTablesSideInputs.add(dynamicDestinations.getSideInput()); - } + writeTablesSideInputs.addAll(dynamicDestinations.getSideInputs()); Coder, List>> partitionsCoder = KvCoder.of( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 07348dfec7ac9..51807d7718f04 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -930,18 +930,14 @@ public void validate(PipelineOptions pipelineOptions) { List allToArgs = Lists.newArrayList(getJsonTableRef(), getTableFunction(), getDynamicDestinations()); - checkArgument( - 1 - == Iterables.size( - Iterables.filter(allToArgs, Predicates.notNull())), + checkArgument(1 + == Iterables.size(Iterables.filter(allToArgs, Predicates.notNull())), "Exactly one of jsonTableRef, tableFunction, or " + "dynamicDestinations must be set"); List allSchemaArgs = Lists.newArrayList(getJsonSchema(), getSchemaFromView(), getDynamicDestinations()); - checkArgument( - 2 - > Iterables.size( - Iterables.filter(allSchemaArgs, Predicates.notNull())), + checkArgument(2 + > Iterables.size(Iterables.filter(allSchemaArgs, Predicates.notNull())), "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may " + "be set"); @@ -1012,18 +1008,19 @@ private WriteResult expandTyped( getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + " PCollection."); - return rowsWithDestination.apply( - new StreamingInserts<>(getCreateDisposition(), dynamicDestinations) - .setTestServices(getBigQueryServices())); + StreamingInserts streamingInserts = + new StreamingInserts<>(getCreateDisposition(), dynamicDestinations); + streamingInserts.setTestServices(getBigQueryServices()); + return rowsWithDestination.apply(streamingInserts); } else { - return rowsWithDestination.apply( - new BatchLoads<>( - getWriteDisposition(), - getCreateDisposition(), - getJsonTableRef() != null, - dynamicDestinations, - destinationCoder) - .withTestServices(getBigQueryServices())); + BatchLoads batchLoads = new BatchLoads<>( + getWriteDisposition(), + getCreateDisposition(), + getJsonTableRef() != null, + dynamicDestinations, + destinationCoder); + batchLoads.setTestServices(getBigQueryServices()); + return rowsWithDestination.apply(batchLoads); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java index c55f91e4b6038..de1ee599ca3ee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java @@ -52,9 +52,8 @@ public CalculateSchemas(DynamicDestinations dynamicDestinations public PCollectionView> expand( PCollection> input) { List> sideInputs = Lists.newArrayList(); - if (dynamicDestinations.getSideInput() != null) { - sideInputs.add(dynamicDestinations.getSideInput()); - } + sideInputs.addAll(dynamicDestinations.getSideInputs()); + return input .apply("Keys", Keys.create()) .apply("Distinct Keys", Distinct.create()) @@ -64,11 +63,8 @@ public PCollectionView> expand( new DoFn>() { @ProcessElement public void processElement(ProcessContext c) throws Exception { - // If the DynamicDestinations class wants to read a side input, give it the - // value. DynamicDestinations.SideInputAccessor sideInputAccessor = - new DynamicDestinations.SideInputAccessor( - c, dynamicDestinations.getSideInput()); + new DynamicDestinations.SideInputAccessor(c); TableSchema tableSchema = dynamicDestinations.getSchema( c.element(), sideInputAccessor); if (tableSchema != null) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 77cb9d9f24cb5..24f2e86d831ca 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -81,19 +81,16 @@ CreateTables withTestServices(BigQueryServices bqServices) { public PCollection> expand( PCollection> input) { List> sideInputs = Lists.newArrayList(); - if (dynamicDestinations.getSideInput() != null) { - sideInputs.add(dynamicDestinations.getSideInput()); - } + sideInputs.addAll(dynamicDestinations.getSideInputs()); + return input.apply( ParDo.of( new DoFn, KV>() { @ProcessElement public void processElement(ProcessContext context) throws InterruptedException, IOException { - // If a side input is needed to produce the table name, set it. DynamicDestinations.SideInputAccessor sideInputAccessor = - new DynamicDestinations.SideInputAccessor( - context, dynamicDestinations.getSideInput()); + new DynamicDestinations.SideInputAccessor(context); TableDestination tableDestination = dynamicDestinations.getTable(context.element().getKey(), sideInputAccessor); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java index c2a570405ec9f..c6d72024349e9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java @@ -19,16 +19,17 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.Lists; import java.io.Serializable; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; +import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; /** @@ -61,33 +62,28 @@ * }

    * *

    An instance of {@link DynamicDestinations} can also request a side input value that can be - * examined from inside {@link #getTable} and {@link #getSchema}. The side input is requested by - * calling {@link #setSideInputRequired} on the base class. The value can be examined via the - * {@link DynamicDestinations.SideInputAccessor} class. + * examined from inside {@link #getTable} and {@link #getSchema}. It must return a list of the + * side inputs it needs access to in {@link #getSideInputs()}. The value can be examined via the + * {@link DynamicDestinations.SideInputAccessor} parameter passed into {@link #getTable} + * and {@link #getSchema}. * *

    {@code DestinationT} is expected to provide proper hash and equality members. Ideally it will * be a compact type with an efficient coder, as these objects may be used as a key in a * {@link org.apache.beam.sdk.transforms.GroupByKey}. */ public abstract class DynamicDestinations implements Serializable { - private PCollectionView sideInput; - /** * Returns the materialized value of the side input. Can be used by concrete * {@link DynamicDestinations} instances in {@link #getSchema} or {@link #getTable}. */ public static class SideInputAccessor { private ProcessContext processContext; - private PCollectionView sideInput; - SideInputAccessor(ProcessContext processContext, PCollectionView sideInput) { + SideInputAccessor(ProcessContext processContext) { this.processContext = processContext; - this.sideInput = sideInput; } - public SideInputT getSideInputValue() throws IllegalStateException { - if (sideInput == null) { - return null; - } + public SideInputT getSideInputValue(PCollectionView sideInput) + throws IllegalStateException { @SuppressWarnings("unchecked") SideInputT materialized = (SideInputT) processContext.sideInput(sideInput); return materialized; @@ -95,16 +91,16 @@ public SideInputT getSideInputValue() throws IllegalStateException } /** - * Specifies that this object needs access to a side input. This side input must be globally - * windowed, as it will be accessed from the global window. + * Specifies that this object needs access to one or more side inputs. This side input must be + * globally windowed, as it will be accessed from the global window. */ - public DynamicDestinations setSideInputRequired(PCollectionView sideInput) { - this.sideInput = sideInput; - return this; + public List> getSideInputs() { + return Lists.newArrayList(); } /** - * Returns an object that represents at a high level which table is being written to. + * Returns an object that represents at a high level which table is being written to. May not + * return null. */ public abstract DestinationT getDestination(ValueInSingleWindow element); @@ -120,25 +116,17 @@ public Coder getDestinationCoder() { } /** - * Returns a {@link TableDestination} object for the destination. + * Returns a {@link TableDestination} object for the destination. May not return null. */ public abstract TableDestination getTable(DestinationT destination, SideInputAccessor sideInputAccessor); /** - * Returns the table schema for the destination. + * Returns the table schema for the destination. May not return null. */ public abstract TableSchema getSchema(DestinationT destination, SideInputAccessor sideInputAccessor); - /** - * This returns the unmaterialized side input used by this transform. - */ - PCollectionView getSideInput() { - @SuppressWarnings("unchecked") - PCollectionView sideInputTyped = (PCollectionView) sideInput; - return sideInputTyped; - } // Gets the destination coder. If the user does not provide one, try to find one in the coder // registry. If no coder can be found, throws CannotProvideCoderException. @@ -157,6 +145,10 @@ Coder getDestinationCoderWithDefault(CoderRegistry registry) @SuppressWarnings("unchecked") Class parameterClass = (Class) parameter; destinationCoder = registry.getDefaultCoder(parameterClass); + if (destinationCoder == null) { + throw new CannotProvideCoderException("Could not find a coder for destination type " + + parameterClass); + } break; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 1ca729653213c..9359a3c7d38c3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -19,6 +19,8 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.ImmutableList; +import java.util.List; import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; @@ -151,8 +153,8 @@ public Coder getDestinationCoder() { */ static class ConstantSchemaDestinations extends DelegatingDynamicDestinations { - private final @Nullable - ValueProvider jsonSchema; + @Nullable + private final ValueProvider jsonSchema; ConstantSchemaDestinations(DynamicDestinations inner, ValueProvider jsonSchema) { @@ -173,16 +175,23 @@ public TableSchema getSchema(TableDestination destination, */ static class SchemaFromViewDestinations extends DelegatingDynamicDestinations { + PCollectionView> schemaView; SchemaFromViewDestinations(DynamicDestinations inner, PCollectionView> schemaView) { super(inner); - setSideInputRequired(schemaView); + this.schemaView = schemaView; + } + + + @Override + public List> getSideInputs() { + return ImmutableList.>builder().add(schemaView).build(); } @Override public TableSchema getSchema(TableDestination destination, SideInputAccessor sideInputAccessor) { - Map mapValue = sideInputAccessor.getSideInputValue(); + Map mapValue = sideInputAccessor.getSideInputValue(schemaView); return BigQueryHelpers.fromJsonString(mapValue.get(destination.getTableSpec()), TableSchema.class); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index bd87456b57374..9cb00278e4d88 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -44,9 +44,8 @@ public class StreamingInserts this.bigQueryServices = new BigQueryServicesImpl(); } - StreamingInserts setTestServices(BigQueryServices bigQueryServices) { + void setTestServices(BigQueryServices bigQueryServices) { this.bigQueryServices = bigQueryServices; - return this; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index 7d7e7c700181a..dfa9d3c2dd2f9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -99,10 +99,8 @@ public WriteTables( @ProcessElement public void processElement(ProcessContext c) throws Exception { - // If the DynamicDestinations class wants to read a side input, give it the value. DynamicDestinations.SideInputAccessor sideInputAccessor = - new DynamicDestinations.SideInputAccessor( - c, dynamicDestinations.getSideInput()); + new DynamicDestinations.SideInputAccessor(c); DestinationT destination = c.element().getKey().getKey(); TableSchema tableSchema = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 02c9d0523f6d0..aeed51a2cdca4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -1845,19 +1845,20 @@ public void testWriteTables() throws Exception { .apply("CreateJobId", Create.of("jobId")) .apply(View.asSingleton()); - PCollectionView> schemaMapView = p.apply("CreateEmptySchema", Create - .empty( - new TypeDescriptor>() {})) - .apply(View.asMap()); - WriteTables writeTables = new WriteTables<>( - false, - fakeBqServices, - jobIdTokenView, - schemaMapView, - tempFilePrefix, - WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - new IdentityDynamicTables()); + PCollectionView> schemaMapView = + p.apply("CreateEmptySchema", + Create.empty(new TypeDescriptor>() {})) + .apply(View.asMap()); + WriteTables writeTables = + new WriteTables<>( + false, + fakeBqServices, + jobIdTokenView, + schemaMapView, + tempFilePrefix, + WriteDisposition.WRITE_EMPTY, + CreateDisposition.CREATE_IF_NEEDED, + new IdentityDynamicTables()); DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); @@ -1868,7 +1869,7 @@ public void testWriteTables() throws Exception { } Map> tempTablesResult = Maps.newHashMap(); - for (KV element : tester.takeOutputElements()) { + for (KV element : tester.takeOutputElements()) { List tables = tempTablesResult.get(element.getKey()); if (tables == null) { tables = Lists.newArrayList(); From ac6b4d2b22a666d9cbc562c2e99554b28423b24a Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 2 May 2017 20:59:43 -0700 Subject: [PATCH 6/6] Add side input value to test --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 42 +++++++++++++++++-- 2 files changed, 40 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 51807d7718f04..1085d4a1f2a90 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -90,7 +90,7 @@ * *

    A fully-qualified BigQuery table name consists of three components: *

      - * < li>{@code projectId}: the Cloud project id (defaults to + *
    • {@code projectId}: the Cloud project id (defaults to * {@link GcpOptions#getProject()}). *
    • {@code datasetId}: the BigQuery dataset id, unique within a project. *
    • {@code tableId}: a table id, unique within a dataset. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index aeed51a2cdca4..aa20bcb4a91e0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -22,8 +22,10 @@ import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.toJsonString; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -479,12 +481,29 @@ public void writeDynamicDestinations(boolean streaming) throws Exception { final Pattern userPattern = Pattern.compile("([a-z]+)([0-9]+)"); Pipeline p = TestPipeline.create(bqOptions); - PCollection users = p.apply(Create.of("bill1", "sam2", "laurence3") - .withCoder(StringUtf8Coder.of())); + + final PCollectionView> sideInput1 = + p.apply("Create SideInput 1", Create.of("a", "b", "c").withCoder(StringUtf8Coder.of())) + .apply("asList", View.asList()); + final PCollectionView> sideInput2 = + p.apply("Create SideInput2", Create.of(KV.of("a", "a"), KV.of("b", "b"), KV.of("c", "c"))) + .apply("AsMap", View.asMap()); + + PCollection users = p.apply("CreateUsers", + Create.of("bill1", "sam2", "laurence3") + .withCoder(StringUtf8Coder.of())) + .apply(Window.into(new PartitionedGlobalWindows<>( + new SerializableFunction() { + @Override + public String apply(String arg) { + return arg; + } + }))); + if (streaming) { users = users.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } - users.apply(BigQueryIO.write() + users.apply("WriteBigQuery", BigQueryIO.write() .withTestServices(fakeBqServices) .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) .withFormatFunction(new SerializableFunction() { @@ -501,6 +520,7 @@ public TableRow apply(String user) { .to(new StringIntegerDestinations() { @Override public Integer getDestination(ValueInSingleWindow element) { + assertThat(element.getWindow(), Matchers.instanceOf(PartitionedGlobalWindow.class)); Matcher matcher = userPattern.matcher(element.getValue()); if (matcher.matches()) { // Since we name tables by userid, we can simply store an Integer to represent @@ -512,6 +532,7 @@ public Integer getDestination(ValueInSingleWindow element) { @Override public TableDestination getTable(Integer userId, SideInputAccessor sideInput) { + verifySideInputs(sideInput); // Each user in it's own table. return new TableDestination("dataset-id.userid-" + userId, "table for userid " + userId); @@ -519,11 +540,26 @@ public TableDestination getTable(Integer userId, SideInputAccessor sideInput) { @Override public TableSchema getSchema(Integer userId, SideInputAccessor sideInput) { + verifySideInputs(sideInput); return new TableSchema().setFields( ImmutableList.of( new TableFieldSchema().setName("name").setType("STRING"), new TableFieldSchema().setName("id").setType("INTEGER"))); } + + @Override + public List> getSideInputs() { + return ImmutableList.of(sideInput1, sideInput2); + } + + void verifySideInputs(SideInputAccessor sideInputAccessor) { + assertThat(sideInputAccessor.getSideInputValue(sideInput1), + containsInAnyOrder("a", "b", "c")); + Map mapSideInput = sideInputAccessor.getSideInputValue(sideInput2); + assertEquals(3, mapSideInput.size()); + assertThat(mapSideInput, + allOf(hasEntry("a", "a"), hasEntry("b", "b"), hasEntry("c", "c"))); + } }) .withoutValidation()); p.run();