From f45af14d7a3d6841bc39edb0c6bd8515b87344d4 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 28 Mar 2017 11:21:59 -0700 Subject: [PATCH 1/7] Use tableRefFunction throughout BigQueryIO. Constant table writes use ConstantTableSpecFunction. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 57 ++++++++++--------- .../sdk/io/gcp/bigquery/StreamWithDeDup.java | 4 +- .../bigquery/TagWithUniqueIdsAndTable.java | 57 +++++-------------- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 19 +------ 4 files changed, 50 insertions(+), 87 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 9753da5cc9ac..af0d561eadb8 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 @@ -700,7 +700,8 @@ abstract static class Builder { abstract Builder setJsonTableRef(ValueProvider jsonTableRef); abstract Builder setTableRefFunction( SerializableFunction, TableReference> tableRefFunction); - abstract Builder setFormatFunction(SerializableFunction formatFunction); + abstract Builder setFormatFunction( + SerializableFunction formatFunction); abstract Builder setJsonSchema(ValueProvider jsonSchema); abstract Builder setCreateDisposition(CreateDisposition createDisposition); abstract Builder setWriteDisposition(WriteDisposition writeDisposition); @@ -781,7 +782,8 @@ public enum WriteDisposition { /** Ensures that methods of the to() family are called at most once. */ private void ensureToNotCalledYet() { checkState( - getJsonTableRef() == null && getTable() == null, "to() already called"); + getJsonTableRef() == null && getTable() == null + && getTableRefFunction() == null, "to() already called"); } /** @@ -805,6 +807,8 @@ public Write to(ValueProvider tableSpec) { NestedValueProvider.of( NestedValueProvider.of(tableSpec, new TableSpecToTableRef()), new TableRefToJson())) + .setTableRefFunction(new TranslateTableSpecFunction( + new ConstantTableSpecFunction(tableSpec))) .build(); } @@ -812,7 +816,8 @@ 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. */ - public Write to(SerializableFunction, String> tableSpecFunction) { + public Write to( + SerializableFunction, String> tableSpecFunction) { return toTableReference(new TranslateTableSpecFunction(tableSpecFunction)); } @@ -848,6 +853,20 @@ public TableReference apply(ValueInSingleWindow value) { } } + static class ConstantTableSpecFunction implements + SerializableFunction, String> { + private ValueProvider tableSpec; + + ConstantTableSpecFunction(ValueProvider tableSpec) { + this.tableSpec = tableSpec; + } + + @Override + public String apply(ValueInSingleWindow value) { + return tableSpec.get(); + } + } + /** * Uses the specified schema for rows to be written. * @@ -900,13 +919,8 @@ public void validate(PCollection input) { BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class); // Exactly one of the table and table reference can be configured. - checkState( - getJsonTableRef() != null || getTableRefFunction() != null, + checkState(getTableRefFunction() != null, "must set the table reference of a BigQueryIO.Write transform"); - checkState( - getJsonTableRef() == null || getTableRefFunction() == null, - "Cannot set both a table reference and a table function for a BigQueryIO.Write" - + " transform"); checkArgument(getFormatFunction() != null, "A function must be provided to convert type into a TableRow. " @@ -920,6 +934,7 @@ public void validate(PCollection input) { // The user specified a table. if (getJsonTableRef() != null && 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. @@ -935,24 +950,12 @@ public void validate(PCollection input) { } } - if (input.isBounded() == PCollection.IsBounded.UNBOUNDED || getTableRefFunction() != null) { + if (input.isBounded() == PCollection.IsBounded.UNBOUNDED) { // We will use BigQuery's streaming write API -- validate supported dispositions. - if (getTableRefFunction() != null) { - checkArgument( - getCreateDisposition() != CreateDisposition.CREATE_NEVER, - "CreateDisposition.CREATE_NEVER is not supported when using a tablespec" - + " function."); - } - if (getJsonSchema() == null) { - checkArgument( - getCreateDisposition() == CreateDisposition.CREATE_NEVER, - "CreateDisposition.CREATE_NEVER must be used if jsonSchema is null."); - } - checkArgument( getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, - "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded PCollection or" - + " when using a tablespec function."); + "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + + " PCollection."); } else { // We will use a BigQuery load job -- validate the temp location. String tempLocation = options.getTempLocation(); @@ -977,7 +980,7 @@ public void validate(PCollection input) { public WriteResult expand(PCollection input) { // When writing an Unbounded PCollection, or when a tablespec function is defined, we use // StreamWithDeDup and BigQuery's streaming import API. - if (input.isBounded() == IsBounded.UNBOUNDED || getTableRefFunction() != null) { + if (input.isBounded() == IsBounded.UNBOUNDED) { return input.apply(new StreamWithDeDup(this)); } else { return input.apply(new BatchLoadBigQuery(this)); @@ -1026,12 +1029,12 @@ public TableSchema getSchema() { * *

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); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java index 1fa26d1542d3..506a564fdbad 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java @@ -64,8 +64,7 @@ public WriteResult expand(PCollection input) { PCollection, TableRowInfo>> tagged = input.apply(ParDo.of(new TagWithUniqueIdsAndTable( - input.getPipeline().getOptions().as(BigQueryOptions.class), write.getTable(), - write.getTableRefFunction(), write.getFormatFunction()))); + input.getPipeline().getOptions().as(BigQueryOptions.class), write))); // To prevent having the same TableRow processed more than once with regenerated // different unique ids, this implementation relies on "checkpointing", which is @@ -85,6 +84,7 @@ public WriteResult expand(PCollection input) { write.getCreateDisposition(), write.getTableDescription(), write.getBigQueryServices()))); + return WriteResult.in(input.getPipeline()); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java index a6608e4a68f0..8d7d1e65ac54 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java @@ -18,23 +18,18 @@ package org.apache.beam.sdk.io.gcp.bigquery; -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.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import java.io.IOException; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableRef; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToTableSpec; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; -import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; 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.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; @@ -49,39 +44,22 @@ @VisibleForTesting class TagWithUniqueIdsAndTable extends DoFn, TableRowInfo>> { - /** TableSpec to write to. */ - private final ValueProvider tableSpec; - - /** User function mapping windowed values to {@link TableReference} in JSON. */ - private final SerializableFunction, TableReference> tableRefFunction; + /** TableSpec to write to in the case of a single static destination. */ + private ValueProvider tableSpec = null; - /** User function mapping user type to a TableRow. */ - private final SerializableFunction formatFunction; + private final Write write; private transient String randomUUID; private transient long sequenceNo = 0L; TagWithUniqueIdsAndTable(BigQueryOptions options, - ValueProvider table, - SerializableFunction, TableReference> - tableRefFunction, - SerializableFunction formatFunction) { - checkArgument(table == null ^ tableRefFunction == null, - "Exactly one of table or tableRefFunction should be set"); + Write write) { + ValueProvider table = write.getTableWithDefaultProject( + options.as(BigQueryOptions.class)); if (table != null) { - if (table.isAccessible() && Strings.isNullOrEmpty(table.get().getProjectId())) { - TableReference tableRef = table.get() - .setProjectId(options.as(BigQueryOptions.class).getProject()); - table = NestedValueProvider.of( - StaticValueProvider.of(BigQueryHelpers.toJsonString(tableRef)), - new JsonTableRefToTableRef()); - } this.tableSpec = NestedValueProvider.of(table, new TableRefToTableSpec()); - } else { - tableSpec = null; } - this.tableRefFunction = tableRefFunction; - this.formatFunction = formatFunction; + this.write = write; } @@ -101,7 +79,7 @@ public void processElement(ProcessContext context, BoundedWindow window) throws // We output on keys 0-50 to ensure that there's enough batching for // BigQuery. context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, 50)), - new TableRowInfo(formatFunction.apply(context.element()), uniqueId))); + new TableRowInfo(write.getFormatFunction().apply(context.element()), uniqueId))); } @Override @@ -109,10 +87,8 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder.addIfNotNull(DisplayData.item("table", tableSpec)); - if (tableRefFunction != null) { - builder.add(DisplayData.item("tableFn", tableRefFunction.getClass()) + builder.add(DisplayData.item("tableFn", write.getTableRefFunction().getClass()) .withLabel("Table Reference Function")); - } } @VisibleForTesting @@ -120,16 +96,13 @@ ValueProvider getTableSpec() { return tableSpec; } + private String tableSpecFromWindowedValue(BigQueryOptions options, ValueInSingleWindow value) { - if (tableSpec != null) { - return tableSpec.get(); - } else { - TableReference table = tableRefFunction.apply(value); - if (table.getProjectId() == null) { - table.setProjectId(options.getProject()); - } - return BigQueryHelpers.toTableSpec(table); + TableReference table = write.getTableRefFunction().apply(value); + if (Strings.isNullOrEmpty(table.getProjectId())) { + table.setProjectId(options.getProject()); } + return BigQueryHelpers.toTableSpec(table); } } 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 83fd8d992b2c..499aa74f1a5f 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 @@ -26,7 +26,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -103,7 +102,6 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef; 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.BigQueryServices.DatasetService; @@ -150,6 +148,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; @@ -1375,7 +1374,8 @@ public void testBuildWriteWithoutValidation() { @Test public void testBuildWriteDefaultProject() { - BigQueryIO.Write write = BigQueryIO.writeTableRows().to("somedataset.sometable"); + BigQueryIO.Write write = BigQueryIO.writeTableRows() + .to("somedataset" + ".sometable"); checkWriteObject( write, null, "somedataset", "sometable", null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, @@ -2350,19 +2350,6 @@ public void testRuntimeOptionsNotCalledInApplyOutput() { DisplayData.from(write); } - @Test - public void testTagWithUniqueIdsAndTableProjectNotNullWithNvp() { - BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class); - bqOptions.setProject("project"); - TagWithUniqueIdsAndTable tag = - new TagWithUniqueIdsAndTable( - bqOptions, NestedValueProvider.of( - StaticValueProvider.of("data_set.table_name"), - new TableSpecToTableRef()), null, null); - TableReference table = BigQueryHelpers.parseTableSpec(tag.getTableSpec().get()); - assertNotNull(table.getProjectId()); - } - private static void testNumFiles(File tempDir, int expectedNumFiles) { assertEquals(expectedNumFiles, tempDir.listFiles(new FileFilter() { @Override From 3c8a68a01e6ca9bf97ceb6c6c9d3572dcdaa9b58 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 28 Mar 2017 12:53:27 -0700 Subject: [PATCH 2/7] Add PrepareWrite transform. --- .../sdk/io/gcp/bigquery/PrepareWrite.java | 58 +++++++++++++++++++ .../sdk/io/gcp/bigquery/TableDestination.java | 7 +++ .../bigquery/TagWithUniqueIdsAndTable.java | 15 ++--- 3 files changed, 69 insertions(+), 11 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java 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 new file mode 100644 index 000000000000..0c08e183c5ee --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java @@ -0,0 +1,58 @@ +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.options.BigQueryOptions; +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.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.ValueInSingleWindow; + +/** + * Prepare an input {@link PCollection} for writing to BigQuery. Use the table-reference + * function to determine 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 static final String NAME = "PrepareWrite"; + private SerializableFunction, TableReference> tableRefFunction; + private SerializableFunction formatFunction; + + public PrepareWrite(SerializableFunction, TableReference> tableRefFunction, + SerializableFunction formatFunction) { + super(NAME); + this.tableRefFunction = tableRefFunction; + this.formatFunction = formatFunction; + } + + @Override + public PCollection> expand(PCollection input) { + PCollection> elementsByTable = + input.apply(ParDo.of(new DoFn>() { + @ProcessElement + public void processElement(ProcessContext context, BoundedWindow window) throws IOException { + String tableSpec = tableSpecFromWindowedValue( + context.getPipelineOptions().as(BigQueryOptions.class), + ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); + TableRow tableRow = formatFunction.apply(context.element()); + context.output(KV.of(tableSpec, tableRow)); + } + })); + return elementsByTable; + } + + private String tableSpecFromWindowedValue(BigQueryOptions options, + ValueInSingleWindow value) { + TableReference table = tableRefFunction.apply(value); + if (Strings.isNullOrEmpty(table.getProjectId())) { + table.setProjectId(options.getProject()); + } + return BigQueryHelpers.toTableSpec(table); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java new file mode 100644 index 000000000000..3cbbf3bb135f --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -0,0 +1,7 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +/** + * Created by relax on 3/28/17. + */ +public class TableDestination { +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java index 8d7d1e65ac54..4e50f7c47dce 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java @@ -73,9 +73,9 @@ public void startBundle(Context context) { public void processElement(ProcessContext context, BoundedWindow window) throws IOException { String uniqueId = randomUUID + sequenceNo++; ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); - String tableSpec = tableSpecFromWindowedValue( - context.getPipelineOptions().as(BigQueryOptions.class), - ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); + String tableSpec = tableSpecFromWindowedValue( + context.getPipelineOptions().as(BigQueryOptions.class), + ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); // We output on keys 0-50 to ensure that there's enough batching for // BigQuery. context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, 50)), @@ -97,12 +97,5 @@ ValueProvider getTableSpec() { } - private String tableSpecFromWindowedValue(BigQueryOptions options, - ValueInSingleWindow value) { - TableReference table = write.getTableRefFunction().apply(value); - if (Strings.isNullOrEmpty(table.getProjectId())) { - table.setProjectId(options.getProject()); - } - return BigQueryHelpers.toTableSpec(table); - } + } From ee855155aa5b4c52a5046fa60861c405be315b0b Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 28 Mar 2017 19:34:56 -0700 Subject: [PATCH 3/7] Refactor streaming write branch into separate reusable components. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 69 ++++++------ .../sdk/io/gcp/bigquery/CreateTables.java | 100 ++++++++++++++++++ .../io/gcp/bigquery/GenerateShardedTable.java | 48 +++++++++ .../sdk/io/gcp/bigquery/PrepareWrite.java | 65 +++++++----- ...amWithDeDup.java => StreamingInserts.java} | 62 +++++++---- .../sdk/io/gcp/bigquery/StreamingWriteFn.java | 82 +------------- .../sdk/io/gcp/bigquery/TableDestination.java | 48 ++++++++- .../gcp/bigquery/TableDestinationCoder.java | 64 +++++++++++ ...IdsAndTable.java => TagWithUniqueIds.java} | 40 +------ .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 18 +--- 10 files changed, 386 insertions(+), 210 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{StreamWithDeDup.java => StreamingInserts.java} (57%) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{TagWithUniqueIdsAndTable.java => TagWithUniqueIds.java} (65%) 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 af0d561eadb8..af19b83eec9d 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 @@ -40,6 +40,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.BoundedSource; @@ -60,6 +61,7 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.PipelineRunner; 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.display.DisplayData; import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; @@ -67,6 +69,7 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -681,8 +684,8 @@ 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, TableReference> - getTableRefFunction(); + @Nullable abstract SerializableFunction, TableDestination> + getTableFunction(); @Nullable abstract SerializableFunction getFormatFunction(); /** Table schema. The schema is required only if the table does not exist. */ @Nullable abstract ValueProvider getJsonSchema(); @@ -783,7 +786,7 @@ public enum WriteDisposition { private void ensureToNotCalledYet() { checkState( getJsonTableRef() == null && getTable() == null - && getTableRefFunction() == null, "to() already called"); + && getTableFunction() == null, "to() already called"); } /** @@ -802,13 +805,16 @@ public Write to(TableReference table) { /** Same as {@link #to(String)}, but with a {@link ValueProvider}. */ public Write to(ValueProvider tableSpec) { ensureToNotCalledYet(); + String tableDescription = getTableDescription(); + if (tableDescription == null) { + tableDescription = ""; + } return toBuilder() .setJsonTableRef( NestedValueProvider.of( NestedValueProvider.of(tableSpec, new TableSpecToTableRef()), new TableRefToJson())) - .setTableRefFunction(new TranslateTableSpecFunction( - new ConstantTableSpecFunction(tableSpec))) + .setTableFunction(new ConstantTableFunction(tableSpec, tableDescription)) .build(); } @@ -819,6 +825,8 @@ public Write to(ValueProvider tableSpec) { public Write to( SerializableFunction, String> tableSpecFunction) { return toTableReference(new TranslateTableSpecFunction(tableSpecFunction)); + ensureToNotCalledYet(); + return toBuilder().setTableFunction(tableFunction).build(); } /** @@ -828,7 +836,7 @@ public Write to( private Write toTableReference( SerializableFunction, TableReference> tableRefFunction) { ensureToNotCalledYet(); - return toBuilder().setTableRefFunction(tableRefFunction).build(); + return toBuilder().setTableFunction(tableFunction).build(); } /** @@ -838,32 +846,19 @@ public Write withFormatFunction(SerializableFunction formatFunct return toBuilder().setFormatFunction(formatFunction).build(); } - private static class TranslateTableSpecFunction implements - SerializableFunction, TableReference> { - private SerializableFunction, String> tableSpecFunction; - - TranslateTableSpecFunction( - SerializableFunction, String> tableSpecFunction) { - this.tableSpecFunction = tableSpecFunction; - } + static class ConstantTableFunction implements + SerializableFunction, TableDestination> { + private final ValueProvider tableSpec; + private final String tableDescription; - @Override - public TableReference apply(ValueInSingleWindow value) { - return BigQueryHelpers.parseTableSpec(tableSpecFunction.apply(value)); - } - } - - static class ConstantTableSpecFunction implements - SerializableFunction, String> { - private ValueProvider tableSpec; - - ConstantTableSpecFunction(ValueProvider tableSpec) { + ConstantTableFunction(ValueProvider tableSpec, String tableDescription) { this.tableSpec = tableSpec; + this.tableDescription = tableDescription; } @Override - public String apply(ValueInSingleWindow value) { - return tableSpec.get(); + public TableDestination apply(ValueInSingleWindow value) { + return new TableDestination(tableSpec.get(), tableDescription); } } @@ -919,7 +914,7 @@ public void validate(PCollection input) { BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class); // Exactly one of the table and table reference can be configured. - checkState(getTableRefFunction() != null, + checkState(getTableFunction() != null, "must set the table reference of a BigQueryIO.Write transform"); checkArgument(getFormatFunction() != null, @@ -978,10 +973,16 @@ public void validate(PCollection input) { @Override public WriteResult expand(PCollection input) { + PCollection> rowsWithDestination = + input.apply("PrepareWrite", ParDo.of( + new PrepareWrite(getTableFunction(), getFormatFunction()))) + .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); + + // When writing an Unbounded PCollection, or when a tablespec function is defined, we use - // StreamWithDeDup and BigQuery's streaming import API. + // StreamingInserts and BigQuery's streaming import API. if (input.isBounded() == IsBounded.UNBOUNDED) { - return input.apply(new StreamWithDeDup(this)); + return rowsWithDestination.apply(new StreamingInserts(this)); } else { return input.apply(new BatchLoadBigQuery(this)); } @@ -1002,8 +1003,8 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("schema", getJsonSchema()) .withLabel("Table Schema")); - if (getTableRefFunction() != null) { - builder.add(DisplayData.item("tableFn", getTableRefFunction().getClass()) + if (getTableFunction() != null) { + builder.add(DisplayData.item("tableFn", getTableFunction().getClass()) .withLabel("Table Reference Function")); } @@ -1025,7 +1026,7 @@ public TableSchema getSchema() { } /** - * Returns the table to write, or {@code null} if writing with {@code tableRefFunction}. + * Returns the table to write, or {@code null} if writing with {@code tableFunction}. * *

If the table's project is not specified, use the executing project. */ @@ -1066,7 +1067,7 @@ public ValueProvider getTable() { */ @VisibleForTesting static void clearCreatedTables() { - StreamingWriteFn.clearCreatedTables(); + CreateTables.clearCreatedTables(); } ///////////////////////////////////////////////////////////////////////////// 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 new file mode 100644 index 000000000000..e216553d3fae --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -0,0 +1,100 @@ +/* +* 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.Table; +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 java.io.IOException; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; + + +/** + * Creates any tables needed before performing streaming writes to the tables. This is a + * side-effect {l@ink DoFn}, and returns the original collection unchanged. + */ +public class CreateTables extends DoFn, + KV> { + private final CreateDisposition createDisposition; + private final BigQueryServices bqServices; + private final SerializableFunction schemaFunction; + + + /** The list of tables created so far, so we don't try the creation + each time. + * TODO: We should put a bound on memory usage of this. Use guava cache instead. + */ + private static Set createdTables = + Collections.newSetFromMap(new ConcurrentHashMap()); + + public CreateTables(CreateDisposition createDisposition, BigQueryServices bqServices, + SerializableFunction schemaFunction) { + this.createDisposition = createDisposition; + this.bqServices = bqServices; + this.schemaFunction = schemaFunction; + } + + @ProcessElement + public void processElement(ProcessContext context) throws InterruptedException, IOException { + BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); + possibleCreateTable(options, context.element().getKey()); + context.output(context.element()); + } + + private void possibleCreateTable(BigQueryOptions options, TableDestination tableDestination) + throws InterruptedException, IOException { + String tableSpec = tableDestination.getTableSpec(); + TableReference tableReference = tableDestination.getTableReference(); + String tableDescription = tableDestination.getTableDescription(); + if (createDisposition != createDisposition.CREATE_NEVER + && !createdTables.contains(tableSpec)) { + synchronized (createdTables) { + // Another thread may have succeeded in creating the table in the meanwhile, so + // check again. This check isn't needed for correctness, but we add it to prevent + // 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() + .setTableReference(tableReference) + .setSchema(tableSchema) + .setDescription(tableDescription)); + } + createdTables.add(tableSpec); + } + } + } + } + + static void clearCreatedTables() { + synchronized (createdTables) { + createdTables.clear(); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java new file mode 100644 index 000000000000..da3a70a861be --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java @@ -0,0 +1,48 @@ +/* +* 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.TableRow; +import java.io.IOException; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; + +/** + * Given a write to a specific table, assign that to one of the + * {@link GenerateShardedTable#numShards} keys assigned to that table. + */ +class GenerateShardedTable extends DoFn, + KV, TableRow>> { + private final int numShards; + + GenerateShardedTable(int numShards) { + this.numShards = numShards; + } + + @ProcessElement + public void processElement(ProcessContext context, BoundedWindow window) throws IOException { + ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); + // We output on keys 0-50 to ensure that there's enough batching for + // BigQuery. + String tableSpec = context.element().getKey().getTableSpec(); + context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, numShards)), + context.element().getValue())); + } +} 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 0c08e183c5ee..771241793757 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 @@ -1,3 +1,20 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableReference; @@ -6,8 +23,6 @@ import java.io.IOException; import org.apache.beam.sdk.options.BigQueryOptions; 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.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; @@ -15,44 +30,38 @@ import org.apache.beam.sdk.values.ValueInSingleWindow; /** - * Prepare an input {@link PCollection} for writing to BigQuery. Use the table-reference + * Prepare an input {@link PCollection} for writing to BigQuery. Use the table-reference * function to determine 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 static final String NAME = "PrepareWrite"; - private SerializableFunction, TableReference> tableRefFunction; +public class PrepareWrite extends DoFn> { + private SerializableFunction, TableDestination> tableFunction; private SerializableFunction formatFunction; - public PrepareWrite(SerializableFunction, TableReference> tableRefFunction, + public PrepareWrite(SerializableFunction, TableDestination> tableFunction, SerializableFunction formatFunction) { - super(NAME); - this.tableRefFunction = tableRefFunction; + this.tableFunction = tableFunction; this.formatFunction = formatFunction; } - @Override - public PCollection> expand(PCollection input) { - PCollection> elementsByTable = - input.apply(ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext context, BoundedWindow window) throws IOException { - String tableSpec = tableSpecFromWindowedValue( - context.getPipelineOptions().as(BigQueryOptions.class), - ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); - TableRow tableRow = formatFunction.apply(context.element()); - context.output(KV.of(tableSpec, tableRow)); - } - })); - return elementsByTable; + @ProcessElement + public void processElement(ProcessContext context, BoundedWindow window) throws IOException { + TableDestination tableDestination = tableSpecFromWindowedValue( + context.getPipelineOptions().as(BigQueryOptions.class), + ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); + TableRow tableRow = formatFunction.apply(context.element()); + context.output(KV.of(tableDestination, tableRow)); } - private String tableSpecFromWindowedValue(BigQueryOptions options, + private TableDestination tableSpecFromWindowedValue(BigQueryOptions options, ValueInSingleWindow value) { - TableReference table = tableRefFunction.apply(value); - if (Strings.isNullOrEmpty(table.getProjectId())) { - table.setProjectId(options.getProject()); + 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 BigQueryHelpers.toTableSpec(table); + return tableDestination; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java similarity index 57% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 506a564fdbad..37afbdfeeb11 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamWithDeDup.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -18,17 +18,18 @@ 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.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; 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.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -37,11 +38,28 @@ * PTransform that performs streaming BigQuery write. To increase consistency, * it leverages BigQuery best effort de-dup mechanism. */ -class StreamWithDeDup extends PTransform, WriteResult> { - private final Write write; + +class StreamingInserts + extends PTransform>, WriteResult> { + private final Write write; + + 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. */ - StreamWithDeDup(Write write) { + StreamingInserts(Write write) { this.write = write; } @@ -51,7 +69,12 @@ protected Coder getDefaultOutputCoder() { } @Override - public WriteResult expand(PCollection input) { + 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()); + // A naive implementation would be to simply stream data directly to BigQuery. // However, this could occasionally lead to duplicated data, e.g., when // a VM that runs this code is restarted and the code is re-run. @@ -61,29 +84,26 @@ public WriteResult expand(PCollection input) { // To use this mechanism, each input TableRow is tagged with a generated // unique id, which is then passed to BigQuery and used to ignore duplicates. - - PCollection, TableRowInfo>> tagged = - input.apply(ParDo.of(new TagWithUniqueIdsAndTable( - input.getPipeline().getOptions().as(BigQueryOptions.class), write))); + PCollection, TableRowInfo>> tagged = input + .apply("CreateTables", ParDo.of(new CreateTables(write.getCreateDisposition(), + write.getBigQueryServices(), schemaFunction))) + // We create 50 keys per BigQuery table to generate output on. This is few enough that we + // get good batching into BigQuery's insert calls, and enough that we can max out the + // streaming insert quota. + .apply("ShardTableWrites", ParDo.of(new GenerateShardedTable(50))) + .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowJsonCoder.of())) + .apply("TagWithUniqueIds", ParDo.of(new TagWithUniqueIds())); // To prevent having the same TableRow processed more than once with regenerated // different unique ids, this implementation relies on "checkpointing", which is // achieved as a side effect of having StreamingWriteFn immediately follow a GBK, // performed by Reshuffle. - NestedValueProvider schema = - write.getJsonSchema() == null - ? null - : NestedValueProvider.of(write.getJsonSchema(), new JsonSchemaToTableSchema()); tagged .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of())) .apply(Reshuffle., TableRowInfo>of()) - .apply( + .apply("StreamingWrite", ParDo.of( - new StreamingWriteFn( - schema, - write.getCreateDisposition(), - write.getTableDescription(), - write.getBigQueryServices()))); + new StreamingWriteFn(write.getBigQueryServices()))); return WriteResult.in(input.getPipeline()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java index 1d93fa34114d..83ed3d2a9453 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java @@ -18,28 +18,16 @@ package org.apache.beam.sdk.io.gcp.bigquery; -import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.services.bigquery.model.Table; 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.annotations.VisibleForTesting; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.SystemDoFnInternal; @@ -52,48 +40,19 @@ @VisibleForTesting class StreamingWriteFn extends DoFn, TableRowInfo>, Void> { - /** TableSchema in JSON. Use String to make the class Serializable. */ - @Nullable - private final ValueProvider jsonTableSchema; - - @Nullable private final String tableDescription; - private final BigQueryServices bqServices; /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */ private transient Map> tableRows; - private final Write.CreateDisposition createDisposition; - /** The list of unique ids for each BigQuery table row. */ private transient Map> uniqueIdsForTableRows; - /** The list of tables created so far, so we don't try the creation - each time. */ - private static Set createdTables = - Collections.newSetFromMap(new ConcurrentHashMap()); - /** Tracks bytes written, exposed as "ByteCount" Counter. */ private Counter byteCounter = Metrics.counter(StreamingWriteFn.class, "ByteCount"); - /** Constructor. */ - StreamingWriteFn(@Nullable ValueProvider schema, - Write.CreateDisposition createDisposition, - @Nullable String tableDescription, BigQueryServices bqServices) { - this.jsonTableSchema = schema == null ? null : - NestedValueProvider.of(schema, new TableSchemaToJsonSchema()); - this.createDisposition = createDisposition; - this.bqServices = checkNotNull(bqServices, "bqServices"); - this.tableDescription = tableDescription; - } - - /** - * Clear the cached map of created tables. Used for testing. - */ - static void clearCreatedTables() { - synchronized (createdTables) { - createdTables.clear(); - } + StreamingWriteFn(BigQueryServices bqServices) { + this.bqServices = bqServices; } /** Prepares a target BigQuery table. */ @@ -119,9 +78,8 @@ public void processElement(ProcessContext context) { @FinishBundle public void finishBundle(Context context) throws Exception { BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - for (Map.Entry> entry : tableRows.entrySet()) { - TableReference tableReference = getOrCreateTable(options, entry.getKey()); + TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey()); flushRows(tableReference, entry.getValue(), uniqueIdsForTableRows.get(entry.getKey()), options); } @@ -132,39 +90,6 @@ public void finishBundle(Context context) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - builder - .addIfNotNull(DisplayData.item("schema", jsonTableSchema) - .withLabel("Table Schema")) - .addIfNotNull(DisplayData.item("tableDescription", tableDescription) - .withLabel("Table Description")); - } - - public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec) - throws InterruptedException, IOException { - TableReference tableReference = BigQueryHelpers.parseTableSpec(tableSpec); - if (createDisposition != createDisposition.CREATE_NEVER - && !createdTables.contains(tableSpec)) { - synchronized (createdTables) { - // Another thread may have succeeded in creating the table in the meanwhile, so - // check again. This check isn't needed for correctness, but we add it to prevent - // every thread from attempting a create and overwhelming our BigQuery quota. - DatasetService datasetService = bqServices.getDatasetService(options); - if (!createdTables.contains(tableSpec)) { - if (datasetService.getTable(tableReference) == null) { - TableSchema tableSchema = BigQueryIO.JSON_FACTORY.fromString( - jsonTableSchema.get(), TableSchema.class); - datasetService.createTable( - new Table() - .setTableReference(tableReference) - .setSchema(tableSchema) - .setDescription(tableDescription)); - } - createdTables.add(tableSpec); - } - } - } - return tableReference; } /** @@ -173,6 +98,7 @@ public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec private void flushRows(TableReference tableReference, List tableRows, List uniqueIds, BigQueryOptions options) throws InterruptedException { + System.out.println("FlUSHING ROWS " + tableRows.size()); if (!tableRows.isEmpty()) { try { long totalBytes = bqServices.getDatasetService(options).insertAll( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index 3cbbf3bb135f..631afeb99b87 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -1,7 +1,53 @@ +/* +* 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.TableReference; + /** - * Created by relax on 3/28/17. + * Encapsulates a BigQuery table destination. */ public class TableDestination { + private final String tableSpec; + private final String tableDescription; + + + public TableDestination(String tableSpec, String tableDescription) { + this.tableSpec = tableSpec; + this.tableDescription = tableDescription; + } + + public TableDestination(TableReference tableReference, String tableDescription) { + this.tableSpec = BigQueryHelpers.toTableSpec(tableReference); + this.tableDescription = tableDescription; + } + + public String getTableSpec() { + return tableSpec; + } + + + public TableReference getTableReference() { + return BigQueryHelpers.parseTableSpec(tableSpec); + } + + public String getTableDescription() { + return tableDescription; + } } 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 new file mode 100644 index 000000000000..fa24700472ab --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -0,0 +1,64 @@ +/* +* 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.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StringUtf8Coder; + +/** + * A coder for {@link TableDestination} objects. + */ +public class TableDestinationCoder extends AtomicCoder { + private static final TableDestinationCoder INSTANCE = new TableDestinationCoder(); + + + @JsonCreator + public static TableDestinationCoder of() { + return INSTANCE; + } + + @Override + public void encode(TableDestination value, OutputStream outStream, Context context) + throws IOException { + if (value == null) { + throw new CoderException("cannot encode a null value"); + } + stringCoder.encode(value.getTableSpec(), outStream, context.nested()); + stringCoder.encode(value.getTableDescription(), outStream, context); + } + + @Override + public TableDestination decode(InputStream inStream, Context context) throws IOException { + return new TableDestination( + stringCoder.decode(inStream, context.nested()), + stringCoder.decode(inStream, context.nested())); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + return; + } + + StringUtf8Coder stringCoder = StringUtf8Coder.of(); +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java similarity index 65% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java index 4e50f7c47dce..6f0186e73989 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIdsAndTable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java @@ -19,6 +19,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.annotations.VisibleForTesting; import com.google.common.base.Strings; import java.io.IOException; @@ -42,27 +43,12 @@ * id is created by concatenating this randomUUID with a sequential number. */ @VisibleForTesting -class TagWithUniqueIdsAndTable - extends DoFn, TableRowInfo>> { - /** TableSpec to write to in the case of a single static destination. */ - private ValueProvider tableSpec = null; - - private final Write write; +class TagWithUniqueIds + extends DoFn, TableRow>, KV, TableRowInfo>> { private transient String randomUUID; private transient long sequenceNo = 0L; - TagWithUniqueIdsAndTable(BigQueryOptions options, - Write write) { - ValueProvider table = write.getTableWithDefaultProject( - options.as(BigQueryOptions.class)); - if (table != null) { - this.tableSpec = NestedValueProvider.of(table, new TableRefToTableSpec()); - } - this.write = write; - } - - @StartBundle public void startBundle(Context context) { randomUUID = UUID.randomUUID().toString(); @@ -72,30 +58,14 @@ public void startBundle(Context context) { @ProcessElement public void processElement(ProcessContext context, BoundedWindow window) throws IOException { String uniqueId = randomUUID + sequenceNo++; - ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); - String tableSpec = tableSpecFromWindowedValue( - context.getPipelineOptions().as(BigQueryOptions.class), - ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); // We output on keys 0-50 to ensure that there's enough batching for // BigQuery. - context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, 50)), - new TableRowInfo(write.getFormatFunction().apply(context.element()), uniqueId))); + context.output(KV.of(context.element().getKey(), + new TableRowInfo(context.element().getValue(), uniqueId))); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - builder.addIfNotNull(DisplayData.item("table", tableSpec)); - builder.add(DisplayData.item("tableFn", write.getTableRefFunction().getClass()) - .withLabel("Table Reference Function")); } - - @VisibleForTesting - ValueProvider getTableSpec() { - return tableSpec; - } - - - } 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 499aa74f1a5f..d953edd1a9a9 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 @@ -518,7 +518,6 @@ List getRows() { /** A fake dataset service that can be serialized, for use in testReadFromTable. */ private static class FakeDatasetService implements DatasetService, Serializable { - @Override public Table getTable(TableReference tableRef) throws InterruptedException, IOException { @@ -630,6 +629,7 @@ public long insertAll( TableContainer tableContainer = getTableContainer( ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); for (int i = 0; i < rowList.size(); ++i) { + System.out.println("adding row " + rowList.get(i)); tableContainer.addRow(rowList.get(i), insertIdList.get(i)); dataSize += rowList.get(i).toString().length(); } @@ -1121,15 +1121,15 @@ public String apply(Integer i) { } ); - SerializableFunction, String> tableFunction = - new SerializableFunction, String>() { + SerializableFunction, TableDestination> tableFunction = + new SerializableFunction, TableDestination>() { @Override - public String apply(ValueInSingleWindow input) { + public TableDestination apply(ValueInSingleWindow input) { PartitionedGlobalWindow window = (PartitionedGlobalWindow) input.getWindow(); // Check that we can access the element as well here. checkArgument(window.value.equals(Integer.toString(input.getValue() % 5)), "Incorrect element"); - return "project-id:dataset-id.table-id-" + window.value; + return new TableDestination("project-id:dataset-id.table-id-" + window.value, ""); } }; @@ -1558,14 +1558,6 @@ public void testWriteValidatesDatasetStreaming() throws Exception { testWriteValidatesDataset(true); } - @Test - public void testStreamingWriteFnCreateNever() throws Exception { - StreamingWriteFn fn = new StreamingWriteFn( - null, CreateDisposition.CREATE_NEVER, null, new FakeBigQueryServices()); - assertEquals(BigQueryHelpers.parseTableSpec("dataset.table"), - fn.getOrCreateTable(null, "dataset.table")); - } - @Test public void testCreateNeverWithStreaming() throws Exception { BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); From 23f1cf10d6ad0551b3acfd8ff0b02564e4e78367 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Wed, 29 Mar 2017 07:34:10 -0700 Subject: [PATCH 4/7] Refactor batch load job path, and add support for data-dependent tables. --- ...BatchLoadBigQuery.java => BatchLoads.java} | 105 +++++++++++------- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 +- .../sdk/io/gcp/bigquery/TableDestination.java | 17 ++- .../sdk/io/gcp/bigquery/TableRowWriter.java | 12 +- ...eBundles.java => WriteBundlesToFiles.java} | 36 ++++-- .../sdk/io/gcp/bigquery/WritePartition.java | 95 +++++++++++----- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 63 ++++++----- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 47 ++++---- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 27 ++--- 9 files changed, 256 insertions(+), 149 deletions(-) rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{BatchLoadBigQuery.java => BatchLoads.java} (60%) rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{WriteBundles.java => WriteBundlesToFiles.java} (66%) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoadBigQuery.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java similarity index 60% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoadBigQuery.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 160b231b68f7..8594211ff0d9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoadBigQuery.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -20,20 +20,22 @@ 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 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.io.gcp.bigquery.BigQueryHelpers.TableRefToJson; 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.BigQueryOptions; import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; @@ -47,20 +49,37 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.sdk.values.TypeDescriptor; + /** * PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ -class BatchLoadBigQuery extends PTransform, WriteResult> { +class BatchLoads extends + PTransform>, WriteResult> { BigQueryIO.Write write; - BatchLoadBigQuery(BigQueryIO.Write write) { + 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); + } + } + + BatchLoads(BigQueryIO.Write write) { this.write = write; } @Override - public WriteResult expand(PCollection input) { + public WriteResult expand(PCollection> input) { Pipeline p = input.getPipeline(); BigQueryOptions options = p.getOptions().as(BigQueryOptions.class); ValueProvider table = write.getTableWithDefaultProject(options); @@ -80,7 +99,8 @@ public WriteResult expand(PCollection input) { e); } - // Create a singleton job ID token at execution time. + // Create a singleton job ID token at execution time. This will be used as the base for all + // load jobs issued from this instance of the transfomr. PCollection singleton = p.apply("Create", Create.of(tempFilePrefix)); PCollectionView jobIdTokenView = p .apply("TriggerIdCreation", Create.of("ignored")) @@ -93,67 +113,71 @@ public String apply(String input) { })) .apply(View.asSingleton()); - PCollection typedInputInGlobalWindow = + PCollection> inputInGlobalWindow = input.apply( - Window.into(new GlobalWindows()) + Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); - // Avoid applying the formatFunction if it is the identity formatter. - PCollection inputInGlobalWindow; - if (write.getFormatFunction() == BigQueryIO.IDENTITY_FORMATTER) { - inputInGlobalWindow = (PCollection) typedInputInGlobalWindow; - } else { - inputInGlobalWindow = - typedInputInGlobalWindow.apply( - MapElements.into(new TypeDescriptor() {}).via(write.getFormatFunction())); - } - // PCollection of filename, file byte size. - PCollection> results = inputInGlobalWindow - .apply("WriteBundles", - ParDo.of(new WriteBundles(tempFilePrefix))); + // PCollection of filename, file byte size, and table destination. + PCollection results = inputInGlobalWindow + .apply("WriteBundlesToFiles", + ParDo.of(new WriteBundlesToFiles(tempFilePrefix))); - TupleTag>> multiPartitionsTag = - new TupleTag>>("multiPartitionsTag") {}; - TupleTag>> singlePartitionTag = - new TupleTag>>("singlePartitionTag") {}; + 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. PCollectionTuple partitions = singleton.apply(ParDo .of(new WritePartition( + write.getTable(), + write.getTableDescription(), resultsView, multiPartitionsTag, singlePartitionTag)) .withSideInputs(resultsView) .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); - // If WriteBundles produced more than MAX_NUM_FILES files or MAX_SIZE_BYTES bytes, then + // 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()); + + // 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. - PCollection tempTables = partitions.get(multiPartitionsTag) - .apply("MultiPartitionsGroupByKey", GroupByKey.>create()) + PCollection> tempTables = partitions.get(multiPartitionsTag) + // What's this GroupByKey for? Is this so we have a deterministic temp tables? If so, maybe + // Reshuffle is better here. + .apply("MultiPartitionsGroupByKey", + GroupByKey., List>create()) .apply("MultiPartitionsWriteTables", ParDo.of(new WriteTables( false, write.getBigQueryServices(), jobIdTokenView, tempFilePrefix, - NestedValueProvider.of(table, new TableRefToJson()), - write.getJsonSchema(), WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, - write.getTableDescription())) + schemaFunction)) .withSideInputs(jobIdTokenView)); - PCollectionView> tempTablesView = tempTables - .apply("TempTablesView", View.asIterable()); + // This view maps each final table destination to the set of temporary partitioned tables + // the PCollection was loaded into. + PCollectionView>> tempTablesView = tempTables + .apply("TempTablesView", View.asMultimap()); + singleton.apply(ParDo .of(new WriteRename( write.getBigQueryServices(), jobIdTokenView, - NestedValueProvider.of(table, new TableRefToJson()), write.getWriteDisposition(), write.getCreateDisposition(), tempTablesView, @@ -162,17 +186,16 @@ public String apply(String input) { // Write single partition to final table partitions.get(singlePartitionTag) - .apply("SinglePartitionGroupByKey", GroupByKey.>create()) + .apply("SinglePartitionGroupByKey", + GroupByKey., List>create()) .apply("SinglePartitionWriteTables", ParDo.of(new WriteTables( true, write.getBigQueryServices(), jobIdTokenView, tempFilePrefix, - NestedValueProvider.of(table, new TableRefToJson()), - write.getJsonSchema(), write.getWriteDisposition(), write.getCreateDisposition(), - write.getTableDescription())) + schemaFunction)) .withSideInputs(jobIdTokenView)); 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 af19b83eec9d..f1baaf736808 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 @@ -984,7 +984,8 @@ public WriteResult expand(PCollection input) { if (input.isBounded() == IsBounded.UNBOUNDED) { return rowsWithDestination.apply(new StreamingInserts(this)); } else { - return input.apply(new BatchLoadBigQuery(this)); + + return rowsWithDestination.apply(new BatchLoads(this)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index 631afeb99b87..1c2b256f27d1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -20,6 +20,8 @@ import com.google.api.services.bigquery.model.TableReference; +import java.util.Objects; + /** * Encapsulates a BigQuery table destination. */ @@ -42,7 +44,6 @@ public String getTableSpec() { return tableSpec; } - public TableReference getTableReference() { return BigQueryHelpers.parseTableSpec(tableSpec); } @@ -50,4 +51,18 @@ public TableReference getTableReference() { public String getTableDescription() { return tableDescription; } + + @Override + public boolean equals(Object o) { + if (!(o instanceof TableDestination)) { + return false; + } + TableDestination other = (TableDestination) o; + return tableSpec == other.tableSpec && tableDescription == other.tableDescription; + } + + @Override + public int hashCode() { + return Objects.hash(tableSpec, tableDescription); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java index 014c498bc83a..a1f615327d73 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java @@ -48,6 +48,14 @@ class TableRowWriter { protected String mimeType = MimeTypes.TEXT; private CountingOutputStream out; + public class Result { + String filename; + long byteSize; + public Result(String filename, long byteSize) { + this.filename = filename; + this.byteSize = byteSize; + } + } TableRowWriter(String basename) { this.tempFilePrefix = basename; } @@ -77,8 +85,8 @@ public void write(TableRow value) throws Exception { out.write(NEWLINE); } - public final KV close() throws IOException { + public final Result close() throws IOException { channel.close(); - return KV.of(fileName, out.getCount()); + return new Result(fileName, out.getCount()); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java similarity index 66% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundles.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index 62192264ff90..4e6167b7d867 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -19,8 +19,11 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; + +import java.util.Map; import java.util.UUID; +import com.google.common.collect.Maps; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; @@ -31,25 +34,41 @@ * Writes each bundle of {@link TableRow} elements out to a separate file using * {@link TableRowWriter}. */ -class WriteBundles extends DoFn> { - private static final Logger LOG = LoggerFactory.getLogger(WriteBundles.class); +class WriteBundlesToFiles extends DoFn, WriteBundlesToFiles.Result> { + private static final Logger LOG = LoggerFactory.getLogger(WriteBundlesToFiles.class); - private transient TableRowWriter writer = null; + // Map from tablespec to a writer for that table. + private transient Map writers; private final String tempFilePrefix; - WriteBundles(String tempFilePrefix) { + public static class Result { + public String filename; + public Long fileByteSize; + public TableDestination tableDestination; + + public Result(String filename, Long fileByteSize, TableDestination tableDestination) { + this.filename = filename; + this.fileByteSize = fileByteSize; + this.tableDestination = tableDestination; + } + } + WriteBundlesToFiles(String tempFilePrefix) { this.tempFilePrefix = tempFilePrefix; + this.writers = Maps.newHashMap(); } @ProcessElement public void processElement(ProcessContext c) throws Exception { + // ??? can we assume Java8? + TableRowWriter writer = writers.getOrDefault(c.element().getKey(), null); if (writer == null) { writer = new TableRowWriter(tempFilePrefix); writer.open(UUID.randomUUID().toString()); + writers.put(c.element().getKey(), writer); LOG.debug("Done opening writer {}", writer); } try { - writer.write(c.element()); + writer.write(c.element().getValue()); } catch (Exception e) { // Discard write result and close the write. try { @@ -65,10 +84,11 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - if (writer != null) { - c.output(writer.close()); - writer = null; + for (Map.Entry entry : writers.entrySet()) { + TableRowWriter.Result result = entry.getValue().close(); + c.output(new Result(result.filename, result.byteSize, entry.getKey())); } + writers.clear(); } @Override 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 1b6492e97416..8e1b16d65427 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,27 +18,40 @@ 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; +import java.util.Map; import java.util.UUID; + 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; import org.apache.beam.sdk.values.TupleTag; /** - * Partitions temporary files based on number of files and file sizes. + * 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>> { - private final PCollectionView>> resultsView; - private TupleTag>> multiPartitionsTag; - private TupleTag>> singlePartitionTag; +class WritePartition extends DoFn, List>> { + private final ValueProvider singletonOutputTable; + private final String singletonOutputTableDescription; + private final PCollectionView> resultsView; + private TupleTag, List>> multiPartitionsTag; + private TupleTag, List>> singlePartitionTag; public WritePartition( - PCollectionView>> resultsView, - TupleTag>> multiPartitionsTag, - TupleTag>> singlePartitionTag) { + ValueProvider singletonOutputTable, + String singletonOutputTableDescription, + PCollectionView> resultsView, + TupleTag, List>> multiPartitionsTag, + TupleTag, List>> singlePartitionTag) { + this.singletonOutputTable = singletonOutputTable; + this.singletonOutputTableDescription = singletonOutputTableDescription; this.resultsView = resultsView; this.multiPartitionsTag = multiPartitionsTag; this.singlePartitionTag = singlePartitionTag; @@ -46,34 +59,62 @@ public WritePartition( @ProcessElement public void processElement(ProcessContext c) throws Exception { - List> results = Lists.newArrayList(c.sideInput(resultsView)); - if (results.isEmpty()) { - TableRowWriter writer = new TableRowWriter(c.element()); - writer.open(UUID.randomUUID().toString()); - results.add(writer.close()); + 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() && singletonOutputTable != null) { + TableReference singletonTable = singletonOutputTable.get(); + if (singletonTable != null) { + TableRowWriter writer = new TableRowWriter(c.element()); + writer.open(UUID.randomUUID().toString()); + TableRowWriter.Result writerResult = writer.close(); + results.add(new Result(writerResult.filename, writerResult.byteSize, + new TableDestination(singletonTable, singletonOutputTableDescription))); + } } + long partitionId = 0; - int currNumFiles = 0; - long currSizeBytes = 0; - List currResults = Lists.newArrayList(); + Map currNumFilesMap = Maps.newHashMap(); + Map currSizeBytesMap = Maps.newHashMap(); + Map>> currResultsMap = Maps.newHashMap(); for (int i = 0; i < results.size(); ++i) { - KV fileResult = results.get(i); + WriteBundlesToFiles.Result fileResult = results.get(i); + TableDestination tableDestination = fileResult.tableDestination; + // JAVA8 + List> partitions = currResultsMap.getOrDefault(tableDestination, null); + if (partitions == null) { + partitions = Lists.newArrayList(); + partitions.add(Lists.newArrayList()); + currResultsMap.put(tableDestination, partitions); + } + int currNumFiles = currNumFilesMap.getOrDefault(tableDestination, 0); + long currSizeBytes = currSizeBytesMap.getOrDefault(tableDestination, 0L); if (currNumFiles + 1 > Write.MAX_NUM_FILES - || currSizeBytes + fileResult.getValue() > Write.MAX_SIZE_BYTES) { - c.output(multiPartitionsTag, KV.of(++partitionId, currResults)); - currResults = Lists.newArrayList(); + || currSizeBytes + fileResult.fileByteSize > Write.MAX_SIZE_BYTES) { + // Add a new partition for this table. + partitions.add(Lists.newArrayList()); + // c.sideOutput(multiPartitionsTag, KV.of(++partitionId, currResults)); currNumFiles = 0; currSizeBytes = 0; + currNumFilesMap.remove(tableDestination); + currSizeBytesMap.remove(tableDestination); } - ++currNumFiles; - currSizeBytes += fileResult.getValue(); - currResults.add(fileResult.getKey()); + currNumFilesMap.put(tableDestination, currNumFiles + 1); + currSizeBytesMap.put(tableDestination, currSizeBytes + fileResult.fileByteSize); + // Always add to the most recent partition for this table. + partitions.get(partitions.size() - 1).add(fileResult.filename); } - if (partitionId == 0) { - c.output(singlePartitionTag, KV.of(++partitionId, currResults)); - } else { - c.output(multiPartitionsTag, KV.of(++partitionId, currResults)); + + for (Map.Entry>> entry : currResultsMap.entrySet()) { + TableDestination tableDestination = entry.getKey(); + List> partitions = entry.getValue(); + TupleTag, List>> outputTag = + (partitions.size() == 1) ? singlePartitionTag : multiPartitionsTag; + for (int i = 0; i < partitions.size(); ++i) { + c.output(outputTag, KV.of(KV.of(tableDestination, i + 1), partitions.get(i))); + } } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index 8cb943913b56..fbfb290fc1e1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; +import avro.shaded.com.google.common.collect.Maps; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationTableCopy; import com.google.api.services.bigquery.model.JobReference; @@ -25,6 +26,7 @@ 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.io.gcp.bigquery.BigQueryHelpers.Status; @@ -49,24 +51,21 @@ class WriteRename extends DoFn { private final BigQueryServices bqServices; private final PCollectionView jobIdToken; - private final ValueProvider jsonTableRef; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; - private final PCollectionView> tempTablesView; + private final PCollectionView>> tempTablesView; @Nullable private final String tableDescription; public WriteRename( BigQueryServices bqServices, PCollectionView jobIdToken, - ValueProvider jsonTableRef, WriteDisposition writeDisposition, CreateDisposition createDisposition, - PCollectionView> tempTablesView, + PCollectionView>> tempTablesView, @Nullable String tableDescription) { this.bqServices = bqServices; this.jobIdToken = jobIdToken; - this.jsonTableRef = jsonTableRef; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; this.tempTablesView = tempTablesView; @@ -75,30 +74,40 @@ public WriteRename( @ProcessElement public void processElement(ProcessContext c) throws Exception { - List tempTablesJson = Lists.newArrayList(c.sideInput(tempTablesView)); + Map> tempTablesMap = + Maps.newHashMap(c.sideInput(tempTablesView)); - // Do not copy if no temp tables are provided - if (tempTablesJson.size() == 0) { - return; - } + // Process each destination table. + for (Map.Entry> entry : tempTablesMap.entrySet()) { + TableDestination finalTableDestination = entry.getKey(); + List tempTablesJson = Lists.newArrayList(entry.getValue()); + // Do not copy if no temp tables are provided + if (tempTablesJson.size() == 0) { + return; + } + + List tempTables = Lists.newArrayList(); + for (String table : tempTablesJson) { + tempTables.add(BigQueryHelpers.fromJsonString(table, TableReference.class)); + } + + // Make sure each destination table gets a unique job id. + String jobIdPrefix = String.format( + c.sideInput(jobIdToken) + "0x%08x", finalTableDestination.hashCode()); + copy( + bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), + bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), + jobIdPrefix, + finalTableDestination.getTableReference(), + tempTables, + writeDisposition, + createDisposition, + tableDescription); - List tempTables = Lists.newArrayList(); - for (String table : tempTablesJson) { - tempTables.add(BigQueryHelpers.fromJsonString(table, TableReference.class)); + DatasetService tableService = + bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); + removeTemporaryTables(tableService, tempTables); } - copy( - bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), - bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), - c.sideInput(jobIdToken), - BigQueryHelpers.fromJsonString(jsonTableRef.get(), TableReference.class), - tempTables, - writeDisposition, - createDisposition, - tableDescription); - - DatasetService tableService = - bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); - removeTemporaryTables(tableService, tempTables); } private void copy( @@ -170,8 +179,6 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("jsonTableRef", jsonTableRef) - .withLabel("Table Reference")) .add(DisplayData.item("writeDisposition", writeDisposition.toString()) .withLabel("Write Disposition")) .add(DisplayData.item("createDisposition", createDisposition.toString()) 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 29680adb1a77..5051c95f7a16 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 @@ -41,6 +41,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; 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; @@ -57,48 +58,45 @@ /** * Writes partitions to BigQuery tables. */ -class WriteTables extends DoFn>>, String> { +class WriteTables extends DoFn, Iterable>>, + KV> { private static final Logger LOG = LoggerFactory.getLogger(WriteTables.class); private final boolean singlePartition; private final BigQueryServices bqServices; private final PCollectionView jobIdToken; private final String tempFilePrefix; - private final ValueProvider jsonTableRef; - private final ValueProvider jsonSchema; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; - @Nullable - private final String tableDescription; + private final SerializableFunction schemaFunction; public WriteTables( boolean singlePartition, BigQueryServices bqServices, PCollectionView jobIdToken, String tempFilePrefix, - ValueProvider jsonTableRef, - ValueProvider jsonSchema, WriteDisposition writeDisposition, CreateDisposition createDisposition, - @Nullable String tableDescription) { + SerializableFunction schemaFunction) { this.singlePartition = singlePartition; this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.tempFilePrefix = tempFilePrefix; - this.jsonTableRef = jsonTableRef; - this.jsonSchema = jsonSchema; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; - this.tableDescription = tableDescription; + this.schemaFunction = schemaFunction; } @ProcessElement public void processElement(ProcessContext c) throws Exception { - List partition = Lists.newArrayList(c.element().getValue()).get(0); + TableDestination tableDestination = c.element().getKey().getKey(); + Integer partition = c.element().getKey().getValue(); + List partitionFiles = Lists.newArrayList(c.element().getValue()).get(0); + // Job ID must be different for each partition of each table. String jobIdPrefix = String.format( - c.sideInput(jobIdToken) + "_%05d", c.element().getKey()); - TableReference ref = BigQueryHelpers.fromJsonString(jsonTableRef.get(), - TableReference.class); + c.sideInput(jobIdToken) + "0x%08x_%05d", tableDestination.hashCode(), partition); + + TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { ref.setTableId(jobIdPrefix); } @@ -108,15 +106,14 @@ public void processElement(ProcessContext c) throws Exception { bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, ref, - BigQueryHelpers.fromJsonString( - jsonSchema == null ? null : jsonSchema.get(), TableSchema.class), - partition, + schemaFunction.apply(tableDestination), + partitionFiles, writeDisposition, createDisposition, - tableDescription); - c.output(BigQueryHelpers.toJsonString(ref)); + tableDestination.getTableDescription()); + c.output(KV.of(tableDestination, BigQueryHelpers.toJsonString(ref))); - removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partition); + removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partitionFiles); } private void load( @@ -202,12 +199,6 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull(DisplayData.item("tempFilePrefix", tempFilePrefix) - .withLabel("Temporary File Prefix")) - .addIfNotNull(DisplayData.item("jsonTableRef", jsonTableRef) - .withLabel("Table Reference")) - .addIfNotNull(DisplayData.item("jsonSchema", jsonSchema) - .withLabel("Table Schema")) - .addIfNotNull(DisplayData.item("tableDescription", tableDescription) - .withLabel("Table Description")); + .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 d953edd1a9a9..af3948301605 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 @@ -2078,26 +2078,27 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa files.add(KV.of(fileName, fileSize)); } - TupleTag>> multiPartitionsTag = - new TupleTag>>("multiPartitionsTag") {}; - TupleTag>> singlePartitionTag = - new TupleTag>>("singlePartitionTag") {}; - - PCollection> filesPCollection = - p.apply(Create.of(files).withType(new TypeDescriptor>() {})); - PCollectionView>> filesView = PCollectionViews.iterableView( - filesPCollection, + TupleTag, List>> multiPartitionsTag = + new TupleTag, List>>("multiPartitionsTag") {}; + TupleTag, List>> singlePartitionTag = + new TupleTag, List>>("singlePartitionTag") {}; + + PCollectionView> resultsView = + PCollectionViews.iterableView( + p, WindowingStrategy.globalDefault(), KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); WritePartition writePartition = - new WritePartition(filesView, multiPartitionsTag, singlePartitionTag); + new WritePartition(null, null, resultsView, + multiPartitionsTag, singlePartitionTag); - DoFnTester>> tester = DoFnTester.of(writePartition); - tester.setSideInput(filesView, GlobalWindow.INSTANCE, files); + DoFnTester, List>> tester = + DoFnTester.of(writePartition); + tester.setSideInput(resultsView, GlobalWindow.INSTANCE, files); tester.processElement(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); - List>> partitions; + List, List>> partitions; if (expectedNumPartitions > 1) { partitions = tester.takeOutputElements(multiPartitionsTag); } else { From 6ef2768ae6d9345339cddc9f3f9e2f20eb9a74c6 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Fri, 31 Mar 2017 11:19:25 -0700 Subject: [PATCH 5/7] Refactor batch loads, and add support for windowed writes. --- .../apache/beam/sdk/util/IOChannelUtils.java | 9 + .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 49 +- .../beam/sdk/io/gcp/bigquery/ShardedKey.java | 24 +- .../sdk/io/gcp/bigquery/TableDestination.java | 10 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 54 +- .../sdk/io/gcp/bigquery/WritePartition.java | 28 +- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 13 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 14 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 838 +++++------------- .../io/gcp/bigquery/FakeBigQueryServices.java | 96 ++ .../io/gcp/bigquery/FakeDatasetService.java | 172 ++++ .../sdk/io/gcp/bigquery/FakeJobService.java | 273 ++++++ .../sdk/io/gcp/bigquery/TableContainer.java | 36 + 13 files changed, 948 insertions(+), 668 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java index d60ee976e061..2c18c5c50442 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java @@ -28,6 +28,7 @@ import com.google.common.collect.TreeMultimap; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; import java.text.DecimalFormat; import java.util.Arrays; @@ -193,6 +194,14 @@ public static WritableByteChannel create(String filename, String mimeType) return getFactory(filename).create(filename, mimeType); } + /** + * Creates a read channel for the given filename. + */ + public static ReadableByteChannel open(String filename) + throws IOException { + return getFactory(filename).open(filename); + } + /** * Creates a write channel for the given file components. * 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 8594211ff0d9..5e80fae778aa 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 @@ -26,6 +26,10 @@ 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; +import org.apache.beam.sdk.coders.ListCoder; +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.BigQueryOptions; @@ -61,16 +65,17 @@ class BatchLoads extends private static class ConstantSchemaFunction implements SerializableFunction { private final @Nullable - String jsonSchema; + ValueProvider jsonSchema; - ConstantSchemaFunction(TableSchema schema) { - this.jsonSchema = BigQueryHelpers.toJsonString(schema); + ConstantSchemaFunction(ValueProvider jsonSchema) { + this.jsonSchema = jsonSchema; } @Override @Nullable public TableSchema apply(TableDestination table) { - return BigQueryHelpers.fromJsonString(jsonSchema, TableSchema.class); + return BigQueryHelpers.fromJsonString( + jsonSchema == null ? null : jsonSchema.get(), TableSchema.class); } } @@ -114,7 +119,7 @@ public String apply(String input) { .apply(View.asSingleton()); PCollection> inputInGlobalWindow = - input.apply( + input.apply("rewindowIntoGlobal", Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); @@ -122,12 +127,13 @@ public String apply(String input) { // PCollection of filename, file byte size, and table destination. PCollection results = inputInGlobalWindow .apply("WriteBundlesToFiles", - ParDo.of(new WriteBundlesToFiles(tempFilePrefix))); + ParDo.of(new WriteBundlesToFiles(tempFilePrefix))) + .setCoder(WriteBundlesToFiles.ResultCoder.of()); - 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") {}; // Turn the list of files and record counts in a PCollectionView that can be used as a // side input. @@ -136,9 +142,9 @@ public String apply(String input) { // 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. - PCollectionTuple partitions = singleton.apply(ParDo - .of(new WritePartition( - write.getTable(), + PCollectionTuple partitions = singleton.apply("WritePartition", + ParDo.of(new WritePartition( + write.getJsonTableRef(), write.getTableDescription(), resultsView, multiPartitionsTag, @@ -148,17 +154,22 @@ public String apply(String 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. + // TODO: Turn this into a side-input instead. SerializableFunction schemaFunction = - new ConstantSchemaFunction(write.getSchema()); + new ConstantSchemaFunction(write.getJsonSchema()); + 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. PCollection> tempTables = partitions.get(multiPartitionsTag) + .setCoder(partitionsCoder) // What's this GroupByKey for? Is this so we have a deterministic temp tables? If so, maybe // Reshuffle is better here. .apply("MultiPartitionsGroupByKey", - GroupByKey., List>create()) + GroupByKey., List>create()) .apply("MultiPartitionsWriteTables", ParDo.of(new WriteTables( false, write.getBigQueryServices(), @@ -174,20 +185,20 @@ public String apply(String input) { PCollectionView>> tempTablesView = tempTables .apply("TempTablesView", View.asMultimap()); - singleton.apply(ParDo + singleton.apply("WriteRename", ParDo .of(new WriteRename( write.getBigQueryServices(), jobIdTokenView, write.getWriteDisposition(), write.getCreateDisposition(), - tempTablesView, - write.getTableDescription())) + tempTablesView)) .withSideInputs(tempTablesView, jobIdTokenView)); // Write single partition to final table partitions.get(singlePartitionTag) + .setCoder(partitionsCoder) .apply("SinglePartitionGroupByKey", - GroupByKey., List>create()) + GroupByKey., List>create()) .apply("SinglePartitionWriteTables", ParDo.of(new WriteTables( true, write.getBigQueryServices(), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java index 8c968df2cf36..ab57446028a4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java @@ -18,10 +18,13 @@ package org.apache.beam.sdk.io.gcp.bigquery; +import java.io.Serializable; +import java.util.Objects; + /** * A key and a shard number. */ -class ShardedKey { +class ShardedKey implements Serializable { private final K key; private final int shardNumber; @@ -41,4 +44,23 @@ public K getKey() { public int getShardNumber() { return shardNumber; } + + @Override + public String toString() { + return "key: " + key + " shard: " + shardNumber; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof ShardedKey)) { + return false; + } + ShardedKey other = (ShardedKey) o; + return (key == other.key) && (shardNumber == other.shardNumber); + } + + @Override + public int hashCode() { + return Objects.hash(key, shardNumber); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index 1c2b256f27d1..e8538e03412e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -20,12 +20,13 @@ import com.google.api.services.bigquery.model.TableReference; +import java.io.Serializable; import java.util.Objects; /** * Encapsulates a BigQuery table destination. */ -public class TableDestination { +public class TableDestination implements Serializable { private final String tableSpec; private final String tableDescription; @@ -52,13 +53,18 @@ public String getTableDescription() { return tableDescription; } + @Override + public String toString() { + return "tableSpec: " + tableSpec + " tableDescription: " + tableDescription; + } + @Override public boolean equals(Object o) { if (!(o instanceof TableDestination)) { return false; } TableDestination other = (TableDestination) o; - return tableSpec == other.tableSpec && tableDescription == other.tableDescription; + return (tableSpec == other.tableSpec) && (tableDescription == other.tableDescription); } @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 4e6167b7d867..b8069f6614e0 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 @@ -20,10 +20,19 @@ import com.google.api.services.bigquery.model.TableRow; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; import java.util.Map; import java.util.UUID; import com.google.common.collect.Maps; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.TableRowJsonCoder; +import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; @@ -41,7 +50,7 @@ class WriteBundlesToFiles extends DoFn, WriteBund private transient Map writers; private final String tempFilePrefix; - public static class Result { + public static class Result implements Serializable { public String filename; public Long fileByteSize; public TableDestination tableDestination; @@ -52,15 +61,54 @@ public Result(String filename, Long fileByteSize, TableDestination tableDestinat this.tableDestination = tableDestination; } } + + public static class ResultCoder extends AtomicCoder { + private static final ResultCoder INSTANCE = new ResultCoder(); + + public static ResultCoder of() { + return INSTANCE; + } + + @Override + 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()); + } + + @Override + public Result decode(InputStream inStream, Context context) + throws IOException { + return new Result(stringCoder.decode(inStream, context.nested()), + longCoder.decode(inStream, context.nested()), + tableDestinationCoder.decode(inStream, context.nested())); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + } + + StringUtf8Coder stringCoder = StringUtf8Coder.of(); + VarLongCoder longCoder = VarLongCoder.of(); + TableDestinationCoder tableDestinationCoder = TableDestinationCoder.of(); + } + WriteBundlesToFiles(String tempFilePrefix) { this.tempFilePrefix = tempFilePrefix; + } + + @StartBundle + public void startBundle(Context c) { this.writers = Maps.newHashMap(); } @ProcessElement public void processElement(ProcessContext c) throws Exception { - // ??? can we assume Java8? - TableRowWriter writer = writers.getOrDefault(c.element().getKey(), null); + TableRowWriter writer = writers.get(c.element().getKey()); if (writer == null) { writer = new TableRowWriter(tempFilePrefix); writer.open(UUID.randomUUID().toString()); 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 8e1b16d65427..c48955b597be 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 @@ -37,20 +37,20 @@ * 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 singletonOutputTable; +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; + private TupleTag, List>> multiPartitionsTag; + private TupleTag, List>> singlePartitionTag; public WritePartition( - ValueProvider singletonOutputTable, + ValueProvider singletonOutputJsonTableRef, String singletonOutputTableDescription, PCollectionView> resultsView, - TupleTag, List>> multiPartitionsTag, - TupleTag, List>> singlePartitionTag) { - this.singletonOutputTable = singletonOutputTable; + TupleTag, List>> multiPartitionsTag, + TupleTag, List>> singlePartitionTag) { + this.singletonOutputJsonTableRef = singletonOutputJsonTableRef; this.singletonOutputTableDescription = singletonOutputTableDescription; this.resultsView = resultsView; this.multiPartitionsTag = multiPartitionsTag; @@ -63,8 +63,9 @@ public void processElement(ProcessContext c) throws Exception { // 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() && singletonOutputTable != null) { - TableReference singletonTable = singletonOutputTable.get(); + if (results.isEmpty() && singletonOutputJsonTableRef != null) { + TableReference singletonTable = BigQueryHelpers.fromJsonString( + singletonOutputJsonTableRef.get(), TableReference.class); if (singletonTable != null) { TableRowWriter writer = new TableRowWriter(c.element()); writer.open(UUID.randomUUID().toString()); @@ -82,8 +83,7 @@ public void processElement(ProcessContext c) throws Exception { for (int i = 0; i < results.size(); ++i) { WriteBundlesToFiles.Result fileResult = results.get(i); TableDestination tableDestination = fileResult.tableDestination; - // JAVA8 - List> partitions = currResultsMap.getOrDefault(tableDestination, null); + List> partitions = currResultsMap.get(tableDestination); if (partitions == null) { partitions = Lists.newArrayList(); partitions.add(Lists.newArrayList()); @@ -110,10 +110,10 @@ public void processElement(ProcessContext c) throws Exception { for (Map.Entry>> entry : currResultsMap.entrySet()) { TableDestination tableDestination = entry.getKey(); List> partitions = entry.getValue(); - TupleTag, List>> outputTag = + TupleTag, List>> outputTag = (partitions.size() == 1) ? singlePartitionTag : multiPartitionsTag; for (int i = 0; i < partitions.size(); ++i) { - c.output(outputTag, KV.of(KV.of(tableDestination, i + 1), partitions.get(i))); + c.output(outputTag, KV.of(ShardedKey.of(tableDestination, i + 1), partitions.get(i))); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index fbfb290fc1e1..752e7d3b63b8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -18,12 +18,12 @@ package org.apache.beam.sdk.io.gcp.bigquery; -import avro.shaded.com.google.common.collect.Maps; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationTableCopy; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.IOException; import java.util.List; import java.util.Map; @@ -36,7 +36,6 @@ 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.options.BigQueryOptions; -import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollectionView; @@ -53,23 +52,21 @@ class WriteRename extends DoFn { private final PCollectionView jobIdToken; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; + // Map from final destination to a list of temporary tables that need to be copied into it. private final PCollectionView>> tempTablesView; - @Nullable - private final String tableDescription; + public WriteRename( BigQueryServices bqServices, PCollectionView jobIdToken, WriteDisposition writeDisposition, CreateDisposition createDisposition, - PCollectionView>> tempTablesView, - @Nullable String tableDescription) { + PCollectionView>> tempTablesView) { this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; this.tempTablesView = tempTablesView; - this.tableDescription = tableDescription; } @ProcessElement @@ -102,7 +99,7 @@ public void processElement(ProcessContext c) throws Exception { tempTables, writeDisposition, createDisposition, - tableDescription); + finalTableDestination.getTableDescription()); DatasetService tableService = bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); 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 5051c95f7a16..f7fe87b9747d 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.BigQueryOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -57,8 +56,12 @@ /** * Writes partitions to BigQuery tables. + * + *

The input is a list of files corresponding to a 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 the final table to the temporary tables for each partition of that table. */ -class WriteTables extends DoFn, Iterable>>, +class WriteTables extends DoFn, Iterable>>, KV> { private static final Logger LOG = LoggerFactory.getLogger(WriteTables.class); @@ -90,23 +93,24 @@ public WriteTables( @ProcessElement public void processElement(ProcessContext c) throws Exception { TableDestination tableDestination = c.element().getKey().getKey(); - Integer partition = c.element().getKey().getValue(); + Integer partition = c.element().getKey().getShardNumber(); List partitionFiles = Lists.newArrayList(c.element().getValue()).get(0); // Job ID must be different for each partition of each table. String jobIdPrefix = String.format( - c.sideInput(jobIdToken) + "0x%08x_%05d", tableDestination.hashCode(), partition); + c.sideInput(jobIdToken) + "_0x%08x_%05d", tableDestination.hashCode(), partition); TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { ref.setTableId(jobIdPrefix); } + TableSchema schema = (schemaFunction != null) ? schemaFunction.apply(tableDestination) : null; load( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, ref, - schemaFunction.apply(tableDestination), + schema, partitionFiles, writeDisposition, createDisposition, 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 af3948301605..d1ef8e24b8a4 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 @@ -18,9 +18,6 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; -import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.fromJsonString; 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; @@ -38,13 +35,7 @@ import com.google.api.client.json.GenericJson; import com.google.api.client.util.Data; -import com.google.api.services.bigquery.model.Dataset; -import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; -import com.google.api.services.bigquery.model.JobConfigurationExtract; -import com.google.api.services.bigquery.model.JobConfigurationLoad; -import com.google.api.services.bigquery.model.JobConfigurationQuery; -import com.google.api.services.bigquery.model.JobConfigurationTableCopy; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatistics; import com.google.api.services.bigquery.model.JobStatistics2; @@ -55,18 +46,16 @@ 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.HashBasedTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import java.io.ByteArrayInputStream; +import com.google.common.collect.Maps; + import java.io.File; import java.io.FileFilter; import java.io.IOException; import java.io.InputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.OutputStream; import java.io.Serializable; import java.nio.channels.Channels; @@ -74,15 +63,12 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Set; -import javax.annotation.Nullable; + import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericDatumWriter; @@ -96,17 +82,15 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; 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.BigQueryServices.DatasetService; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; +import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -142,7 +126,6 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; @@ -175,484 +158,17 @@ @RunWith(JUnit4.class) public class BigQueryIOTest implements Serializable { - // Status.UNKNOWN maps to null - private static final Map JOB_STATUS_MAP = ImmutableMap.of( - Status.SUCCEEDED, new Job().setStatus(new JobStatus()), - Status.FAILED, new Job().setStatus(new JobStatus().setErrorResult(new ErrorProto()))); - - - private static class FakeBigQueryServices implements BigQueryServices { - - private String[] jsonTableRowReturns = new String[0]; - private JobService jobService; - private DatasetService datasetService; - - public FakeBigQueryServices withJobService(JobService jobService) { - this.jobService = jobService; - return this; - } - - public FakeBigQueryServices withDatasetService(DatasetService datasetService) { - this.datasetService = datasetService; - return this; - } - - public FakeBigQueryServices readerReturns(String... jsonTableRowReturns) { - this.jsonTableRowReturns = jsonTableRowReturns; - return this; - } - - @Override - public JobService getJobService(BigQueryOptions bqOptions) { - return jobService; - } - - @Override - public DatasetService getDatasetService(BigQueryOptions bqOptions) { - return datasetService; - } - - @Override - public BigQueryJsonReader getReaderFromTable( - BigQueryOptions bqOptions, TableReference tableRef) { - return new FakeBigQueryReader(jsonTableRowReturns); - } - - @Override - public BigQueryJsonReader getReaderFromQuery( - BigQueryOptions bqOptions, String projectId, JobConfigurationQuery queryConfig) { - return new FakeBigQueryReader(jsonTableRowReturns); - } - - private static class FakeBigQueryReader implements BigQueryJsonReader { - private static final int UNSTARTED = -1; - private static final int CLOSED = Integer.MAX_VALUE; - - private String[] jsonTableRowReturns; - private int currIndex; - - FakeBigQueryReader(String[] jsonTableRowReturns) { - this.jsonTableRowReturns = jsonTableRowReturns; - this.currIndex = UNSTARTED; - } - - @Override - public boolean start() throws IOException { - assertEquals(UNSTARTED, currIndex); - currIndex = 0; - return currIndex < jsonTableRowReturns.length; - } - - @Override - public boolean advance() throws IOException { - return ++currIndex < jsonTableRowReturns.length; - } - - @Override - public TableRow getCurrent() throws NoSuchElementException { - if (currIndex >= jsonTableRowReturns.length) { - throw new NoSuchElementException(); - } - return fromJsonString(jsonTableRowReturns[currIndex], TableRow.class); - } - - @Override - public void close() throws IOException { - currIndex = CLOSED; - } - } - } - - private static class FakeJobService implements JobService, Serializable { - - private Object[] startJobReturns; - private Object[] pollJobReturns; - private Object[] getJobReturns; - private String executingProject; - // Both counts will be reset back to zeros after serialization. - // This is a work around for DoFn's verifyUnmodified check. - private transient int startJobCallsCount; - private transient int pollJobStatusCallsCount; - private transient int getJobCallsCount; - - public FakeJobService() { - this.startJobReturns = new Object[0]; - this.pollJobReturns = new Object[0]; - this.getJobReturns = new Object[0]; - this.startJobCallsCount = 0; - this.pollJobStatusCallsCount = 0; - this.getJobCallsCount = 0; - } - - /** - * Sets the return values to mock {@link JobService#startLoadJob}, - * {@link JobService#startExtractJob} and {@link JobService#startQueryJob}. - * - *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. - */ - public FakeJobService startJobReturns(Object... startJobReturns) { - this.startJobReturns = startJobReturns; - return this; - } - - /** - * Sets the return values to mock {@link JobService#getJob}. - * - *

Throws if the {@link Object} is a {@link InterruptedException}, returns otherwise. - */ - public FakeJobService getJobReturns(Object... getJobReturns) { - this.getJobReturns = getJobReturns; - return this; - } - - /** - * Sets the return values to mock {@link JobService#pollJob}. - * - *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. - */ - public FakeJobService pollJobReturns(Object... pollJobReturns) { - this.pollJobReturns = pollJobReturns; - return this; - } - - /** - * Verifies executing project. - */ - public FakeJobService verifyExecutingProject(String executingProject) { - this.executingProject = executingProject; - return this; - } - - @Override - public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig) - throws InterruptedException, IOException { - startJob(jobRef, loadConfig); - } - - @Override - public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig) - throws InterruptedException, IOException { - startJob(jobRef, extractConfig); - } - - @Override - public void startQueryJob(JobReference jobRef, JobConfigurationQuery query) - throws IOException, InterruptedException { - startJob(jobRef, query); - } - - @Override - public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) - throws IOException, InterruptedException { - startJob(jobRef, copyConfig); - } - - @Override - public Job pollJob(JobReference jobRef, int maxAttempts) - throws InterruptedException { - if (!Strings.isNullOrEmpty(executingProject)) { - checkArgument( - jobRef.getProjectId().equals(executingProject), - "Project id: %s is not equal to executing project: %s", - jobRef.getProjectId(), executingProject); - } - - if (pollJobStatusCallsCount < pollJobReturns.length) { - Object ret = pollJobReturns[pollJobStatusCallsCount++]; - if (ret instanceof Job) { - return (Job) ret; - } else if (ret instanceof Status) { - return JOB_STATUS_MAP.get(ret); - } else if (ret instanceof InterruptedException) { - throw (InterruptedException) ret; - } else { - throw new RuntimeException("Unexpected return type: " + ret.getClass()); - } - } else { - throw new RuntimeException( - "Exceeded expected number of calls: " + pollJobReturns.length); - } - } - - private void startJob(JobReference jobRef, GenericJson config) - throws IOException, InterruptedException { - if (!Strings.isNullOrEmpty(executingProject)) { - checkArgument( - jobRef.getProjectId().equals(executingProject), - "Project id: %s is not equal to executing project: %s", - jobRef.getProjectId(), executingProject); - } - - if (startJobCallsCount < startJobReturns.length) { - Object ret = startJobReturns[startJobCallsCount++]; - if (ret instanceof IOException) { - throw (IOException) ret; - } else if (ret instanceof InterruptedException) { - throw (InterruptedException) ret; - } else if (ret instanceof SerializableFunction) { - SerializableFunction fn = - (SerializableFunction) ret; - fn.apply(config); - return; - } else { - return; - } - } else { - throw new RuntimeException( - "Exceeded expected number of calls: " + startJobReturns.length); - } - } - - @Override - public JobStatistics dryRunQuery(String projectId, JobConfigurationQuery query) - throws InterruptedException, IOException { - throw new UnsupportedOperationException(); - } - - @Override - public Job getJob(JobReference jobRef) throws InterruptedException { - if (!Strings.isNullOrEmpty(executingProject)) { - checkArgument( - jobRef.getProjectId().equals(executingProject), - "Project id: %s is not equal to executing project: %s", - jobRef.getProjectId(), executingProject); - } - - if (getJobCallsCount < getJobReturns.length) { - Object ret = getJobReturns[getJobCallsCount++]; - if (ret == null) { - return null; - } else if (ret instanceof Job) { - return (Job) ret; - } else if (ret instanceof InterruptedException) { - throw (InterruptedException) ret; - } else { - throw new RuntimeException("Unexpected return type: " + ret.getClass()); - } - } else { - throw new RuntimeException( - "Exceeded expected number of calls: " + getJobReturns.length); - } - } - - ////////////////////////////////// SERIALIZATION METHODS //////////////////////////////////// - private void writeObject(ObjectOutputStream out) throws IOException { - out.writeObject(replaceJobsWithBytes(startJobReturns)); - out.writeObject(replaceJobsWithBytes(pollJobReturns)); - out.writeObject(replaceJobsWithBytes(getJobReturns)); - out.writeObject(executingProject); - } - - private Object[] replaceJobsWithBytes(Object[] objs) { - Object[] copy = Arrays.copyOf(objs, objs.length); - for (int i = 0; i < copy.length; i++) { - checkArgument( - copy[i] == null || copy[i] instanceof Serializable || copy[i] instanceof Job, - "Only serializable elements and jobs can be added add to Job Returns"); - if (copy[i] instanceof Job) { - try { - // Job is not serializable, so encode the job as a byte array. - copy[i] = Transport.getJsonFactory().toByteArray(copy[i]); - } catch (IOException e) { - throw new IllegalArgumentException( - String.format("Could not encode Job %s via available JSON factory", copy[i])); - } - } - } - return copy; - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - this.startJobReturns = replaceBytesWithJobs(in.readObject()); - this.pollJobReturns = replaceBytesWithJobs(in.readObject()); - this.getJobReturns = replaceBytesWithJobs(in.readObject()); - this.executingProject = (String) in.readObject(); - } - - private Object[] replaceBytesWithJobs(Object obj) throws IOException { - checkState(obj instanceof Object[]); - Object[] objs = (Object[]) obj; - Object[] copy = Arrays.copyOf(objs, objs.length); - for (int i = 0; i < copy.length; i++) { - if (copy[i] instanceof byte[]) { - Job job = Transport.getJsonFactory() - .createJsonParser(new ByteArrayInputStream((byte[]) copy[i])) - .parse(Job.class); - copy[i] = job; - } - } - return copy; - } - } - - private static class TableContainer { - Table table; - List rows; - List ids; - - TableContainer(Table table) { - this.table = table; - this.rows = new ArrayList<>(); - this.ids = new ArrayList<>(); - } - - TableContainer addRow(TableRow row, String id) { - rows.add(row); - ids.add(id); - return this; - } - - Table getTable() { - return table; - } - - List getRows() { - return rows; - } - } - // Table information must be static, as each ParDo will get a separate instance of // FakeDatasetServices, and they must all modify the same storage. - private static com.google.common.collect.Table> + static com.google.common.collect.Table> tables = HashBasedTable.create(); - /** A fake dataset service that can be serialized, for use in testReadFromTable. */ - private static class FakeDatasetService implements DatasetService, Serializable { - @Override - public Table getTable(TableReference tableRef) - throws InterruptedException, IOException { - synchronized (tables) { - Map dataset = - checkNotNull( - tables.get(tableRef.getProjectId(), tableRef.getDatasetId()), - "Tried to get a dataset %s:%s from %s, but no such dataset was set", - tableRef.getProjectId(), - tableRef.getDatasetId(), - tableRef.getTableId(), - FakeDatasetService.class.getSimpleName()); - TableContainer tableContainer = dataset.get(tableRef.getTableId()); - return tableContainer == null ? null : tableContainer.getTable(); - } - } - - public List getAllRows(String projectId, String datasetId, String tableId) - throws InterruptedException, IOException { - synchronized (tables) { - return getTableContainer(projectId, datasetId, tableId).getRows(); - } - } - - private TableContainer getTableContainer(String projectId, String datasetId, String tableId) - throws InterruptedException, IOException { - synchronized (tables) { - Map dataset = - checkNotNull( - tables.get(projectId, datasetId), - "Tried to get a dataset %s:%s from %s, but no such dataset was set", - projectId, - datasetId, - FakeDatasetService.class.getSimpleName()); - return checkNotNull(dataset.get(tableId), - "Tried to get a table %s:%s.%s from %s, but no such table was set", - projectId, - datasetId, - tableId, - FakeDatasetService.class.getSimpleName()); - } - } - - @Override - public void deleteTable(TableReference tableRef) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); - } - - - @Override - public void createTable(Table table) throws IOException { - TableReference tableReference = table.getTableReference(); - synchronized (tables) { - Map dataset = - checkNotNull( - tables.get(tableReference.getProjectId(), tableReference.getDatasetId()), - "Tried to get a dataset %s:%s from %s, but no such table was set", - tableReference.getProjectId(), - tableReference.getDatasetId(), - FakeDatasetService.class.getSimpleName()); - TableContainer tableContainer = dataset.get(tableReference.getTableId()); - if (tableContainer == null) { - tableContainer = new TableContainer(table); - dataset.put(tableReference.getTableId(), tableContainer); - } - } - } - - @Override - public boolean isTableEmpty(TableReference tableRef) - throws IOException, InterruptedException { - Long numBytes = getTable(tableRef).getNumBytes(); - return numBytes == null || numBytes == 0L; - } - - @Override - public Dataset getDataset( - String projectId, String datasetId) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); - } - - @Override - public void createDataset( - String projectId, String datasetId, String location, String description) - throws IOException, InterruptedException { - synchronized (tables) { - Map dataset = tables.get(projectId, datasetId); - if (dataset == null) { - dataset = new HashMap<>(); - tables.put(projectId, datasetId, dataset); - } - } - } - - @Override - public void deleteDataset(String projectId, String datasetId) - throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); - } - - @Override - public long insertAll( - TableReference ref, List rowList, @Nullable List insertIdList) - throws IOException, InterruptedException { - synchronized (tables) { - assertEquals(rowList.size(), insertIdList.size()); - - long dataSize = 0; - TableContainer tableContainer = getTableContainer( - ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); - for (int i = 0; i < rowList.size(); ++i) { - System.out.println("adding row " + rowList.get(i)); - tableContainer.addRow(rowList.get(i), insertIdList.get(i)); - dataSize += rowList.get(i).toString().length(); - } - return dataSize; - } - } - - @Override - public Table patchTableDescription(TableReference tableReference, - @Nullable String tableDescription) - throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); - } - } - @Rule public final transient TestPipeline p = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); @Rule public transient ExpectedLogs loggedBigQueryIO = ExpectedLogs.none(BigQueryIO.class); @Rule public transient ExpectedLogs loggedWriteRename = ExpectedLogs.none(WriteRename.class); @Rule public transient ExpectedLogs loggedWriteTables = ExpectedLogs.none(WriteTables.class); @Rule public transient TemporaryFolder testFolder = new TemporaryFolder(); - @Mock(extraInterfaces = Serializable.class) - public transient BigQueryServices.JobService mockJobService; @Mock private transient IOChannelFactory mockIOChannelFactory; @Mock(extraInterfaces = Serializable.class) private transient DatasetService mockDatasetService; @@ -801,7 +317,7 @@ public void testValidateReadSetsDefaultProject() throws Exception { @Test public void testBuildSourceWithTableAndFlatten() { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation("gs://testbucket/testdir"); Pipeline p = TestPipeline.create(bqOptions); @@ -819,7 +335,7 @@ public void testBuildSourceWithTableAndFlatten() { @Test public void testBuildSourceWithTableAndFlattenWithoutValidation() { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation("gs://testbucket/testdir"); Pipeline p = TestPipeline.create(bqOptions); @@ -838,7 +354,7 @@ public void testBuildSourceWithTableAndFlattenWithoutValidation() { @Test public void testBuildSourceWithTableAndSqlDialect() { BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation("gs://testbucket/testdir"); Pipeline p = TestPipeline.create(bqOptions); @@ -856,7 +372,7 @@ public void testBuildSourceWithTableAndSqlDialect() { @Test public void testReadFromTable() throws IOException, InterruptedException { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); Job job = new Job(); @@ -906,11 +422,11 @@ public Schema apply(Void input) { new WriteExtractFiles(schemaGenerator, records); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns(onStartJob, "done") - .pollJobReturns(job) - .getJobReturns((Job) null) - .verifyExecutingProject(bqOptions.getProject())) + .withJobService(new FakeJobService()) + // .startJobReturns(onStartJob, "done") + // .pollJobReturns(job) + // .getJobReturns((Job) null) + // .verifyExecutingProject(bqOptions.getProject())) .withDatasetService(fakeDatasetService) .readerReturns( toJsonString(new TableRow().set("name", "a").set("number", 1)), @@ -938,13 +454,16 @@ public void processElement(ProcessContext c) throws Exception { @Test public void testWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns("done", "done", "done") - .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)); + .withJobService(new FakeJobService()) + // .startJobReturns("done", "done", "done") + // .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)) + .withDatasetService(mockDatasetService); + + mockDatasetService.createDataset("defaultproject", "dataset-id", "", ""); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -969,7 +488,7 @@ public void testWrite() throws Exception { @Test public void testStreamingWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); FakeDatasetService datasetService = new FakeDatasetService(); @@ -1095,15 +614,27 @@ public PartitionedGlobalWindow decode(InputStream inStream, Context context) } @Test - public void testStreamingWriteWithWindowFn() throws Exception { + @Category(NeedsRunner.class) + public void testStreamingWriteWithDynamicTables() throws Exception { + testWriteWithDynamicTables(true); + } + + @Test + @Category(NeedsRunner.class) + public void testBatchWriteWithDynamicTables() throws Exception { + testWriteWithDynamicTables(false); + } + + public void testWriteWithDynamicTables(boolean streaming) throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); FakeDatasetService datasetService = new FakeDatasetService(); datasetService.createDataset("project-id", "dataset-id", "", ""); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withDatasetService(datasetService); + .withDatasetService(datasetService) + .withJobService(new FakeJobService()); List inserts = new ArrayList<>(); for (int i = 0; i < 10; i++) { @@ -1134,9 +665,11 @@ public TableDestination apply(ValueInSingleWindow input) { }; Pipeline p = TestPipeline.create(bqOptions); - p.apply(Create.of(inserts)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) - .apply(Window.into(window)) + PCollection input = p.apply(Create.of(inserts)); + if (streaming) { + input = input.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + } + input.apply(Window.into(window)) .apply(BigQueryIO.write() .to(tableFunction) .withFormatFunction(new SerializableFunction() { @@ -1179,13 +712,13 @@ public TableRow apply(Integer i) { @Test public void testWriteUnknown() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns("done", "done") - .pollJobReturns(Status.FAILED, Status.UNKNOWN)); + .withJobService(new FakeJobService()); + // .startJobReturns("done", "done") + // .pollJobReturns(Status.FAILED, Status.UNKNOWN)); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -1211,13 +744,13 @@ public void testWriteUnknown() throws Exception { @Test public void testWriteFailedJobs() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns("done", "done", "done") - .pollJobReturns(Status.FAILED, Status.FAILED, Status.FAILED)); + .withJobService(new FakeJobService()); + // .startJobReturns("done", "done", "done") + // .pollJobReturns(Status.FAILED, Status.FAILED, Status.FAILED)); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -1285,7 +818,7 @@ public void testTableSourcePrimitiveDisplayData() throws IOException, Interrupte .from("project:dataset.tableId") .withTestServices(new FakeBigQueryServices() .withDatasetService(mockDatasetService) - .withJobService(mockJobService)) + .withJobService(new FakeJobService())) .withoutValidation(); Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); @@ -1301,7 +834,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte .fromQuery("foobar") .withTestServices(new FakeBigQueryServices() .withDatasetService(mockDatasetService) - .withJobService(mockJobService)) + .withJobService(new FakeJobService())) .withoutValidation(); Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); @@ -1342,7 +875,7 @@ private void testWritePrimitiveDisplayData(boolean streaming) throws IOException .withSchema(new TableSchema().set("col1", "type1").set("col2", "type2")) .withTestServices(new FakeBigQueryServices() .withDatasetService(mockDatasetService) - .withJobService(mockJobService)) + .withJobService(new FakeJobService())) .withoutValidation(); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); @@ -1506,7 +1039,7 @@ private void testWriteValidatesDataset(boolean unbounded) throws Exception { options.setProject(projectId); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) + .withJobService(new FakeJobService()) .withDatasetService(mockDatasetService); when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow( new RuntimeException("Unable to confirm BigQuery dataset presence")); @@ -1674,7 +1207,7 @@ public void testWriteValidateFailsCreateNoSchema() { @Test public void testBigQueryTableSourceThroughJsonAPI() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) + .withJobService(new FakeJobService()) .readerReturns( toJsonString(new TableRow().set("name", "a").set("number", "1")), toJsonString(new TableRow().set("name", "b").set("number", "2")), @@ -1712,7 +1245,7 @@ public void testBigQueryTableSourceInitSplit() throws Exception { .setStatistics(jobStats); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) + .withJobService(new FakeJobService()) .withDatasetService(mockDatasetService) .readerReturns( toJsonString(new TableRow().set("name", "a").set("number", "1")), @@ -1731,8 +1264,6 @@ public void testBigQueryTableSourceInitSplit() throws Exception { new TableRow().set("name", "b").set("number", "2"), new TableRow().set("name", "c").set("number", "3")); - when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) - .thenReturn(extractJob); PipelineOptions options = PipelineOptionsFactory.create(); options.setTempLocation("mock://tempLocation"); @@ -1752,9 +1283,6 @@ public void testBigQueryTableSourceInitSplit() throws Exception { assertEquals(1, sources.size()); BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); - - Mockito.verify(mockJobService) - .startExtractJob(Mockito.any(), Mockito.any()); } @Test @@ -1777,8 +1305,9 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { extractJob.setStatus(new JobStatus()) .setStatistics(extractJobStats); + FakeJobService fakeJobService = new FakeJobService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) + .withJobService(fakeJobService) .withDatasetService(mockDatasetService) .readerReturns( toJsonString(new TableRow().set("name", "a").set("number", "1")), @@ -1803,23 +1332,29 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { options.setTempLocation(extractDestinationDir); TableReference queryTable = new TableReference() - .setProjectId("testProejct") + .setProjectId("testproject") .setDatasetId("testDataset") .setTableId("testTable"); - when(mockJobService.dryRunQuery(anyString(), Mockito.any())) - .thenReturn(new JobStatistics().setQuery( + // when(mockJobService.dryRunQuery(anyString(), Mockito.any())) + // .thenReturn(new JobStatistics().setQuery( + // new JobStatistics2() + // .setTotalBytesProcessed(100L) + // .setReferencedTables(ImmutableList.of(queryTable)))); + fakeJobService.expectDryRunQuery("testproject", "query", + new JobStatistics().setQuery( new JobStatistics2() .setTotalBytesProcessed(100L) .setReferencedTables(ImmutableList.of(queryTable)))); - when(mockDatasetService.getTable(eq(queryTable))) - .thenReturn(new Table().setSchema(new TableSchema())); - when(mockDatasetService.getTable(eq(destinationTable))) - .thenReturn(new Table().setSchema(new TableSchema())); + + // when(mockDatasetService.getTable(eq(queryTable))) + // .thenReturn(new Table().setSchema(new TableSchema())); + // when(mockDatasetService.getTable(eq(destinationTable))) + // .thenReturn(new Table().setSchema(new TableSchema())); IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true /* override */); when(mockIOChannelFactory.resolve(anyString(), anyString())) .thenReturn("mock://tempLocation/output"); - when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) - .thenReturn(extractJob); + //when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) + // .thenReturn(extractJob); Assert.assertThat( SourceTestUtils.readFromSource(bqSource, options), @@ -1832,6 +1367,7 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); + /* Mockito.verify(mockJobService) .startQueryJob( Mockito.any(), Mockito.any()); @@ -1843,7 +1379,7 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { ArgumentCaptor.forClass(JobConfigurationQuery.class); Mockito.verify(mockJobService).dryRunQuery(anyString(), queryConfigArg.capture()); assertEquals(true, queryConfigArg.getValue().getFlattenResults()); - assertEquals(true, queryConfigArg.getValue().getUseLegacySql()); + assertEquals(true, queryConfigArg.getValue().getUseLegacySql());*/ } @Test @@ -1867,7 +1403,7 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { .setStatistics(extractJobStats); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) + .withJobService(new FakeJobService()) .withDatasetService(mockDatasetService) .readerReturns( toJsonString(new TableRow().set("name", "a").set("number", "1")), @@ -1891,17 +1427,18 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); options.setTempLocation(extractDestinationDir); + /* when(mockJobService.dryRunQuery(anyString(), Mockito.any())) .thenReturn(new JobStatistics().setQuery( new JobStatistics2() .setTotalBytesProcessed(100L))); when(mockDatasetService.getTable(eq(destinationTable))) .thenReturn(new Table().setSchema(new TableSchema())); - IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true /* override */); + IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true); when(mockIOChannelFactory.resolve(anyString(), anyString())) .thenReturn("mock://tempLocation/output"); when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) - .thenReturn(extractJob); + .thenReturn(extractJob);*/ Assert.assertThat( SourceTestUtils.readFromSource(bqSource, options), @@ -1914,7 +1451,8 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); - Mockito.verify(mockJobService) + /* + Mockito.verify(Service) .startQueryJob( Mockito.any(), Mockito.any()); Mockito.verify(mockJobService) @@ -1925,7 +1463,7 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { ArgumentCaptor.forClass(JobConfigurationQuery.class); Mockito.verify(mockJobService).dryRunQuery(anyString(), queryConfigArg.capture()); assertEquals(true, queryConfigArg.getValue().getFlattenResults()); - assertEquals(true, queryConfigArg.getValue().getUseLegacySql()); + assertEquals(true, queryConfigArg.getValue().getUseLegacySql());*/ } @Test @@ -2028,7 +1566,7 @@ public void testWritePartitionEmptyData() throws Exception { // An empty file is created for no input data. One partition is needed. long expectedNumPartitions = 1; - testWritePartition(numFiles, fileSize, expectedNumPartitions); + testWritePartition(1, numFiles, fileSize, expectedNumPartitions); } @Test @@ -2038,7 +1576,7 @@ public void testWritePartitionSinglePartition() throws Exception { // One partition is needed. long expectedNumPartitions = 1; - testWritePartition(numFiles, fileSize, expectedNumPartitions); + testWritePartition(2, numFiles, fileSize, expectedNumPartitions); } @Test @@ -2048,7 +1586,7 @@ public void testWritePartitionManyFiles() throws Exception { // One partition is needed for each group of BigQueryWrite.MAX_NUM_FILES files. long expectedNumPartitions = 3; - testWritePartition(numFiles, fileSize, expectedNumPartitions); + testWritePartition(2, numFiles, fileSize, expectedNumPartitions); } @Test @@ -2058,69 +1596,103 @@ public void testWritePartitionLargeFileSize() throws Exception { // One partition is needed for each group of three files. long expectedNumPartitions = 4; - testWritePartition(numFiles, fileSize, expectedNumPartitions); + testWritePartition(2, numFiles, fileSize, expectedNumPartitions); } - private void testWritePartition(long numFiles, long fileSize, long expectedNumPartitions) + private void testWritePartition(long numTables, long numFilesPerTable, long fileSize, + long expectedNumPartitionsPerTable) throws Exception { p.enableAbandonedNodeEnforcement(false); - List expectedPartitionIds = Lists.newArrayList(); - for (long i = 1; i <= expectedNumPartitions; ++i) { - expectedPartitionIds.add(i); + List> expectedPartitions = Lists.newArrayList(); + 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)); + } } - List> files = Lists.newArrayList(); - List fileNames = Lists.newArrayList(); - for (int i = 0; i < numFiles; ++i) { - String fileName = String.format("files%05d", i); - fileNames.add(fileName); - files.add(KV.of(fileName, fileSize)); + 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); + if (filenames == null) { + filenames = Lists.newArrayList(); + filenamesPerTable.put(destination, 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)); + } } - 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") {}; PCollectionView> resultsView = PCollectionViews.iterableView( p, WindowingStrategy.globalDefault(), - KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); + WriteBundlesToFiles.ResultCoder.of()); + ValueProvider singletonTable = null; + if (numFilesPerTable == 0 && numTables == 1) { + TableReference singletonReference = new TableReference() + .setProjectId("projectid") + .setDatasetId("dataset") + .setTableId("table"); + singletonTable = StaticValueProvider.of(BigQueryHelpers.toJsonString(singletonReference)); + } WritePartition writePartition = - new WritePartition(null, null, resultsView, + new WritePartition(singletonTable, + "singleton", 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; - if (expectedNumPartitions > 1) { + List, List>> partitions; + if (expectedNumPartitionsPerTable > 1) { partitions = tester.takeOutputElements(multiPartitionsTag); } else { partitions = tester.takeOutputElements(singlePartitionTag); } - List partitionIds = Lists.newArrayList(); - List partitionFileNames = Lists.newArrayList(); - for (KV> partition : partitions) { - partitionIds.add(partition.getKey()); - for (String name : partition.getValue()) { - partitionFileNames.add(name); + + + List> partitionsResult = Lists.newArrayList(); + Map> filesPerTableResult = Maps.newHashMap(); + for (KV, List> partition : partitions) { + TableDestination table = partition.getKey().getKey(); + partitionsResult.add(partition.getKey()); + List tableFilesResult = filesPerTableResult.get(table); + if (tableFilesResult == null) { + tableFilesResult = Lists.newArrayList(); + filesPerTableResult.put(table, tableFilesResult); } + tableFilesResult.addAll(partition.getValue()); } - assertEquals(expectedPartitionIds, partitionIds); - if (numFiles == 0) { - assertThat(partitionFileNames, Matchers.hasSize(1)); - assertTrue(Files.exists(Paths.get(partitionFileNames.get(0)))); - assertThat(Files.readAllBytes(Paths.get(partitionFileNames.get(0))).length, + assertEquals(expectedPartitions.size(), partitionsResult.size()); + + // assertThat(partitionsResult, + // containsInAnyOrder(Iterables.toArray(expectedPartitions, ShardedKey.class))); + + if (numFilesPerTable == 0 && numTables == 1) { + assertEquals(1, filesPerTableResult.size()); + List singletonFiles = filesPerTableResult.values().iterator().next(); + assertTrue(Files.exists(Paths.get(singletonFiles.get(0)))); + assertThat(Files.readAllBytes(Paths.get(singletonFiles.get(0))).length, Matchers.equalTo(0)); } else { - assertEquals(fileNames, partitionFileNames); + assertEquals(filenamesPerTable, filesPerTableResult); } } @@ -2129,26 +1701,46 @@ public void testWriteTables() throws Exception { p.enableAbandonedNodeEnforcement(false); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns("done", "done", "done", "done") - .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED)); + .withJobService(new FakeJobService()) + // .startJobReturns("done", "done", "done", "done", "done", "done", "done", "done", + // "done", "done") + // .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, + // Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, + // Status.SUCCEEDED, Status.SUCCEEDED)) + .withDatasetService(mockDatasetService); + long numTables = 3; long numPartitions = 3; long numFilesPerPartition = 10; String jobIdToken = "jobIdToken"; String tempFilePrefix = "tempFilePrefix"; - String jsonTable = "{}"; - String jsonSchema = "{}"; - List expectedTempTables = Lists.newArrayList(); - - List>>> partitions = Lists.newArrayList(); - for (long i = 0; i < numPartitions; ++i) { - List filesPerPartition = Lists.newArrayList(); - for (int j = 0; j < numFilesPerPartition; ++j) { - filesPerPartition.add(String.format("files%05d", j)); + Map> expectedTempTables = Maps.newHashMap(); + + List, Iterable>>> 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); + for (int j = 0; j < numPartitions; ++j) { + String tempTableId = String.format( + jobIdToken + "_0x%08x_%05d", tableDestination.hashCode(), j); + List filesPerPartition = Lists.newArrayList(); + for (int k = 0; k < numFilesPerPartition; ++k) { + filesPerPartition.add(String.format("files0x%08x_%05d", tableDestination.hashCode(), k)); + } + partitions.add(KV.of(ShardedKey.of(tableDestination, j), + (Iterable>) Collections.singleton(filesPerPartition))); + + List expectedTables = expectedTempTables.get(tableDestination); + if (expectedTables == null) { + expectedTables = Lists.newArrayList(); + expectedTempTables.put(tableDestination, expectedTables); + } + String json = String.format( + "{\"datasetId\":\"dataset-id\",\"projectId\":\"project-id\",\"tableId\":\"%s\"}", + tempTableId); + expectedTables.add(json); } - partitions.add(KV.of(i, (Iterable>) Collections.singleton(filesPerPartition))); - expectedTempTables.add(String.format("{\"tableId\":\"%s_%05d\"}", jobIdToken, i)); } PCollection expectedTempTablesPCollection = p.apply(Create.of(expectedTempTables)); @@ -2165,27 +1757,33 @@ public void testWriteTables() throws Exception { fakeBqServices, jobIdTokenView, tempFilePrefix, - StaticValueProvider.of(jsonTable), - StaticValueProvider.of(jsonSchema), WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, null); - DoFnTester>>, String> tester = DoFnTester.of(writeTables); + DoFnTester, Iterable>>, + KV> tester = DoFnTester.of(writeTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); - for (KV>> partition : partitions) { + for (KV, Iterable>> partition : partitions) { tester.processElement(partition); } - List tempTables = tester.takeOutputElements(); - - assertEquals(expectedTempTables, tempTables); + Map> tempTablesResult = Maps.newHashMap(); + for (KV element : tester.takeOutputElements()) { + List tables = tempTablesResult.get(element.getKey()); + if (tables == null) { + tables = Lists.newArrayList(); + tempTablesResult.put(element.getKey(), tables); + } + tables.add(element.getValue()); + } + assertEquals(expectedTempTables, tempTablesResult); } @Test public void testRemoveTemporaryFiles() throws Exception { BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class); - bqOptions.setProject("defaultProject"); + bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); int numFiles = 10; @@ -2195,7 +1793,7 @@ public void testRemoveTemporaryFiles() throws Exception { for (int i = 0; i < numFiles; ++i) { String fileName = String.format("files%05d", i); writer.open(fileName); - fileNames.add(writer.close().getKey()); + fileNames.add(writer.close().filename); } fileNames.add(tempFilePrefix + String.format("files%05d", numFiles)); @@ -2217,23 +1815,33 @@ public void testWriteRename() throws Exception { p.enableAbandonedNodeEnforcement(false); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService() - .startJobReturns("done", "done") - .pollJobReturns(Status.FAILED, Status.SUCCEEDED)) + .withJobService(new FakeJobService()) + // .startJobReturns("done", "done") + // .pollJobReturns(Status.FAILED, Status.SUCCEEDED)) .withDatasetService(mockDatasetService); - long numTempTables = 3; + int numFinalTables = 3; + int numTempTables = 3; String jobIdToken = "jobIdToken"; String jsonTable = "{}"; - List tempTables = Lists.newArrayList(); - for (long i = 0; i < numTempTables; ++i) { - tempTables.add(String.format("{\"tableId\":\"%s_%05d\"}", jobIdToken, i)); + Map> tempTables = Maps.newHashMap(); + for (int i = 0; i < numFinalTables; ++i) { + String tableName = "project-id:dataset-id.table_" + i; + TableDestination tableDestination = new TableDestination(tableName, tableName); + List tables = Lists.newArrayList(); + tempTables.put(tableDestination, tables); + for (int j = 0; i < numTempTables; ++i) { + tables.add(String.format( + "{\"project-id:dataset-id.tableId\":\"%s_%05d_%05d\"}", jobIdToken, i, j)); + } } - PCollection tempTablesPCollection = p.apply(Create.of(tempTables)); - PCollectionView> tempTablesView = - PCollectionViews.iterableView( - tempTablesPCollection, WindowingStrategy.globalDefault(), StringUtf8Coder.of()); + PCollectionView>> tempTablesView = + PCollectionViews.multimapView( + p, + WindowingStrategy.globalDefault(), + KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of())); + PCollection jobIdTokenCollection = p.apply("CreateJobId", Create.of("jobId")); PCollectionView jobIdTokenView = jobIdTokenCollection.apply(View.asSingleton()); @@ -2241,11 +1849,9 @@ public void testWriteRename() throws Exception { WriteRename writeRename = new WriteRename( fakeBqServices, jobIdTokenView, - StaticValueProvider.of(jsonTable), WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, - tempTablesView, - null); + tempTablesView); DoFnTester tester = DoFnTester.of(writeRename); tester.setSideInput(tempTablesView, GlobalWindow.INSTANCE, tempTables); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java new file mode 100644 index 000000000000..ed3ab37d1f37 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java @@ -0,0 +1,96 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.fromJsonString; +import static org.junit.Assert.assertEquals; + +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.beam.sdk.options.BigQueryOptions; + + +/** + * Created by relax on 3/30/17. + */ +class FakeBigQueryServices implements BigQueryServices { + private String[] jsonTableRowReturns = new String[0]; + private JobService jobService; + private DatasetService datasetService; + + public FakeBigQueryServices withJobService(JobService jobService) { + this.jobService = jobService; + return this; + } + + public FakeBigQueryServices withDatasetService(DatasetService datasetService) { + this.datasetService = datasetService; + return this; + } + + public FakeBigQueryServices readerReturns(String... jsonTableRowReturns) { + this.jsonTableRowReturns = jsonTableRowReturns; + return this; + } + + @Override + public JobService getJobService(BigQueryOptions bqOptions) { + return jobService; + } + + @Override + public DatasetService getDatasetService(BigQueryOptions bqOptions) { + return datasetService; + } + + @Override + public BigQueryJsonReader getReaderFromTable( + BigQueryOptions bqOptions, TableReference tableRef) { + return new FakeBigQueryReader(jsonTableRowReturns); + } + + @Override + public BigQueryJsonReader getReaderFromQuery( + BigQueryOptions bqOptions, String projectId, JobConfigurationQuery queryConfig) { + return new FakeBigQueryReader(jsonTableRowReturns); + } + + private static class FakeBigQueryReader implements BigQueryJsonReader { + private static final int UNSTARTED = -1; + private static final int CLOSED = Integer.MAX_VALUE; + + private String[] jsonTableRowReturns; + private int currIndex; + + FakeBigQueryReader(String[] jsonTableRowReturns) { + this.jsonTableRowReturns = jsonTableRowReturns; + this.currIndex = UNSTARTED; + } + + @Override + public boolean start() throws IOException { + assertEquals(UNSTARTED, currIndex); + currIndex = 0; + return currIndex < jsonTableRowReturns.length; + } + + @Override + public boolean advance() throws IOException { + return ++currIndex < jsonTableRowReturns.length; + } + + @Override + public TableRow getCurrent() throws NoSuchElementException { + if (currIndex >= jsonTableRowReturns.length) { + throw new NoSuchElementException(); + } + return fromJsonString(jsonTableRowReturns[currIndex], TableRow.class); + } + + @Override + public void close() throws IOException { + currIndex = CLOSED; + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java new file mode 100644 index 000000000000..9b2cf631e1bf --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java @@ -0,0 +1,172 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; + +import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; + +/** A fake dataset service that can be serialized, for use in testReadFromTable. */ +class FakeDatasetService implements DatasetService, Serializable { + @Override + public Table getTable(TableReference tableRef) + throws InterruptedException, IOException { + synchronized (BigQueryIOTest.tables) { + Map dataset = + checkNotNull( + BigQueryIOTest.tables.get(tableRef.getProjectId(), tableRef.getDatasetId()), + "Tried to get a dataset %s:%s from %s, but no such dataset was set", + tableRef.getProjectId(), + tableRef.getDatasetId(), + tableRef.getTableId(), + FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = dataset.get(tableRef.getTableId()); + return tableContainer == null ? null : tableContainer.getTable(); + } + } + + List getAllRows(String projectId, String datasetId, String tableId) + throws InterruptedException, IOException { + synchronized (BigQueryIOTest.tables) { + return getTableContainer(projectId, datasetId, tableId).getRows(); + } + } + + private TableContainer getTableContainer(String projectId, String datasetId, String tableId) + throws InterruptedException, IOException { + synchronized (BigQueryIOTest.tables) { + Map dataset = + checkNotNull( + BigQueryIOTest.tables.get(projectId, datasetId), + "Tried to get a dataset %s:%s from %s, but no such dataset was set", + projectId, + datasetId, + FakeDatasetService.class.getSimpleName()); + return checkNotNull(dataset.get(tableId), + "Tried to get a table %s:%s.%s from %s, but no such table was set", + projectId, + datasetId, + tableId, + FakeDatasetService.class.getSimpleName()); + } + } + + @Override + public void deleteTable(TableReference tableRef) throws IOException, InterruptedException { + throw new UnsupportedOperationException("Unsupported"); + } + + + @Override + public void createTable(Table table) throws IOException { + TableReference tableReference = table.getTableReference(); + synchronized (BigQueryIOTest.tables) { + Map dataset = + checkNotNull( + BigQueryIOTest.tables.get(tableReference.getProjectId(), + tableReference.getDatasetId()), + "Tried to get a dataset %s:%s from %s, but no such table was set", + tableReference.getProjectId(), + tableReference.getDatasetId(), + FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = dataset.get(tableReference.getTableId()); + if (tableContainer == null) { + tableContainer = new TableContainer(table); + dataset.put(tableReference.getTableId(), tableContainer); + } + } + } + + @Override + public boolean isTableEmpty(TableReference tableRef) + throws IOException, InterruptedException { + Long numBytes = getTable(tableRef).getNumBytes(); + return numBytes == null || numBytes == 0L; + } + + @Override + public Dataset getDataset( + String projectId, String datasetId) throws IOException, InterruptedException { + throw new UnsupportedOperationException("Unsupported"); + } + + @Override + public void createDataset( + String projectId, String datasetId, String location, String description) + throws IOException, InterruptedException { + synchronized (BigQueryIOTest.tables) { + Map dataset = BigQueryIOTest.tables.get(projectId, datasetId); + if (dataset == null) { + dataset = new HashMap<>(); + BigQueryIOTest.tables.put(projectId, datasetId, dataset); + } + } + } + + @Override + public void deleteDataset(String projectId, String datasetId) + throws IOException, InterruptedException { + throw new UnsupportedOperationException("Unsupported"); + } + + @Override + public long insertAll( + TableReference ref, List rowList, @Nullable List insertIdList) + throws IOException, InterruptedException { + synchronized (BigQueryIOTest.tables) { + if (insertIdList != null) { + assertEquals(rowList.size(), insertIdList.size()); + } else { + insertIdList = Lists.newArrayListWithExpectedSize(rowList.size()); + for (int i = 0; i < rowList.size(); ++i) { + insertIdList.add(Integer.toString(ThreadLocalRandom.current().nextInt())); + } + } + + long dataSize = 0; + TableContainer tableContainer = getTableContainer( + ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + for (int i = 0; i < rowList.size(); ++i) { + tableContainer.addRow(rowList.get(i), insertIdList.get(i)); + dataSize += rowList.get(i).toString().length(); + } + return dataSize; + } + } + + @Override + public Table patchTableDescription(TableReference tableReference, + @Nullable String tableDescription) + throws IOException, InterruptedException { + synchronized (BigQueryIOTest.tables) { + Map dataset = + checkNotNull( + BigQueryIOTest.tables.get(tableReference.getProjectId(), + tableReference.getDatasetId()), + "Tried to get a dataset %s:%s from %s, but no such dataset was set", + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId(), + FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = checkNotNull(dataset.get(tableReference.getTableId()), + "Tried to patch a table %s:%s.%s from %s, but no such table was set", + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId(), + FakeDatasetService.class.getSimpleName()); + tableContainer.getTable().setDescription(tableDescription); + return tableContainer.getTable(); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java new file mode 100644 index 000000000000..3c67c3da5288 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -0,0 +1,273 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import com.google.api.client.json.JsonFactory; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfiguration; +import com.google.api.services.bigquery.model.JobConfigurationExtract; +import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobConfigurationTableCopy; +import com.google.api.services.bigquery.model.JobReference; +import com.google.api.services.bigquery.model.JobStatistics; +import com.google.api.services.bigquery.model.JobStatistics4; +import com.google.api.services.bigquery.model.JobStatus; +import com.google.api.services.bigquery.model.Table; +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.collect.HashBasedTable; +import com.google.common.collect.Lists; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.List; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.TableRowJsonCoder; +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.BigQueryServices.JobService; +import org.apache.beam.sdk.util.FluentBackoff; + +import org.apache.beam.sdk.util.Transport; +import org.joda.time.Duration; + +/** + */ +class FakeJobService implements JobService, Serializable { + static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); + + // Whenever a job is started, the first 5 calls to GetJob will report the job as pending, + // the next 5 will return the job as running, and only then will the job report as done. + private static final int GET_JOBS_TRANSITION_INTERVAL = 5; + + private FakeDatasetService datasetService; + + private static class JobInfo { + Job job; + int getJobCount = 0; + + JobInfo(Job job) { + this.job = job; + } + } + + private static final com.google.common.collect.Table allJobs = + HashBasedTable.create(); + + private static final com.google.common.collect.Table + dryRunQueryResults = HashBasedTable.create(); + + FakeJobService() { + this.datasetService = new FakeDatasetService(); + } + + @Override + public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig) + throws InterruptedException, IOException { + synchronized (allJobs) { + Job job = new Job(); + job.setJobReference(jobRef); + job.setConfiguration(new JobConfiguration().setLoad(loadConfig)); + job.setKind(" bigquery#job"); + job.setStatus(new JobStatus().setState("PENDING")); + allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); + } + } + + @Override + public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig) + throws InterruptedException, IOException { + checkArgument(extractConfig.getDestinationFormat().equals("AVRO"), + "Only extract to AVRO is supported"); + checkArgument(extractConfig.getDestinationUris().size() == 1, + "Must specify exactly one destination URI."); + synchronized (allJobs) { + Job job = new Job(); + job.setJobReference(jobRef); + job.setConfiguration(new JobConfiguration().setExtract(extractConfig)); + job.setKind(" bigquery#job"); + job.setStatus(new JobStatus().setState("PENDING")); + allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); + } + } + + @Override + public void startQueryJob(JobReference jobRef, JobConfigurationQuery query) + throws IOException, InterruptedException { + } + + @Override + public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) + throws IOException, InterruptedException { + synchronized (allJobs) { + Job job = new Job(); + job.setJobReference(jobRef); + job.setConfiguration(new JobConfiguration().setCopy(copyConfig)); + job.setKind(" bigquery#job"); + job.setStatus(new JobStatus().setState("PENDING")); + allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); + } + } + + @Override + public Job pollJob(JobReference jobRef, int maxAttempts) + throws InterruptedException { + BackOff backoff = + FluentBackoff.DEFAULT + .withMaxRetries(maxAttempts) + .withInitialBackoff(Duration.millis(50)) + .withMaxBackoff(Duration.standardMinutes(1)) + .backoff(); + Sleeper sleeper = Sleeper.DEFAULT; + try { + do { + Job job = getJob(jobRef); + if (job != null) { + JobStatus status = job.getStatus(); + if (status != null && status.getState() != null && status.getState().equals("DONE")) { + return job; + } + } + } while (BackOffUtils.next(sleeper, backoff)); + } catch (IOException e) { + return null; + } + return null; + } + + public void expectDryRunQuery(String projectId, String query, JobStatistics result) { + synchronized (dryRunQueryResults) { + dryRunQueryResults.put(projectId, query, result); + } + } + + @Override + public JobStatistics dryRunQuery(String projectId, JobConfigurationQuery query) + throws InterruptedException, IOException { + synchronized (dryRunQueryResults) { + JobStatistics result = dryRunQueryResults.get(projectId, query.getQuery()); + if (result != null) { + return result; + } + } + throw new UnsupportedOperationException(); + } + + @Override + public Job getJob(JobReference jobRef) throws InterruptedException { + try { + synchronized (allJobs) { + JobInfo job = allJobs.get(jobRef.getProjectId(), jobRef.getJobId()); + if (job == null) { + return null; + } + ++job.getJobCount; + if (job.getJobCount == GET_JOBS_TRANSITION_INTERVAL + 1) { + job.job.getStatus().setState("RUNNING"); + } else if (job.getJobCount == 2 * GET_JOBS_TRANSITION_INTERVAL + 1) { + runJob(job.job); + job.job.getStatus().setState("DONE"); + } + return JSON_FACTORY.fromString(JSON_FACTORY.toString(job.job), Job.class); + } + } catch (IOException e) { + return null; + } + } + + private void runJob(Job job) throws InterruptedException, IOException { + if (job.getConfiguration().getLoad() != null) { + runLoadJob(job.getConfiguration().getLoad()); + } else if (job.getConfiguration().getCopy() != null) { + runCopyJob(job.getConfiguration().getCopy()); + } else if (job.getConfiguration().getExtract() != null) { + runExtractJob(job, job.getConfiguration().getExtract()); + } + } + + private void validateDispositions(Table table, CreateDisposition createDisposition, + WriteDisposition writeDisposition) + throws InterruptedException, IOException { + if (table == null) { + checkState(createDisposition != CreateDisposition.CREATE_NEVER, + "CreateDisposition == CREATE_NEVER but the table doesn't exist."); + } else if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) { + datasetService.deleteTable(table.getTableReference()); + } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) { + List allRows = datasetService.getAllRows(table.getTableReference().getProjectId(), + table.getTableReference().getDatasetId(), table.getTableReference().getTableId()); + checkState(allRows.isEmpty(), "Write disposition was set to WRITE_EMPTY," + + " but the table was not empty."); + } + } + private void runLoadJob(JobConfigurationLoad load) + throws InterruptedException, IOException { + TableReference destination = load.getDestinationTable(); + TableSchema schema = load.getSchema(); + List sourceFiles = load.getSourceUris(); + WriteDisposition writeDisposition = WriteDisposition.valueOf(load.getWriteDisposition()); + CreateDisposition createDisposition = CreateDisposition.valueOf(load.getCreateDisposition()); + checkArgument(load.getSourceFormat().equals("NEWLINE_DELIMITED_JSON")); + Table existingTable = datasetService.getTable(destination); + validateDispositions(existingTable, createDisposition, writeDisposition); + + datasetService.createTable(new Table().setTableReference(destination).setSchema(schema)); + + List rows = Lists.newArrayList(); + for (String filename : sourceFiles) { + rows.addAll(readRows(filename)); + } + datasetService.insertAll(destination, rows, null); + } + + private void runCopyJob(JobConfigurationTableCopy copy) + throws InterruptedException, IOException { + List sources = copy.getSourceTables(); + TableReference destination = copy.getDestinationTable(); + WriteDisposition writeDisposition = WriteDisposition.valueOf(copy.getWriteDisposition()); + CreateDisposition createDisposition = CreateDisposition.valueOf(copy.getCreateDisposition()); + Table existingTable = datasetService.getTable(destination); + validateDispositions(existingTable, createDisposition, writeDisposition); + + List allRows = Lists.newArrayList(); + for (TableReference source : sources) { + allRows.addAll(datasetService.getAllRows( + source.getProjectId(), source.getDatasetId(), source.getTableId())); + } + datasetService.insertAll(destination, allRows, null); + } + + private void runExtractJob(Job job, JobConfigurationExtract extract) { + TableReference sourceTable = extract.getSourceTable(); + extract.getDestinationUris().get(0); + List destinationFileCounts = Lists.newArrayList(0L); + job.setStatistics(new JobStatistics().setExtract( + new JobStatistics4().setDestinationUriFileCounts(destinationFileCounts))); + } + + private List readRows(String filename) throws IOException { + Coder coder = TableRowJsonCoder.of(); + List tableRows = Lists.newArrayList(); + try (BufferedReader reader = new BufferedReader(new FileReader(filename))) { + String line; + while ((line = reader.readLine()) != null) { + TableRow tableRow = coder.decode( + new ByteArrayInputStream(line.getBytes(StandardCharsets.UTF_8)), Context.OUTER); + tableRows.add(tableRow); + } + } + return tableRows; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java new file mode 100644 index 000000000000..b2fc1706e20a --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java @@ -0,0 +1,36 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableRow; + +import java.util.ArrayList; +import java.util.List; + +/** + * Created by relax on 3/30/17. + */ +class TableContainer { + Table table; + List rows; + List ids; + + TableContainer(Table table) { + this.table = table; + this.rows = new ArrayList<>(); + this.ids = new ArrayList<>(); + } + + TableContainer addRow(TableRow row, String id) { + rows.add(row); + ids.add(id); + return this; + } + + Table getTable() { + return table; + } + + List getRows() { + return rows; + } +} From f8a62af892375ab5a5d884443abde723e53dae7d Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Fri, 31 Mar 2017 14:16:48 -0700 Subject: [PATCH 6/7] Fix tests to properly fake out BigQueryService, and add tests for dynamic-table functionality. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 7 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 15 +- .../beam/sdk/io/gcp/bigquery/ShardedKey.java | 2 +- .../sdk/io/gcp/bigquery/StreamingInserts.java | 5 +- .../sdk/io/gcp/bigquery/StreamingWriteFn.java | 1 - .../sdk/io/gcp/bigquery/TableDestination.java | 3 +- .../sdk/io/gcp/bigquery/TableRowWriter.java | 3 +- .../sdk/io/gcp/bigquery/TagWithUniqueIds.java | 9 - .../io/gcp/bigquery/WriteBundlesToFiles.java | 12 +- .../sdk/io/gcp/bigquery/WritePartition.java | 13 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 613 +++++++++--------- .../io/gcp/bigquery/FakeBigQueryServices.java | 114 +++- .../io/gcp/bigquery/FakeDatasetService.java | 138 ++-- .../sdk/io/gcp/bigquery/FakeJobService.java | 182 +++++- .../sdk/io/gcp/bigquery/TableContainer.java | 33 +- 15 files changed, 703 insertions(+), 447 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 5e80fae778aa..06fdfcebdc0f 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 @@ -58,9 +58,8 @@ /** * PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ -class BatchLoads extends - PTransform>, WriteResult> { - BigQueryIO.Write write; +class BatchLoads extends PTransform>, WriteResult> { + BigQueryIO.Write write; private static class ConstantSchemaFunction implements SerializableFunction { @@ -79,7 +78,7 @@ public TableSchema apply(TableDestination table) { } } - BatchLoads(BigQueryIO.Write write) { + BatchLoads(BigQueryIO.Write write) { this.write = write; } 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 f1baaf736808..54a25c70e177 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 @@ -64,7 +64,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -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.Transport; @@ -536,7 +535,7 @@ void cleanup(PipelineOptions options) throws Exception { } } if (extractFiles != null && !extractFiles.isEmpty()) { - new GcsUtilFactory().create(options).remove(extractFiles); + IOChannelUtils.getFactory(extractFiles.iterator().next()).remove(extractFiles); } } }; @@ -701,8 +700,8 @@ public abstract static class Write extends PTransform, WriteRe @AutoValue.Builder abstract static class Builder { abstract Builder setJsonTableRef(ValueProvider jsonTableRef); - abstract Builder setTableRefFunction( - SerializableFunction, TableReference> tableRefFunction); + abstract Builder setTableFunction( + SerializableFunction, TableDestination> tableFunction); abstract Builder setFormatFunction( SerializableFunction formatFunction); abstract Builder setJsonSchema(ValueProvider jsonSchema); @@ -823,8 +822,7 @@ public Write to(ValueProvider tableSpec) { * {@link ValueInSingleWindow}, so can be determined by the value or by the window. */ public Write to( - SerializableFunction, String> tableSpecFunction) { - return toTableReference(new TranslateTableSpecFunction(tableSpecFunction)); + SerializableFunction, TableDestination> tableFunction) { ensureToNotCalledYet(); return toBuilder().setTableFunction(tableFunction).build(); } @@ -834,7 +832,7 @@ public Write to( * {@link TableReference} instead of a string table specification. */ private Write toTableReference( - SerializableFunction, TableReference> tableRefFunction) { + SerializableFunction, TableDestination> tableFunction) { ensureToNotCalledYet(); return toBuilder().setTableFunction(tableFunction).build(); } @@ -984,8 +982,7 @@ public WriteResult expand(PCollection input) { if (input.isBounded() == IsBounded.UNBOUNDED) { return rowsWithDestination.apply(new StreamingInserts(this)); } else { - - return rowsWithDestination.apply(new BatchLoads(this)); + return rowsWithDestination.apply(new BatchLoads(this)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java index ab57446028a4..09b4fbf01191 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java @@ -56,7 +56,7 @@ public boolean equals(Object o) { return false; } ShardedKey other = (ShardedKey) o; - return (key == other.key) && (shardNumber == other.shardNumber); + return Objects.equals(key, other.key) && Objects.equals(shardNumber, other.shardNumber); } @Override 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 37afbdfeeb11..ced1d662cdf5 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 @@ -38,9 +38,8 @@ * PTransform that performs streaming BigQuery write. To increase consistency, * it leverages BigQuery best effort de-dup mechanism. */ - -class StreamingInserts - extends PTransform>, WriteResult> { +class StreamingInserts extends PTransform>, + WriteResult> { private final Write write; private static class ConstantSchemaFunction implements diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java index 83ed3d2a9453..22b2078962a8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java @@ -98,7 +98,6 @@ public void populateDisplayData(DisplayData.Builder builder) { private void flushRows(TableReference tableReference, List tableRows, List uniqueIds, BigQueryOptions options) throws InterruptedException { - System.out.println("FlUSHING ROWS " + tableRows.size()); if (!tableRows.isEmpty()) { try { long totalBytes = bqServices.getDatasetService(options).insertAll( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index e8538e03412e..36e14019bda8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -64,7 +64,8 @@ public boolean equals(Object o) { return false; } TableDestination other = (TableDestination) o; - return (tableSpec == other.tableSpec) && (tableDescription == other.tableDescription); + return Objects.equals(this.tableSpec, other.tableSpec) + && Objects.equals(this.tableDescription, other.tableDescription); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java index a1f615327d73..ee8f46696865 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.values.KV; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +56,7 @@ public Result(String filename, long byteSize) { } } TableRowWriter(String basename) { - this.tempFilePrefix = basename; + this.tempFilePrefix = basename; } public final void open(String uId) throws Exception { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java index 6f0186e73989..737978483327 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java @@ -18,23 +18,14 @@ 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.annotations.VisibleForTesting; -import com.google.common.base.Strings; import java.io.IOException; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToTableSpec; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.ValueInSingleWindow; /** * Fn that tags each table row with a unique id and destination table. 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 b8069f6614e0..869e68a069fa 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 @@ -19,19 +19,16 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; - +import com.google.common.collect.Maps; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; import java.util.Map; import java.util.UUID; - -import com.google.common.collect.Maps; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -50,6 +47,10 @@ class WriteBundlesToFiles extends DoFn, WriteBund 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 class Result implements Serializable { public String filename; public Long fileByteSize; @@ -62,6 +63,9 @@ public Result(String filename, Long fileByteSize, TableDestination tableDestinat } } + /** + * a coder for the {@link Result} class. + */ public static class ResultCoder extends AtomicCoder { private static final ResultCoder INSTANCE = new ResultCoder(); 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 c48955b597be..9c48b8233aea 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 @@ -89,8 +89,8 @@ public void processElement(ProcessContext c) throws Exception { partitions.add(Lists.newArrayList()); currResultsMap.put(tableDestination, partitions); } - int currNumFiles = currNumFilesMap.getOrDefault(tableDestination, 0); - long currSizeBytes = currSizeBytesMap.getOrDefault(tableDestination, 0L); + int currNumFiles = getOrDefault(currNumFilesMap, tableDestination, 0); + long currSizeBytes = getOrDefault(currSizeBytesMap, tableDestination, 0L); if (currNumFiles + 1 > Write.MAX_NUM_FILES || currSizeBytes + fileResult.fileByteSize > Write.MAX_SIZE_BYTES) { // Add a new partition for this table. @@ -117,4 +117,13 @@ public void processElement(ProcessContext c) throws Exception { } } } + + private T getOrDefault(Map map, TableDestination tableDestination, + T defaultValue) { + if (map.containsKey(tableDestination)) { + return map.get(tableDestination); + } else { + return defaultValue; + } + } } 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 d1ef8e24b8a4..f10be13c85e1 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 @@ -18,6 +18,8 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +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.containsInAnyOrder; @@ -26,17 +28,9 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.when; - -import com.google.api.client.json.GenericJson; + import com.google.api.client.util.Data; import com.google.api.services.bigquery.model.Job; -import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatistics; import com.google.api.services.bigquery.model.JobStatistics2; import com.google.api.services.bigquery.model.JobStatistics4; @@ -48,7 +42,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; @@ -58,9 +52,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.math.BigDecimal; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; @@ -69,14 +66,10 @@ import java.util.Map; import java.util.Set; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -88,7 +81,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; 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.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.options.BigQueryOptions; @@ -122,7 +114,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.PCollectionViews; @@ -140,6 +131,7 @@ import org.joda.time.Instant; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -147,10 +139,6 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.ArgumentCaptor; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.MockitoAnnotations; /** * Tests for BigQueryIO. @@ -158,6 +146,8 @@ @RunWith(JUnit4.class) public class BigQueryIOTest implements Serializable { + private static Path tempFolder; + // Table information must be static, as each ParDo will get a separate instance of // FakeDatasetServices, and they must all modify the same storage. static com.google.common.collect.Table> @@ -169,8 +159,6 @@ public class BigQueryIOTest implements Serializable { @Rule public transient ExpectedLogs loggedWriteRename = ExpectedLogs.none(WriteRename.class); @Rule public transient ExpectedLogs loggedWriteTables = ExpectedLogs.none(WriteTables.class); @Rule public transient TemporaryFolder testFolder = new TemporaryFolder(); - @Mock private transient IOChannelFactory mockIOChannelFactory; - @Mock(extraInterfaces = Serializable.class) private transient DatasetService mockDatasetService; private void checkReadTableObject( BigQueryIO.Read read, String project, String dataset, String table) { @@ -227,9 +215,13 @@ private void checkWriteObjectWithValidate( assertEquals(validate, write.getValidate()); } + @BeforeClass + public static void setupClass() throws IOException { + tempFolder = Files.createTempDirectory("BigQueryIOTest"); + } + @Before public void setUp() throws IOException { - MockitoAnnotations.initMocks(this); tables = HashBasedTable.create(); BigQueryIO.clearCreatedTables(); } @@ -289,29 +281,53 @@ public void testValidateReadSetsDefaultProject() throws Exception { String tableId = "sometable"; BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject(projectId); - bqOptions.setTempLocation("gs://testbucket/testdir"); + + Path baseDir = Files.createTempDirectory(tempFolder, "testValidateReadSetsDefaultProject"); + bqOptions.setTempLocation(baseDir.toString()); FakeDatasetService fakeDatasetService = new FakeDatasetService(); fakeDatasetService.createDataset(projectId, datasetId, "", ""); TableReference tableReference = new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId); - fakeDatasetService.createTable(new Table().setTableReference(tableReference)); + fakeDatasetService.createTable(new Table() + .setTableReference(tableReference) + .setSchema(new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER"))))); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) .withDatasetService(fakeDatasetService); + List expected = ImmutableList.of( + new TableRow().set("name", "a").set("number", 1L), + new TableRow().set("name", "b").set("number", 2L), + new TableRow().set("name", "c").set("number", 3L), + new TableRow().set("name", "d").set("number", 4L), + new TableRow().set("name", "e").set("number", 5L), + new TableRow().set("name", "f").set("number", 6L)); + fakeDatasetService.insertAll(tableReference, expected, null); + Pipeline p = TestPipeline.create(bqOptions); TableReference tableRef = new TableReference(); tableRef.setDatasetId(datasetId); tableRef.setTableId(tableId); - thrown.expect(RuntimeException.class); - // Message will be one of following depending on the execution environment. - thrown.expectMessage(Matchers.containsString("Unsupported")); - p.apply(BigQueryIO.read().from(tableRef) - .withTestServices(fakeBqServices)); + PCollection> output = + p.apply(BigQueryIO.read().from(tableRef).withTestServices(fakeBqServices)) + .apply(ParDo.of(new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + c.output(KV.of((String) c.element().get("name"), + Long.valueOf((String) c.element().get("number")))); + } + })); + PAssert.that(output).containsInAnyOrder(ImmutableList.of(KV.of("a", 1L), KV.of("b", 2L), + KV.of("c", 3L), KV.of("d", 4L), KV.of("e", 5L), KV.of("f", 6L))); + p.run(); } @Test @@ -400,54 +416,32 @@ public void testReadFromTable() throws IOException, InterruptedException { FakeDatasetService fakeDatasetService = new FakeDatasetService(); fakeDatasetService.createDataset("non-executing-project", "somedataset", "", ""); fakeDatasetService.createTable(sometable); - SerializableFunction schemaGenerator = - new SerializableFunction() { - @Override - public Schema apply(Void input) { - return BigQueryAvroUtils.toGenericAvroSchema( - "sometable", - ImmutableList.of( - new TableFieldSchema().setName("name").setType("STRING"), - new TableFieldSchema().setName("number").setType("INTEGER"))); - } - }; - Collection> records = - ImmutableList.>builder() - .add(ImmutableMap.builder().put("name", "a").put("number", 1L).build()) - .add(ImmutableMap.builder().put("name", "b").put("number", 2L).build()) - .add(ImmutableMap.builder().put("name", "c").put("number", 3L).build()) - .build(); - SerializableFunction onStartJob = - new WriteExtractFiles(schemaGenerator, records); + List records = Lists.newArrayList( + new TableRow().set("name", "a").set("number", 1L), + new TableRow().set("name", "b").set("number", 2L), + new TableRow().set("name", "c").set("number", 3L)); + fakeDatasetService.insertAll(sometable.getTableReference(), records, null); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - // .startJobReturns(onStartJob, "done") - // .pollJobReturns(job) - // .getJobReturns((Job) null) - // .verifyExecutingProject(bqOptions.getProject())) - .withDatasetService(fakeDatasetService) - .readerReturns( - toJsonString(new TableRow().set("name", "a").set("number", 1)), - toJsonString(new TableRow().set("name", "b").set("number", 2)), - toJsonString(new TableRow().set("name", "c").set("number", 3))); + .withDatasetService(fakeDatasetService); Pipeline p = TestPipeline.create(bqOptions); - PCollection output = p + PCollection> output = p .apply(BigQueryIO.read().from("non-executing-project:somedataset.sometable") .withTestServices(fakeBqServices) .withoutValidation()) - .apply(ParDo.of(new DoFn() { + .apply(ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) throws Exception { - c.output((String) c.element().get("name")); + c.output(KV.of((String) c.element().get("name"), + Long.valueOf((String) c.element().get("number")))); } })); PAssert.that(output) - .containsInAnyOrder(ImmutableList.of("a", "b", "c")); - + .containsInAnyOrder(ImmutableList.of(KV.of("a", 1L), KV.of("b", 2L), KV.of("c", 3L))); p.run(); } @@ -457,13 +451,12 @@ public void testWrite() throws Exception { bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - // .startJobReturns("done", "done", "done") - // .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .withDatasetService(datasetService); - mockDatasetService.createDataset("defaultproject", "dataset-id", "", ""); + datasetService.createDataset("defaultproject", "dataset-id", "", ""); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -715,11 +708,11 @@ public void testWriteUnknown() throws Exception { bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService()); - // .startJobReturns("done", "done") - // .pollJobReturns(Status.FAILED, Status.UNKNOWN)); - + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); + datasetService.createDataset("project-id", "dataset-id", "", ""); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( new TableRow().set("name", "a").set("number", 1), @@ -732,7 +725,7 @@ public void testWriteUnknown() throws Exception { .withoutValidation()); thrown.expect(RuntimeException.class); - thrown.expectMessage("UNKNOWN status of load job"); + thrown.expectMessage("Failed to create load job"); try { p.run(); } finally { @@ -747,10 +740,10 @@ public void testWriteFailedJobs() throws Exception { bqOptions.setProject("defaultproject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService()); - // .startJobReturns("done", "done", "done") - // .pollJobReturns(Status.FAILED, Status.FAILED, Status.FAILED)); + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -817,7 +810,7 @@ public void testTableSourcePrimitiveDisplayData() throws IOException, Interrupte BigQueryIO.Read read = BigQueryIO.read() .from("project:dataset.tableId") .withTestServices(new FakeBigQueryServices() - .withDatasetService(mockDatasetService) + .withDatasetService(new FakeDatasetService()) .withJobService(new FakeJobService())) .withoutValidation(); @@ -833,7 +826,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte BigQueryIO.Read read = BigQueryIO.read() .fromQuery("foobar") .withTestServices(new FakeBigQueryServices() - .withDatasetService(mockDatasetService) + .withDatasetService(new FakeDatasetService()) .withJobService(new FakeJobService())) .withoutValidation(); @@ -874,7 +867,7 @@ private void testWritePrimitiveDisplayData(boolean streaming) throws IOException .to("project:dataset.table") .withSchema(new TableSchema().set("col1", "type1").set("col2", "type2")) .withTestServices(new FakeBigQueryServices() - .withDatasetService(mockDatasetService) + .withDatasetService(new FakeDatasetService()) .withJobService(new FakeJobService())) .withoutValidation(); @@ -1040,9 +1033,7 @@ private void testWriteValidatesDataset(boolean unbounded) throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - .withDatasetService(mockDatasetService); - when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow( - new RuntimeException("Unable to confirm BigQuery dataset presence")); + .withDatasetService(new FakeDatasetService()); Pipeline p = TestPipeline.create(options); @@ -1206,26 +1197,31 @@ public void testWriteValidateFailsCreateNoSchema() { @Test public void testBigQueryTableSourceThroughJsonAPI() throws Exception { + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - .readerReturns( - toJsonString(new TableRow().set("name", "a").set("number", "1")), - toJsonString(new TableRow().set("name", "b").set("number", "2")), - toJsonString(new TableRow().set("name", "c").set("number", "3"))); + .withDatasetService(datasetService); + List expected = ImmutableList.of( + new TableRow().set("name", "a").set("number", "1"), + new TableRow().set("name", "b").set("number", "2"), + new TableRow().set("name", "c").set("number", "3"), + new TableRow().set("name", "d").set("number", "4"), + new TableRow().set("name", "e").set("number", "5"), + new TableRow().set("name", "f").set("number", "6")); + + TableReference table = BigQueryHelpers.parseTableSpec("project:data_set.table_name"); + datasetService.createDataset(table.getProjectId(), table.getDatasetId(), "", ""); + datasetService.createTable(new Table().setTableReference(table)); + datasetService.insertAll(table, expected, null); + + Path baseDir = Files.createTempDirectory(tempFolder, "testBigQueryTableSourceThroughJsonAPI"); String jobIdToken = "testJobIdToken"; - TableReference table = BigQueryHelpers.parseTableSpec("project.data_set.table_name"); - String extractDestinationDir = "mock://tempLocation"; BoundedSource bqSource = BigQueryTableSource.create( StaticValueProvider.of(jobIdToken), StaticValueProvider.of(table), - extractDestinationDir, fakeBqServices, + baseDir.toString(), fakeBqServices, StaticValueProvider.of("project")); - List expected = ImmutableList.of( - new TableRow().set("name", "a").set("number", "1"), - new TableRow().set("name", "b").set("number", "2"), - new TableRow().set("name", "c").set("number", "3")); - PipelineOptions options = PipelineOptionsFactory.create(); Assert.assertThat( SourceTestUtils.readFromSource(bqSource, options), @@ -1244,43 +1240,48 @@ public void testBigQueryTableSourceInitSplit() throws Exception { extractJob.setStatus(new JobStatus()) .setStatistics(jobStats); + FakeDatasetService fakeDatasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - .withDatasetService(mockDatasetService) - .readerReturns( - toJsonString(new TableRow().set("name", "a").set("number", "1")), - toJsonString(new TableRow().set("name", "b").set("number", "2")), - toJsonString(new TableRow().set("name", "c").set("number", "3"))); + .withDatasetService(fakeDatasetService); + + List expected = ImmutableList.of( + new TableRow().set("name", "a").set("number", 1L), + new TableRow().set("name", "b").set("number", 2L), + new TableRow().set("name", "c").set("number", 3L), + new TableRow().set("name", "d").set("number", 4L), + new TableRow().set("name", "e").set("number", 5L), + new TableRow().set("name", "f").set("number", 6L)); - String jobIdToken = "testJobIdToken"; TableReference table = BigQueryHelpers.parseTableSpec("project:data_set.table_name"); - String extractDestinationDir = "mock://tempLocation"; + fakeDatasetService.createDataset("project", "data_set", "", ""); + fakeDatasetService.createTable(new Table().setTableReference(table) + .setSchema(new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER"))))); + fakeDatasetService.insertAll(table, expected, null); + + Path baseDir = Files.createTempDirectory(tempFolder, "testBigQueryTableSourceInitSplit"); + + String jobIdToken = "testJobIdToken"; + String extractDestinationDir = baseDir.toString(); BoundedSource bqSource = BigQueryTableSource.create( StaticValueProvider.of(jobIdToken), StaticValueProvider.of(table), extractDestinationDir, fakeBqServices, StaticValueProvider.of("project")); - List expected = ImmutableList.of( - new TableRow().set("name", "a").set("number", "1"), - new TableRow().set("name", "b").set("number", "2"), - new TableRow().set("name", "c").set("number", "3")); PipelineOptions options = PipelineOptionsFactory.create(); - options.setTempLocation("mock://tempLocation"); - - IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true /* override */); - when(mockIOChannelFactory.resolve(anyString(), anyString())) - .thenReturn("mock://tempLocation/output"); - when(mockDatasetService.getTable(any(TableReference.class))) - .thenReturn(new Table().setSchema(new TableSchema())); + options.setTempLocation(baseDir.toString()); - Assert.assertThat( - SourceTestUtils.readFromSource(bqSource, options), - CoreMatchers.is(expected)); + List read = SourceTestUtils.readFromSource(bqSource, options); + assertThat(read, containsInAnyOrder(Iterables.toArray(expected, TableRow.class))); SourceTestUtils.assertSplitAtFractionBehavior( bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); List> sources = bqSource.split(100, options); - assertEquals(1, sources.size()); + assertEquals(2, sources.size()); BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); } @@ -1306,80 +1307,63 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { .setStatistics(extractJobStats); FakeJobService fakeJobService = new FakeJobService(); + FakeDatasetService fakeDatasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(fakeJobService) - .withDatasetService(mockDatasetService) - .readerReturns( - toJsonString(new TableRow().set("name", "a").set("number", "1")), - toJsonString(new TableRow().set("name", "b").set("number", "2")), - toJsonString(new TableRow().set("name", "c").set("number", "3"))); + .withDatasetService(fakeDatasetService); + + List expected = ImmutableList.of( + new TableRow().set("name", "a").set("number", 1L), + new TableRow().set("name", "b").set("number", 2L), + new TableRow().set("name", "c").set("number", 3L), + new TableRow().set("name", "d").set("number", 4L), + new TableRow().set("name", "e").set("number", 5L), + new TableRow().set("name", "f").set("number", 6L)); - String jobIdToken = "testJobIdToken"; - String extractDestinationDir = "mock://tempLocation"; TableReference destinationTable = BigQueryHelpers.parseTableSpec("project:data_set.table_name"); + fakeDatasetService.createDataset("project", "data_set", "", ""); + fakeDatasetService.createTable(new Table() + .setTableReference(destinationTable) + .setSchema(new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER"))))); + Path baseDir = Files.createTempDirectory(tempFolder, "testBigQueryQuerySourceInitSplit"); + + String jobIdToken = "testJobIdToken"; + String query = FakeBigQueryServices.encodeQuery(expected); + String extractDestinationDir = baseDir.toString(); BoundedSource bqSource = BigQueryQuerySource.create( - StaticValueProvider.of(jobIdToken), StaticValueProvider.of("query"), + StaticValueProvider.of(jobIdToken), StaticValueProvider.of(query), StaticValueProvider.of(destinationTable), true /* flattenResults */, true /* useLegacySql */, extractDestinationDir, fakeBqServices); - List expected = ImmutableList.of( - new TableRow().set("name", "a").set("number", "1"), - new TableRow().set("name", "b").set("number", "2"), - new TableRow().set("name", "c").set("number", "3")); - PipelineOptions options = PipelineOptionsFactory.create(); options.setTempLocation(extractDestinationDir); TableReference queryTable = new TableReference() - .setProjectId("testproject") - .setDatasetId("testDataset") - .setTableId("testTable"); - // when(mockJobService.dryRunQuery(anyString(), Mockito.any())) - // .thenReturn(new JobStatistics().setQuery( - // new JobStatistics2() - // .setTotalBytesProcessed(100L) - // .setReferencedTables(ImmutableList.of(queryTable)))); - fakeJobService.expectDryRunQuery("testproject", "query", + .setProjectId("project") + .setDatasetId("data_set") + .setTableId("table_name"); + + fakeJobService.expectDryRunQuery("project", query, new JobStatistics().setQuery( new JobStatistics2() .setTotalBytesProcessed(100L) .setReferencedTables(ImmutableList.of(queryTable)))); - // when(mockDatasetService.getTable(eq(queryTable))) - // .thenReturn(new Table().setSchema(new TableSchema())); - // when(mockDatasetService.getTable(eq(destinationTable))) - // .thenReturn(new Table().setSchema(new TableSchema())); - IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true /* override */); - when(mockIOChannelFactory.resolve(anyString(), anyString())) - .thenReturn("mock://tempLocation/output"); - //when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) - // .thenReturn(extractJob); - - Assert.assertThat( - SourceTestUtils.readFromSource(bqSource, options), - CoreMatchers.is(expected)); + List read = SourceTestUtils.readFromSource(bqSource, options); + assertThat(read, containsInAnyOrder(Iterables.toArray(expected, TableRow.class))); SourceTestUtils.assertSplitAtFractionBehavior( bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); + List> sources = bqSource.split(100, options); - assertEquals(1, sources.size()); + assertEquals(2, sources.size()); BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); - - /* - Mockito.verify(mockJobService) - .startQueryJob( - Mockito.any(), Mockito.any()); - Mockito.verify(mockJobService) - .startExtractJob(Mockito.any(), Mockito.any()); - Mockito.verify(mockDatasetService) - .createDataset(anyString(), anyString(), anyString(), anyString()); - ArgumentCaptor queryConfigArg = - ArgumentCaptor.forClass(JobConfigurationQuery.class); - Mockito.verify(mockJobService).dryRunQuery(anyString(), queryConfigArg.capture()); - assertEquals(true, queryConfigArg.getValue().getFlattenResults()); - assertEquals(true, queryConfigArg.getValue().getUseLegacySql());*/ } @Test @@ -1402,68 +1386,60 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { extractJob.setStatus(new JobStatus()) .setStatistics(extractJobStats); + FakeDatasetService datasetService = new FakeDatasetService(); + FakeJobService jobService = new FakeJobService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService()) - .withDatasetService(mockDatasetService) - .readerReturns( - toJsonString(new TableRow().set("name", "a").set("number", "1")), - toJsonString(new TableRow().set("name", "b").set("number", "2")), - toJsonString(new TableRow().set("name", "c").set("number", "3"))); + .withJobService(jobService) + .withDatasetService(datasetService); - String jobIdToken = "testJobIdToken"; - String extractDestinationDir = "mock://tempLocation"; TableReference destinationTable = BigQueryHelpers.parseTableSpec("project:data_set.table_name"); + List expected = ImmutableList.of( + new TableRow().set("name", "a").set("number", 1L), + new TableRow().set("name", "b").set("number", 2L), + new TableRow().set("name", "c").set("number", 3L), + new TableRow().set("name", "d").set("number", 4L), + new TableRow().set("name", "e").set("number", 5L), + new TableRow().set("name", "f").set("number", 6L)); + datasetService.createDataset(destinationTable.getProjectId(), destinationTable.getDatasetId(), + "", ""); + Table table = new Table() + .setTableReference(destinationTable) + .setSchema(new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER")))); + datasetService.createTable(table); + + String query = FakeBigQueryServices.encodeQuery(expected); + jobService.expectDryRunQuery("project", query, + new JobStatistics().setQuery( + new JobStatistics2() + .setTotalBytesProcessed(100L) + .setReferencedTables(ImmutableList.of(table.getTableReference())))); + + Path baseDir = Files.createTempDirectory(tempFolder, "testBigQueryNoTableQuerySourceInitSplit"); + String jobIdToken = "testJobIdToken"; BoundedSource bqSource = BigQueryQuerySource.create( - StaticValueProvider.of(jobIdToken), StaticValueProvider.of("query"), + StaticValueProvider.of(jobIdToken), + StaticValueProvider.of(query), StaticValueProvider.of(destinationTable), - true /* flattenResults */, true /* useLegacySql */, - extractDestinationDir, fakeBqServices); + true /* flattenResults */, true /* useLegacySql */, baseDir.toString(), fakeBqServices); - List expected = ImmutableList.of( - new TableRow().set("name", "a").set("number", "1"), - new TableRow().set("name", "b").set("number", "2"), - new TableRow().set("name", "c").set("number", "3")); - PipelineOptions options = PipelineOptionsFactory.create(); - options.setTempLocation(extractDestinationDir); - - /* - when(mockJobService.dryRunQuery(anyString(), Mockito.any())) - .thenReturn(new JobStatistics().setQuery( - new JobStatistics2() - .setTotalBytesProcessed(100L))); - when(mockDatasetService.getTable(eq(destinationTable))) - .thenReturn(new Table().setSchema(new TableSchema())); - IOChannelUtils.setIOFactoryInternal("mock", mockIOChannelFactory, true); - when(mockIOChannelFactory.resolve(anyString(), anyString())) - .thenReturn("mock://tempLocation/output"); - when(mockJobService.pollJob(Mockito.any(), Mockito.anyInt())) - .thenReturn(extractJob);*/ - Assert.assertThat( - SourceTestUtils.readFromSource(bqSource, options), - CoreMatchers.is(expected)); + PipelineOptions options = PipelineOptionsFactory.create(); + options.setTempLocation(baseDir.toString()); + List read = convertBigDecimaslToLong( + SourceTestUtils.readFromSource(bqSource, options)); + assertThat(read, containsInAnyOrder(Iterables.toArray(expected, TableRow.class))); SourceTestUtils.assertSplitAtFractionBehavior( bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); List> sources = bqSource.split(100, options); - assertEquals(1, sources.size()); + assertEquals(2, sources.size()); BoundedSource actual = sources.get(0); assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); - - /* - Mockito.verify(Service) - .startQueryJob( - Mockito.any(), Mockito.any()); - Mockito.verify(mockJobService) - .startExtractJob(Mockito.any(), Mockito.any()); - Mockito.verify(mockDatasetService) - .createDataset(anyString(), anyString(), anyString(), anyString()); - ArgumentCaptor queryConfigArg = - ArgumentCaptor.forClass(JobConfigurationQuery.class); - Mockito.verify(mockJobService).dryRunQuery(anyString(), queryConfigArg.capture()); - assertEquals(true, queryConfigArg.getValue().getFlattenResults()); - assertEquals(true, queryConfigArg.getValue().getUseLegacySql());*/ } @Test @@ -1604,12 +1580,27 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file throws Exception { p.enableAbandonedNodeEnforcement(false); + // 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(); - 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)); + if (isSingleton) { + expectedPartitions.add(ShardedKey.of( + new TableDestination(singletonReference, singletonDescription), 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)); + } } } @@ -1642,11 +1633,7 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file WriteBundlesToFiles.ResultCoder.of()); ValueProvider singletonTable = null; - if (numFilesPerTable == 0 && numTables == 1) { - TableReference singletonReference = new TableReference() - .setProjectId("projectid") - .setDatasetId("dataset") - .setTableId("table"); + if (isSingleton) { singletonTable = StaticValueProvider.of(BigQueryHelpers.toJsonString(singletonReference)); } WritePartition writePartition = @@ -1680,12 +1667,10 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file tableFilesResult.addAll(partition.getValue()); } - assertEquals(expectedPartitions.size(), partitionsResult.size()); + assertThat(partitionsResult, + containsInAnyOrder(Iterables.toArray(expectedPartitions, ShardedKey.class))); - // assertThat(partitionsResult, - // containsInAnyOrder(Iterables.toArray(expectedPartitions, ShardedKey.class))); - - if (numFilesPerTable == 0 && numTables == 1) { + if (isSingleton) { assertEquals(1, filesPerTableResult.size()); List singletonFiles = filesPerTableResult.values().iterator().next(); assertTrue(Files.exists(Paths.get(singletonFiles.get(0)))); @@ -1700,15 +1685,11 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file public void testWriteTables() throws Exception { p.enableAbandonedNodeEnforcement(false); + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - // .startJobReturns("done", "done", "done", "done", "done", "done", "done", "done", - // "done", "done") - // .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, - // Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED, - // Status.SUCCEEDED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); - + .withDatasetService(datasetService); + datasetService.createDataset("project-id", "dataset-id", "", ""); long numTables = 3; long numPartitions = 3; long numFilesPerPartition = 10; @@ -1716,6 +1697,8 @@ public void testWriteTables() throws Exception { String tempFilePrefix = "tempFilePrefix"; Map> expectedTempTables = Maps.newHashMap(); + Path baseDir = Files.createTempDirectory(tempFolder, "testWriteTables"); + List, Iterable>>> partitions = Lists.newArrayList(); for (int i = 0; i < numTables; ++i) { @@ -1726,7 +1709,16 @@ public void testWriteTables() throws Exception { jobIdToken + "_0x%08x_%05d", tableDestination.hashCode(), j); List filesPerPartition = Lists.newArrayList(); for (int k = 0; k < numFilesPerPartition; ++k) { - filesPerPartition.add(String.format("files0x%08x_%05d", tableDestination.hashCode(), k)); + String filename = Paths.get(baseDir.toString(), + String.format("files0x%08x_%05d", tempTableId.hashCode(), k)).toString(); + try (WritableByteChannel channel = IOChannelUtils.create(filename, MimeTypes.TEXT)) { + try (OutputStream output = Channels.newOutputStream(channel)) { + TableRow tableRow = new TableRow().set("name", tableName); + TableRowJsonCoder.of().encode(tableRow, output, Context.OUTER); + output.write("\n".getBytes(StandardCharsets.UTF_8)); + } + } + filesPerPartition.add(filename); } partitions.add(KV.of(ShardedKey.of(tableDestination, j), (Iterable>) Collections.singleton(filesPerPartition))); @@ -1814,25 +1806,45 @@ public void testRemoveTemporaryFiles() throws Exception { public void testWriteRename() throws Exception { p.enableAbandonedNodeEnforcement(false); + FakeDatasetService datasetService = new FakeDatasetService(); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) - // .startJobReturns("done", "done") - // .pollJobReturns(Status.FAILED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .withDatasetService(datasetService); + datasetService.createDataset("project-id", "dataset-id", "", ""); - int numFinalTables = 3; - int numTempTables = 3; + final int numFinalTables = 3; + final int numTempTablesPerFinalTable = 3; + final int numRecordsPerTempTable = 10; + + Map> expectedRowsPerTable = Maps.newHashMap(); String jobIdToken = "jobIdToken"; - String jsonTable = "{}"; Map> tempTables = Maps.newHashMap(); for (int i = 0; i < numFinalTables; ++i) { String tableName = "project-id:dataset-id.table_" + i; - TableDestination tableDestination = new TableDestination(tableName, tableName); + TableDestination tableDestination = new TableDestination( + tableName, "table_" + i + "_desc"); List tables = Lists.newArrayList(); tempTables.put(tableDestination, tables); - for (int j = 0; i < numTempTables; ++i) { - tables.add(String.format( - "{\"project-id:dataset-id.tableId\":\"%s_%05d_%05d\"}", jobIdToken, i, j)); + + List expectedRows = expectedRowsPerTable.get(tableDestination); + if (expectedRows == null) { + expectedRows = Lists.newArrayList(); + expectedRowsPerTable.put(tableDestination, expectedRows); + } + for (int j = 0; i < numTempTablesPerFinalTable; ++i) { + TableReference tempTable = new TableReference() + .setProjectId("project-id") + .setDatasetId("dataset-id") + .setTableId(String.format("%s_%05d_%05d", jobIdToken, i, j)); + datasetService.createTable(new Table().setTableReference(tempTable)); + + List rows = Lists.newArrayList(); + for (int k = 0; k < numRecordsPerTempTable; ++k) { + rows.add(new TableRow().set("number", j * numTempTablesPerFinalTable + k)); + } + datasetService.insertAll(tempTable, rows, null); + expectedRows.addAll(rows); + tables.add(BigQueryHelpers.toJsonString(tempTable)); } } @@ -1857,37 +1869,52 @@ public void testWriteRename() throws Exception { tester.setSideInput(tempTablesView, GlobalWindow.INSTANCE, tempTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); tester.processElement(null); + + for (Map.Entry> entry : tempTables.entrySet()) { + TableDestination tableDestination = entry.getKey(); + TableReference tableReference = tableDestination.getTableReference(); + Table table = checkNotNull(datasetService.getTable(tableReference)); + assertEquals(tableReference.getTableId() + "_desc", tableDestination.getTableDescription()); + + List expectedRows = expectedRowsPerTable.get(tableDestination); + assertThat(datasetService.getAllRows(tableReference.getProjectId(), + tableReference.getDatasetId(), tableReference.getTableId()), + containsInAnyOrder(Iterables.toArray(expectedRows, TableRow.class))); + + // Temp tables should be deleted. + for (String tempTableJson : entry.getValue()) { + TableReference tempTable = BigQueryHelpers.fromJsonString( + tempTableJson, TableReference.class); + assertEquals(null, datasetService.getTable(tempTable)); + } + } } @Test public void testRemoveTemporaryTables() throws Exception { - String projectId = "someproject"; - String datasetId = "somedataset"; - List tables = Lists.newArrayList("table1", "table2", "table3"); + FakeDatasetService datasetService = new FakeDatasetService(); + String projectId = "project"; + String datasetId = "dataset"; + datasetService.createDataset(projectId, datasetId, "", ""); List tableRefs = Lists.newArrayList( - BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, - tables.get(0))), - BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, - tables.get(1))), - BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, - tables.get(2)))); + BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, "table1")), + BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, "table2")), + BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, "table3"))); + for (TableReference tableRef : tableRefs) { + datasetService.createTable(new Table().setTableReference(tableRef)); + } - doThrow(new IOException("Unable to delete table")) - .when(mockDatasetService).deleteTable(tableRefs.get(0)); - doNothing().when(mockDatasetService).deleteTable(tableRefs.get(1)); - doNothing().when(mockDatasetService).deleteTable(tableRefs.get(2)); + // Add one more table to delete that does not actually exist. + tableRefs.add( + BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, "table4"))); - WriteRename.removeTemporaryTables(mockDatasetService, tableRefs); + WriteRename.removeTemporaryTables(datasetService, tableRefs); for (TableReference ref : tableRefs) { loggedWriteRename.verifyDebug("Deleting table " + toJsonString(ref)); + checkState(datasetService.getTable(ref) == null, + "Table " + ref + " was not deleted!"); } - loggedWriteRename.verifyWarn("Failed to delete the table " - + toJsonString(tableRefs.get(0))); - loggedWriteRename.verifyNotLogged("Failed to delete the table " - + toJsonString(tableRefs.get(1))); - loggedWriteRename.verifyNotLogged("Failed to delete the table " - + toJsonString(tableRefs.get(2))); } /** Test options. **/ @@ -1957,43 +1984,6 @@ public boolean accept(File pathname) { }}).length); } - private class WriteExtractFiles implements SerializableFunction { - private final SerializableFunction schemaGenerator; - private final Collection> records; - - private WriteExtractFiles( - SerializableFunction schemaGenerator, - Collection> records) { - this.schemaGenerator = schemaGenerator; - this.records = records; - } - - @Override - public Void apply(GenericJson input) { - List destinations = (List) input.get("destinationUris"); - for (String destination : destinations) { - String newDest = destination.replace("*", "000000000000"); - Schema schema = schemaGenerator.apply(null); - try (WritableByteChannel channel = IOChannelUtils.create(newDest, MimeTypes.BINARY); - DataFileWriter tableRowWriter = - new DataFileWriter<>(new GenericDatumWriter(schema)) - .create(schema, Channels.newOutputStream(channel))) { - for (Map record : records) { - GenericRecordBuilder genericRecordBuilder = new GenericRecordBuilder(schema); - for (Map.Entry field : record.entrySet()) { - genericRecordBuilder.set(field.getKey(), field.getValue()); - } - tableRowWriter.append(genericRecordBuilder.build()); - } - } catch (IOException e) { - throw new IllegalStateException( - String.format("Could not create destination for extract job %s", destination), e); - } - } - return null; - } - } - @Test public void testShardedKeyCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable(ShardedKeyCoder.of(GlobalWindow.Coder.INSTANCE)); @@ -2013,4 +2003,19 @@ public void testComplexCoderSerializable() { TableRowInfoCoder.of()), IntervalWindow.getCoder())); } + + List convertBigDecimaslToLong(List toConvert) { + // The numbers come back as BigDecimal objects after JSON serialization. Change them back to + // longs so that we can assert the output. + List converted = Lists.newArrayList(); + for (TableRow entry : toConvert) { + TableRow convertedEntry = entry.clone(); + Object num = convertedEntry.get("number"); + if (num instanceof BigDecimal) { + convertedEntry.set("number", ((BigDecimal) num).longValue()); + } + converted.add(convertedEntry); + } + return converted; + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java index ed3ab37d1f37..6dfd9d78ff5b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java @@ -1,39 +1,59 @@ +/* + * 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 static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.fromJsonString; import static org.junit.Assert.assertEquals; +import com.google.api.client.util.Base64; import com.google.api.services.bigquery.model.JobConfigurationQuery; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; +import com.google.common.collect.Lists; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.List; import java.util.NoSuchElementException; + +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.options.BigQueryOptions; /** - * Created by relax on 3/30/17. + * A fake implementation of BigQuery's query service.. */ class FakeBigQueryServices implements BigQueryServices { - private String[] jsonTableRowReturns = new String[0]; private JobService jobService; - private DatasetService datasetService; + private FakeDatasetService datasetService; - public FakeBigQueryServices withJobService(JobService jobService) { + FakeBigQueryServices withJobService(JobService jobService) { this.jobService = jobService; return this; } - public FakeBigQueryServices withDatasetService(DatasetService datasetService) { + FakeBigQueryServices withDatasetService(FakeDatasetService datasetService) { this.datasetService = datasetService; return this; } - public FakeBigQueryServices readerReturns(String... jsonTableRowReturns) { - this.jsonTableRowReturns = jsonTableRowReturns; - return this; - } - @Override public JobService getJobService(BigQueryOptions bqOptions) { return jobService; @@ -45,26 +65,58 @@ public DatasetService getDatasetService(BigQueryOptions bqOptions) { } @Override - public BigQueryJsonReader getReaderFromTable( - BigQueryOptions bqOptions, TableReference tableRef) { - return new FakeBigQueryReader(jsonTableRowReturns); + public BigQueryJsonReader getReaderFromTable(BigQueryOptions bqOptions, TableReference tableRef) { + try { + List rows = datasetService.getAllRows( + tableRef.getProjectId(), tableRef.getDatasetId(), tableRef.getTableId()); + return new FakeBigQueryReader(rows); + } catch (Exception e) { + return null; + } } @Override public BigQueryJsonReader getReaderFromQuery( BigQueryOptions bqOptions, String projectId, JobConfigurationQuery queryConfig) { - return new FakeBigQueryReader(jsonTableRowReturns); + try { + List rows = rowsFromEncodedQuery(queryConfig.getQuery()); + return new FakeBigQueryReader(rows); + } catch (IOException e) { + return null; + } + } + + static List rowsFromEncodedQuery(String query) throws IOException { + ListCoder listCoder = ListCoder.of(TableRowJsonCoder.of()); + ByteArrayInputStream input = new ByteArrayInputStream(Base64.decodeBase64(query)); + List rows = listCoder.decode(input, Context.OUTER); + for (TableRow row : rows) { + convertNumbers(row); + } + return rows; + } + + static String encodeQuery(List rows) throws IOException { + ListCoder listCoder = ListCoder.of(TableRowJsonCoder.of()); + ByteArrayOutputStream output = new ByteArrayOutputStream(); + listCoder.encode(rows, output, Context.OUTER); + return Base64.encodeBase64String(output.toByteArray()); } private static class FakeBigQueryReader implements BigQueryJsonReader { private static final int UNSTARTED = -1; private static final int CLOSED = Integer.MAX_VALUE; - private String[] jsonTableRowReturns; + private List serializedTableRowReturns; private int currIndex; - FakeBigQueryReader(String[] jsonTableRowReturns) { - this.jsonTableRowReturns = jsonTableRowReturns; + FakeBigQueryReader(List tableRowReturns) throws IOException { + this.serializedTableRowReturns = Lists.newArrayListWithExpectedSize(tableRowReturns.size()); + for (TableRow tableRow : tableRowReturns) { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + TableRowJsonCoder.of().encode(tableRow, output, Context.OUTER); + serializedTableRowReturns.add(output.toByteArray()); + } this.currIndex = UNSTARTED; } @@ -72,20 +124,27 @@ private static class FakeBigQueryReader implements BigQueryJsonReader { public boolean start() throws IOException { assertEquals(UNSTARTED, currIndex); currIndex = 0; - return currIndex < jsonTableRowReturns.length; + return currIndex < serializedTableRowReturns.size(); } @Override public boolean advance() throws IOException { - return ++currIndex < jsonTableRowReturns.length; + return ++currIndex < serializedTableRowReturns.size(); } @Override public TableRow getCurrent() throws NoSuchElementException { - if (currIndex >= jsonTableRowReturns.length) { + if (currIndex >= serializedTableRowReturns.size()) { throw new NoSuchElementException(); } - return fromJsonString(jsonTableRowReturns[currIndex], TableRow.class); + + ByteArrayInputStream input = new ByteArrayInputStream( + serializedTableRowReturns.get(currIndex)); + try { + return convertNumbers(TableRowJsonCoder.of().decode(input, Context.OUTER)); + } catch (IOException e) { + return null; + } } @Override @@ -93,4 +152,15 @@ public void close() throws IOException { currIndex = CLOSED; } } + + + // Longs tend to get converted back to Integers due to JSON serialization. Convert them back. + static TableRow convertNumbers(TableRow tableRow) { + for (TableRow.Entry entry : tableRow.entrySet()) { + if (entry.getValue() instanceof Integer) { + entry.setValue(new Long((Integer) entry.getValue())); + } + } + return tableRow; + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java index 9b2cf631e1bf..5103adb07c8c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java @@ -1,9 +1,29 @@ +/* + * 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 static com.google.common.base.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.client.http.HttpHeaders; import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.DatasetReference; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; @@ -24,13 +44,13 @@ public Table getTable(TableReference tableRef) throws InterruptedException, IOException { synchronized (BigQueryIOTest.tables) { Map dataset = - checkNotNull( - BigQueryIOTest.tables.get(tableRef.getProjectId(), tableRef.getDatasetId()), - "Tried to get a dataset %s:%s from %s, but no such dataset was set", - tableRef.getProjectId(), - tableRef.getDatasetId(), - tableRef.getTableId(), - FakeDatasetService.class.getSimpleName()); + BigQueryIOTest.tables.get(tableRef.getProjectId(), tableRef.getDatasetId()); + if (dataset == null) { + throwNotFound( + "Tried to get a dataset %s:%s from, but no such dataset was set", + tableRef.getProjectId(), + tableRef.getDatasetId()); + } TableContainer tableContainer = dataset.get(tableRef.getTableId()); return tableContainer == null ? null : tableContainer.getTable(); } @@ -44,27 +64,40 @@ List getAllRows(String projectId, String datasetId, String tableId) } private TableContainer getTableContainer(String projectId, String datasetId, String tableId) - throws InterruptedException, IOException { - synchronized (BigQueryIOTest.tables) { - Map dataset = - checkNotNull( - BigQueryIOTest.tables.get(projectId, datasetId), - "Tried to get a dataset %s:%s from %s, but no such dataset was set", - projectId, - datasetId, - FakeDatasetService.class.getSimpleName()); - return checkNotNull(dataset.get(tableId), - "Tried to get a table %s:%s.%s from %s, but no such table was set", - projectId, - datasetId, - tableId, - FakeDatasetService.class.getSimpleName()); - } + throws InterruptedException, IOException { + synchronized (BigQueryIOTest.tables) { + Map dataset = BigQueryIOTest.tables.get(projectId, datasetId); + if (dataset == null) { + throwNotFound( + "Tried to get a dataset %s:%s, but no such dataset was set", + projectId, + datasetId); + } + TableContainer tableContainer = dataset.get(tableId); + if (tableContainer == null) { + throwNotFound( + "Tried to get a table %s:%s.%s, but no such table was set", + projectId, + datasetId, + tableId); + } + return tableContainer; + } } @Override public void deleteTable(TableReference tableRef) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); + synchronized (BigQueryIOTest.tables) { + Map dataset = + BigQueryIOTest.tables.get(tableRef.getProjectId(), tableRef.getDatasetId()); + if (dataset == null) { + throwNotFound( + "Tried to get a dataset %s:%s, but no such table was set", + tableRef.getProjectId(), + tableRef.getDatasetId()); + } + dataset.remove(tableRef.getTableId()); + } } @@ -73,13 +106,13 @@ public void createTable(Table table) throws IOException { TableReference tableReference = table.getTableReference(); synchronized (BigQueryIOTest.tables) { Map dataset = - checkNotNull( - BigQueryIOTest.tables.get(tableReference.getProjectId(), - tableReference.getDatasetId()), - "Tried to get a dataset %s:%s from %s, but no such table was set", - tableReference.getProjectId(), - tableReference.getDatasetId(), - FakeDatasetService.class.getSimpleName()); + BigQueryIOTest.tables.get(tableReference.getProjectId(), tableReference.getDatasetId()); + if (dataset == null) { + throwNotFound( + "Tried to get a dataset %s:%s, but no such table was set", + tableReference.getProjectId(), + tableReference.getDatasetId()); + } TableContainer tableContainer = dataset.get(tableReference.getTableId()); if (tableContainer == null) { tableContainer = new TableContainer(table); @@ -98,7 +131,16 @@ public boolean isTableEmpty(TableReference tableRef) @Override public Dataset getDataset( String projectId, String datasetId) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); + synchronized (BigQueryIOTest.tables) { + Map dataset = BigQueryIOTest.tables.get(projectId, datasetId); + if (dataset == null) { + throwNotFound("Tried to get a dataset %s:%s, but no such table was set", + projectId, datasetId); + } + return new Dataset().setDatasetReference(new DatasetReference() + .setDatasetId(datasetId) + .setProjectId(projectId)); + } } @Override @@ -117,7 +159,9 @@ public void createDataset( @Override public void deleteDataset(String projectId, String datasetId) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); + synchronized (BigQueryIOTest.tables) { + BigQueryIOTest.tables.remove(projectId, datasetId); + } } @Override @@ -138,8 +182,7 @@ public long insertAll( TableContainer tableContainer = getTableContainer( ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); for (int i = 0; i < rowList.size(); ++i) { - tableContainer.addRow(rowList.get(i), insertIdList.get(i)); - dataSize += rowList.get(i).toString().length(); + dataSize += tableContainer.addRow(rowList.get(i), insertIdList.get(i)); } return dataSize; } @@ -150,23 +193,16 @@ public Table patchTableDescription(TableReference tableReference, @Nullable String tableDescription) throws IOException, InterruptedException { synchronized (BigQueryIOTest.tables) { - Map dataset = - checkNotNull( - BigQueryIOTest.tables.get(tableReference.getProjectId(), - tableReference.getDatasetId()), - "Tried to get a dataset %s:%s from %s, but no such dataset was set", - tableReference.getProjectId(), - tableReference.getDatasetId(), - tableReference.getTableId(), - FakeDatasetService.class.getSimpleName()); - TableContainer tableContainer = checkNotNull(dataset.get(tableReference.getTableId()), - "Tried to patch a table %s:%s.%s from %s, but no such table was set", - tableReference.getProjectId(), - tableReference.getDatasetId(), - tableReference.getTableId(), - FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = getTableContainer(tableReference.getProjectId(), + tableReference.getDatasetId(), tableReference.getTableId()); tableContainer.getTable().setDescription(tableDescription); return tableContainer.getTable(); } } + + void throwNotFound(String format, Object... args) throws IOException { + throw new IOException( + new GoogleJsonResponseException.Builder(404, + String.format(format, args), new HttpHeaders()).build()); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java index 3c67c3da5288..a2454fbb56f8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -1,12 +1,30 @@ +/* + * 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 static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; import com.google.api.client.json.JsonFactory; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; +import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfiguration; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -29,9 +47,18 @@ import java.io.FileReader; import java.io.IOException; import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; import java.nio.charset.StandardCharsets; import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.TableRowJsonCoder; @@ -40,10 +67,13 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.Transport; import org.joda.time.Duration; /** + * A fake implementation of BigQuery's job service. */ class FakeJobService implements JobService, Serializable { static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); @@ -66,6 +96,8 @@ private static class JobInfo { private static final com.google.common.collect.Table allJobs = HashBasedTable.create(); + private static final com.google.common.collect.Table> + filesForLoadJobs = HashBasedTable.create(); private static final com.google.common.collect.Table dryRunQueryResults = HashBasedTable.create(); @@ -82,6 +114,18 @@ public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig) job.setConfiguration(new JobConfiguration().setLoad(loadConfig)); job.setKind(" bigquery#job"); job.setStatus(new JobStatus().setState("PENDING")); + + // Copy the files to a new location for import, as the temporary files will be deleted by + // the caller. + if (loadConfig.getSourceUris().size() > 0) { + List loadFiles = Lists.newArrayList(); + for (String filename : loadConfig.getSourceUris()) { + loadFiles.add(filename + ThreadLocalRandom.current().nextInt()); + } + IOChannelUtils.getFactory(loadFiles.get(0)).copy(loadConfig.getSourceUris(), loadFiles); + filesForLoadJobs.put(jobRef.getProjectId(), jobRef.getJobId(), loadFiles); + } + allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); } } @@ -91,8 +135,6 @@ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extract throws InterruptedException, IOException { checkArgument(extractConfig.getDestinationFormat().equals("AVRO"), "Only extract to AVRO is supported"); - checkArgument(extractConfig.getDestinationUris().size() == 1, - "Must specify exactly one destination URI."); synchronized (allJobs) { Job job = new Job(); job.setJobReference(jobRef); @@ -106,6 +148,14 @@ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extract @Override public void startQueryJob(JobReference jobRef, JobConfigurationQuery query) throws IOException, InterruptedException { + synchronized (allJobs) { + Job job = new Job(); + job.setJobReference(jobRef); + job.setConfiguration(new JobConfiguration().setQuery(query)); + job.setKind(" bigquery#job"); + job.setStatus(new JobStatus().setState("PENDING")); + allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); + } } @Override @@ -127,8 +177,8 @@ public Job pollJob(JobReference jobRef, int maxAttempts) BackOff backoff = FluentBackoff.DEFAULT .withMaxRetries(maxAttempts) - .withInitialBackoff(Duration.millis(50)) - .withMaxBackoff(Duration.standardMinutes(1)) + .withInitialBackoff(Duration.millis(10)) + .withMaxBackoff(Duration.standardSeconds(1)) .backoff(); Sleeper sleeper = Sleeper.DEFAULT; try { @@ -136,7 +186,8 @@ public Job pollJob(JobReference jobRef, int maxAttempts) Job job = getJob(jobRef); if (job != null) { JobStatus status = job.getStatus(); - if (status != null && status.getState() != null && status.getState().equals("DONE")) { + if (status != null && status.getState() != null + && (status.getState().equals("DONE") || status.getState().equals("FAILED"))) { return job; } } @@ -173,12 +224,15 @@ public Job getJob(JobReference jobRef) throws InterruptedException { if (job == null) { return null; } - ++job.getJobCount; - if (job.getJobCount == GET_JOBS_TRANSITION_INTERVAL + 1) { - job.job.getStatus().setState("RUNNING"); - } else if (job.getJobCount == 2 * GET_JOBS_TRANSITION_INTERVAL + 1) { - runJob(job.job); - job.job.getStatus().setState("DONE"); + try { + ++job.getJobCount; + if (job.getJobCount == GET_JOBS_TRANSITION_INTERVAL + 1) { + job.job.getStatus().setState("RUNNING"); + } else if (job.getJobCount == 2 * GET_JOBS_TRANSITION_INTERVAL + 1) { + job.job.setStatus(runJob(job.job)); + } + } catch (Exception e) { + job.job.getStatus().setState("FAILED").setErrorResult(new ErrorProto()); } return JSON_FACTORY.fromString(JSON_FACTORY.toString(job.job), Job.class); } @@ -187,41 +241,50 @@ public Job getJob(JobReference jobRef) throws InterruptedException { } } - private void runJob(Job job) throws InterruptedException, IOException { + private JobStatus runJob(Job job) throws InterruptedException, IOException { if (job.getConfiguration().getLoad() != null) { - runLoadJob(job.getConfiguration().getLoad()); + return runLoadJob(job.getJobReference(), job.getConfiguration().getLoad()); } else if (job.getConfiguration().getCopy() != null) { - runCopyJob(job.getConfiguration().getCopy()); + return runCopyJob(job.getConfiguration().getCopy()); } else if (job.getConfiguration().getExtract() != null) { - runExtractJob(job, job.getConfiguration().getExtract()); + return runExtractJob(job, job.getConfiguration().getExtract()); + } else if (job.getConfiguration().getQuery() != null) { + return runQueryJob(job.getConfiguration().getQuery()); } + return new JobStatus().setState("DONE"); } - private void validateDispositions(Table table, CreateDisposition createDisposition, - WriteDisposition writeDisposition) + private boolean validateDispositions(Table table, CreateDisposition createDisposition, + WriteDisposition writeDisposition) throws InterruptedException, IOException { if (table == null) { - checkState(createDisposition != CreateDisposition.CREATE_NEVER, - "CreateDisposition == CREATE_NEVER but the table doesn't exist."); + if (createDisposition == CreateDisposition.CREATE_NEVER) { + return false; + } } else if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) { datasetService.deleteTable(table.getTableReference()); } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) { List allRows = datasetService.getAllRows(table.getTableReference().getProjectId(), table.getTableReference().getDatasetId(), table.getTableReference().getTableId()); - checkState(allRows.isEmpty(), "Write disposition was set to WRITE_EMPTY," - + " but the table was not empty."); + if (!allRows.isEmpty()) { + return false; + } } + return true; } - private void runLoadJob(JobConfigurationLoad load) + + private JobStatus runLoadJob(JobReference jobRef, JobConfigurationLoad load) throws InterruptedException, IOException { TableReference destination = load.getDestinationTable(); TableSchema schema = load.getSchema(); - List sourceFiles = load.getSourceUris(); + List sourceFiles = filesForLoadJobs.get(jobRef.getProjectId(), jobRef.getJobId()); WriteDisposition writeDisposition = WriteDisposition.valueOf(load.getWriteDisposition()); CreateDisposition createDisposition = CreateDisposition.valueOf(load.getCreateDisposition()); checkArgument(load.getSourceFormat().equals("NEWLINE_DELIMITED_JSON")); Table existingTable = datasetService.getTable(destination); - validateDispositions(existingTable, createDisposition, writeDisposition); + if (!validateDispositions(existingTable, createDisposition, writeDisposition)) { + return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); + } datasetService.createTable(new Table().setTableReference(destination).setSchema(schema)); @@ -230,31 +293,52 @@ private void runLoadJob(JobConfigurationLoad load) rows.addAll(readRows(filename)); } datasetService.insertAll(destination, rows, null); + return new JobStatus().setState("DONE"); } - private void runCopyJob(JobConfigurationTableCopy copy) + private JobStatus runCopyJob(JobConfigurationTableCopy copy) throws InterruptedException, IOException { List sources = copy.getSourceTables(); TableReference destination = copy.getDestinationTable(); WriteDisposition writeDisposition = WriteDisposition.valueOf(copy.getWriteDisposition()); CreateDisposition createDisposition = CreateDisposition.valueOf(copy.getCreateDisposition()); Table existingTable = datasetService.getTable(destination); - validateDispositions(existingTable, createDisposition, writeDisposition); + if (!validateDispositions(existingTable, createDisposition, writeDisposition)) { + return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); + } List allRows = Lists.newArrayList(); for (TableReference source : sources) { allRows.addAll(datasetService.getAllRows( source.getProjectId(), source.getDatasetId(), source.getTableId())); } + datasetService.createTable(new Table().setTableReference(destination)); datasetService.insertAll(destination, allRows, null); + return new JobStatus().setState("DONE"); } - private void runExtractJob(Job job, JobConfigurationExtract extract) { + private JobStatus runExtractJob(Job job, JobConfigurationExtract extract) + throws InterruptedException, IOException { TableReference sourceTable = extract.getSourceTable(); - extract.getDestinationUris().get(0); - List destinationFileCounts = Lists.newArrayList(0L); + + List rows = datasetService.getAllRows( + sourceTable.getProjectId(), sourceTable.getDatasetId(), sourceTable.getTableId()); + TableSchema schema = datasetService.getTable(sourceTable).getSchema(); + List destinationFileCounts = Lists.newArrayList(); + for (String destination : extract.getDestinationUris()) { + destinationFileCounts.add(writeRows(sourceTable.getTableId(), rows, schema, destination)); + } job.setStatistics(new JobStatistics().setExtract( new JobStatistics4().setDestinationUriFileCounts(destinationFileCounts))); + return new JobStatus().setState("DONE"); + } + + private JobStatus runQueryJob(JobConfigurationQuery query) + throws IOException, InterruptedException { + List rows = FakeBigQueryServices.rowsFromEncodedQuery(query.getQuery()); + datasetService.createTable(new Table().setTableReference(query.getDestinationTable())); + datasetService.insertAll(query.getDestinationTable(), rows, null); + return new JobStatus().setState("DONE"); } private List readRows(String filename) throws IOException { @@ -270,4 +354,42 @@ private List readRows(String filename) throws IOException { } return tableRows; } + + private long writeRows(String tableId, List rows, TableSchema schema, + String destinationPattern) throws IOException { + Schema avroSchema = BigQueryAvroUtils.toGenericAvroSchema(tableId, schema.getFields()); + List rowsToWrite = Lists.newArrayList(); + int shard = 0; + for (int i = 0; i < rows.size(); ++i) { + rowsToWrite.add(rows.get(i)); + if (rowsToWrite.size() == 5) { + writeRowsHelper(rowsToWrite, avroSchema, destinationPattern, shard++); + rowsToWrite.clear(); + } + } + if (!rowsToWrite.isEmpty()) { + writeRowsHelper(rowsToWrite, avroSchema, destinationPattern, shard++); + } + return shard; + } + + private void writeRowsHelper(List rows, Schema avroSchema, + String destinationPattern, int shard) throws IOException { + String filename = destinationPattern.replace("*", String.format("%012d", shard)); + try (WritableByteChannel channel = IOChannelUtils.create(filename, MimeTypes.BINARY); + DataFileWriter tableRowWriter = + new DataFileWriter<>(new GenericDatumWriter(avroSchema)) + .create(avroSchema, Channels.newOutputStream(channel))) { + for (Map record : rows) { + GenericRecordBuilder genericRecordBuilder = new GenericRecordBuilder(avroSchema); + for (Map.Entry field : record.entrySet()) { + genericRecordBuilder.set(field.getKey(), field.getValue()); + } + tableRowWriter.append(genericRecordBuilder.build()); + } + } catch (IOException e) { + throw new IllegalStateException( + String.format("Could not create destination for extract job %s", filename), e); + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java index b2fc1706e20a..d52723b79bc0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.Table; @@ -7,23 +24,31 @@ import java.util.List; /** - * Created by relax on 3/30/17. + * Encapsulates a BigQuery Table, and it's contents. */ class TableContainer { Table table; List rows; List ids; - + Long sizeBytes; TableContainer(Table table) { this.table = table; this.rows = new ArrayList<>(); this.ids = new ArrayList<>(); + this.sizeBytes = 0L; } - TableContainer addRow(TableRow row, String id) { + long addRow(TableRow row, String id) { rows.add(row); ids.add(id); - return this; + long rowSize = row.toString().length(); + Long tableSize = table.getNumBytes(); + if (tableSize == null) { + table.setNumBytes(rowSize); + } else { + table.setNumBytes(tableSize + rowSize); + } + return rowSize; } Table getTable() { From ac8e202a6239743d383c46c353a679dc57b7470d Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Sun, 2 Apr 2017 21:39:50 -0700 Subject: [PATCH 7/7] Separate streaming writes into two pluggable components - CreateTables, and StreamingWriteTables. Also address many code review comments. Also merge with master. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 176 ++++++++++-------- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 13 ++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 21 +-- .../io/gcp/bigquery/BigQueryTableSource.java | 4 +- .../sdk/io/gcp/bigquery/CreateTables.java | 95 ++++++---- .../io/gcp/bigquery/GenerateShardedTable.java | 3 +- .../sdk/io/gcp/bigquery/PrepareWrite.java | 80 ++++---- .../beam/sdk/io/gcp/bigquery/ShardedKey.java | 1 + .../sdk/io/gcp/bigquery/StreamingInserts.java | 44 +---- .../io/gcp/bigquery/StreamingWriteTables.java | 86 +++++++++ .../sdk/io/gcp/bigquery/TableDestination.java | 1 + .../gcp/bigquery/TableDestinationCoder.java | 62 +++--- .../sdk/io/gcp/bigquery/TableRowWriter.java | 14 +- .../sdk/io/gcp/bigquery/TagWithUniqueIds.java | 14 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 25 +-- .../sdk/io/gcp/bigquery/WritePartition.java | 127 +++++++++---- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 5 +- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 17 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 66 ++++--- 19 files changed, 516 insertions(+), 338 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.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 06fdfcebdc0f..236b234146d2 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 @@ -18,7 +18,6 @@ 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.api.services.bigquery.model.TableSchema; import java.io.IOException; @@ -35,7 +34,6 @@ import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -47,6 +45,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -54,17 +53,13 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -/** - * PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. - */ +/** 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; + private static class ConstantSchemaFunction + implements SerializableFunction { + private final @Nullable ValueProvider jsonSchema; ConstantSchemaFunction(ValueProvider jsonSchema) { this.jsonSchema = jsonSchema; @@ -86,7 +81,6 @@ public TableSchema apply(TableDestination table) { public WriteResult expand(PCollection> input) { Pipeline p = input.getPipeline(); BigQueryOptions options = p.getOptions().as(BigQueryOptions.class); - ValueProvider table = write.getTableWithDefaultProject(options); final String stepUuid = BigQueryHelpers.randomUUIDString(); @@ -94,40 +88,41 @@ public WriteResult expand(PCollection> input) { String tempFilePrefix; try { IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation); - tempFilePrefix = factory.resolve( - factory.resolve(tempLocation, "BigQueryWriteTemp"), - stepUuid); + tempFilePrefix = + factory.resolve(factory.resolve(tempLocation, "BigQueryWriteTemp"), stepUuid); } catch (IOException e) { throw new RuntimeException( - String.format("Failed to resolve BigQuery temp location in %s", tempLocation), - e); + String.format("Failed to resolve BigQuery temp location in %s", tempLocation), e); } // Create a singleton job ID token at execution time. This will be used as the base for all // load jobs issued from this instance of the transfomr. PCollection singleton = p.apply("Create", Create.of(tempFilePrefix)); - PCollectionView jobIdTokenView = p - .apply("TriggerIdCreation", Create.of("ignored")) - .apply("CreateJobId", MapElements.via( - new SimpleFunction() { - @Override - public String apply(String input) { - return stepUuid; - } - })) - .apply(View.asSingleton()); + PCollectionView jobIdTokenView = + p.apply("TriggerIdCreation", Create.of("ignored")) + .apply( + "CreateJobId", + MapElements.via( + new SimpleFunction() { + @Override + public String apply(String input) { + return stepUuid; + } + })) + .apply(View.asSingleton()); PCollection> inputInGlobalWindow = - input.apply("rewindowIntoGlobal", + input.apply( + "rewindowIntoGlobal", Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); // PCollection of filename, file byte size, and table destination. - PCollection results = inputInGlobalWindow - .apply("WriteBundlesToFiles", - ParDo.of(new WriteBundlesToFiles(tempFilePrefix))) - .setCoder(WriteBundlesToFiles.ResultCoder.of()); + PCollection results = + inputInGlobalWindow + .apply("WriteBundlesToFiles", ParDo.of(new WriteBundlesToFiles(tempFilePrefix))) + .setCoder(WriteBundlesToFiles.ResultCoder.of()); TupleTag, List>> multiPartitionsTag = new TupleTag, List>>("multiPartitionsTag") {}; @@ -136,20 +131,23 @@ public String apply(String input) { // 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. - PCollectionTuple partitions = singleton.apply("WritePartition", - ParDo.of(new WritePartition( - write.getJsonTableRef(), - write.getTableDescription(), - resultsView, - multiPartitionsTag, - singlePartitionTag)) - .withSideInputs(resultsView) - .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); + PCollectionTuple partitions = + singleton.apply( + "WritePartition", + ParDo.of( + new WritePartition( + write.getJsonTableRef(), + write.getTableDescription(), + 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. @@ -158,55 +156,69 @@ public String apply(String input) { new ConstantSchemaFunction(write.getJsonSchema()); Coder, List>> partitionsCoder = - KvCoder.of(ShardedKeyCoder.of(TableDestinationCoder.of()), - ListCoder.of(StringUtf8Coder.of())); + 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. - PCollection> tempTables = partitions.get(multiPartitionsTag) - .setCoder(partitionsCoder) - // What's this GroupByKey for? Is this so we have a deterministic temp tables? If so, maybe - // Reshuffle is better here. - .apply("MultiPartitionsGroupByKey", - GroupByKey., List>create()) - .apply("MultiPartitionsWriteTables", ParDo.of(new WriteTables( - false, - write.getBigQueryServices(), - jobIdTokenView, - tempFilePrefix, - WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - schemaFunction)) - .withSideInputs(jobIdTokenView)); + PCollection> tempTables = + partitions + .get(multiPartitionsTag) + .setCoder(partitionsCoder) + // Reshuffle will distribute this among multiple workers, and also guard against + // reexecution of the WritePartitions step once WriteTables has begun. + .apply( + "MultiPartitionsReshuffle", + Reshuffle., List>of()) + .apply( + "MultiPartitionsWriteTables", + ParDo.of( + new WriteTables( + false, + write.getBigQueryServices(), + jobIdTokenView, + tempFilePrefix, + WriteDisposition.WRITE_EMPTY, + CreateDisposition.CREATE_IF_NEEDED, + schemaFunction)) + .withSideInputs(jobIdTokenView)); // This view maps each final table destination to the set of temporary partitioned tables // the PCollection was loaded into. - PCollectionView>> tempTablesView = tempTables - .apply("TempTablesView", View.asMultimap()); - - singleton.apply("WriteRename", ParDo - .of(new WriteRename( - write.getBigQueryServices(), - jobIdTokenView, - write.getWriteDisposition(), - write.getCreateDisposition(), - tempTablesView)) - .withSideInputs(tempTablesView, jobIdTokenView)); + PCollectionView>> tempTablesView = + tempTables.apply("TempTablesView", View.asMultimap()); + + singleton.apply( + "WriteRename", + ParDo.of( + new WriteRename( + write.getBigQueryServices(), + jobIdTokenView, + write.getWriteDisposition(), + write.getCreateDisposition(), + tempTablesView)) + .withSideInputs(tempTablesView, jobIdTokenView)); // Write single partition to final table - partitions.get(singlePartitionTag) + partitions + .get(singlePartitionTag) .setCoder(partitionsCoder) - .apply("SinglePartitionGroupByKey", - GroupByKey., List>create()) - .apply("SinglePartitionWriteTables", ParDo.of(new WriteTables( - true, - write.getBigQueryServices(), - jobIdTokenView, - tempFilePrefix, - write.getWriteDisposition(), - write.getCreateDisposition(), - schemaFunction)) - .withSideInputs(jobIdTokenView)); + // 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()) + .apply( + "SinglePartitionWriteTables", + ParDo.of( + new WriteTables( + true, + write.getBigQueryServices(), + jobIdTokenView, + tempFilePrefix, + write.getWriteDisposition(), + write.getCreateDisposition(), + schemaFunction)) + .withSideInputs(jobIdTokenView)); 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 846103da8ea6..e04361c960ef 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 @@ -26,6 +26,7 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; +import com.google.common.hash.Hashing; import java.io.IOException; import java.util.ArrayList; @@ -234,6 +235,18 @@ static void verifyTablePresence(DatasetService datasetService, TableReference ta } } + // Create a unique job id for a table load. + static String createJobId(String prefix, TableDestination tableDestination, int partition) { + // Job ID must be different for each partition of each table. + String destinationHash = + Hashing.murmur3_128().hashUnencodedChars(tableDestination.toString()).toString(); + if (partition >= 0) { + return String.format("%s_%s_%05d", prefix, destinationHash, partition); + } else { + return String.format("%s_%s", prefix, destinationHash); + } + } + @VisibleForTesting static class JsonSchemaToTableSchema implements SerializableFunction { 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 54a25c70e177..3f5947e0a5dc 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 @@ -61,7 +61,6 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.PipelineRunner; 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.display.DisplayData; import org.apache.beam.sdk.util.IOChannelFactory; @@ -445,7 +444,8 @@ public void validate(PBegin input) { // 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) { + 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); @@ -650,6 +650,7 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac public static Write write() { return new AutoValue_BigQueryIO_Write.Builder() .setValidate(true) + .setTableDescription("") .setBigQueryServices(new BigQueryServicesImpl()) .setCreateDisposition(Write.CreateDisposition.CREATE_IF_NEEDED) .setWriteDisposition(Write.WriteDisposition.WRITE_EMPTY) @@ -690,7 +691,8 @@ public abstract static class Write extends PTransform, WriteRe @Nullable abstract ValueProvider getJsonSchema(); abstract CreateDisposition getCreateDisposition(); abstract WriteDisposition getWriteDisposition(); - @Nullable abstract String getTableDescription(); + /** 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(); @@ -805,9 +807,6 @@ public Write to(TableReference table) { public Write to(ValueProvider tableSpec) { ensureToNotCalledYet(); String tableDescription = getTableDescription(); - if (tableDescription == null) { - tableDescription = ""; - } return toBuilder() .setJsonTableRef( NestedValueProvider.of( @@ -911,7 +910,7 @@ Write withTestServices(BigQueryServices testServices) { public void validate(PCollection input) { BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class); - // Exactly one of the table and table reference can be configured. + // We must have a destination to write to! checkState(getTableFunction() != null, "must set the table reference of a BigQueryIO.Write transform"); @@ -972,8 +971,8 @@ public void validate(PCollection input) { @Override public WriteResult expand(PCollection input) { PCollection> rowsWithDestination = - input.apply("PrepareWrite", ParDo.of( - new PrepareWrite(getTableFunction(), getFormatFunction()))) + input.apply("PrepareWrite", new PrepareWrite( + getTableFunction(), getFormatFunction())) .setCoder(KvCoder.of(TableDestinationCoder.of(), TableRowJsonCoder.of())); @@ -1013,8 +1012,8 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Table WriteDisposition")) .addIfNotDefault(DisplayData.item("validation", getValidate()) .withLabel("Validation Enabled"), true) - .addIfNotNull(DisplayData.item("tableDescription", getTableDescription()) - .withLabel("Table Description")); + .addIfNotDefault(DisplayData.item("tableDescription", getTableDescription()) + .withLabel("Table Description"), ""); } /** Returns the table schema. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java index 22aba64644ad..a28da9201490 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java @@ -109,8 +109,8 @@ public BoundedReader createReader(PipelineOptions options) throws IOEx @Override public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception { if (tableSizeBytes.get() == null) { - TableReference table = BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), - TableReference.class); + TableReference table = setDefaultProjectIfAbsent(options.as(BigQueryOptions.class), + BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), TableReference.class)); Long numBytes = bqServices.getDatasetService(options.as(BigQueryOptions.class)) .getTable(table).getNumBytes(); 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 e216553d3fae..a78f32d2571e 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 @@ -1,68 +1,94 @@ /* -* 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. -*/ + * 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.Table; 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.annotations.VisibleForTesting; import java.io.IOException; import java.util.Collections; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; + import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.options.BigQueryOptions; 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.values.KV; - +import org.apache.beam.sdk.values.PCollection; /** - * Creates any tables needed before performing streaming writes to the tables. This is a - * side-effect {l@ink DoFn}, and returns the original collection unchanged. + * 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 extends DoFn, - KV> { +public class CreateTables + extends PTransform< + PCollection>, PCollection>> { private final CreateDisposition createDisposition; private final BigQueryServices bqServices; private final SerializableFunction schemaFunction; - - /** The list of tables created so far, so we don't try the creation - each time. - * TODO: We should put a bound on memory usage of this. Use guava cache instead. + /** + * The list of tables created so far, so we don't try the creation each time. + * + *

TODO: We should put a bound on memory usage of this. Use guava cache instead. */ private static Set createdTables = Collections.newSetFromMap(new ConcurrentHashMap()); - public CreateTables(CreateDisposition createDisposition, BigQueryServices bqServices, - SerializableFunction schemaFunction) { + public CreateTables( + CreateDisposition createDisposition, + SerializableFunction schemaFunction) { + this(createDisposition, new BigQueryServicesImpl(), schemaFunction); + } + + private CreateTables( + CreateDisposition createDisposition, + BigQueryServices bqServices, + SerializableFunction schemaFunction) { this.createDisposition = createDisposition; this.bqServices = bqServices; this.schemaFunction = schemaFunction; } - @ProcessElement - public void processElement(ProcessContext context) throws InterruptedException, IOException { - BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - possibleCreateTable(options, context.element().getKey()); - context.output(context.element()); + CreateTables withTestServices(BigQueryServices bqServices) { + return new CreateTables(createDisposition, bqServices, schemaFunction); + } + + @Override + public PCollection> expand( + PCollection> input) { + return input.apply( + ParDo.of( + new DoFn, KV>() { + @ProcessElement + public void processElement(ProcessContext context) + throws InterruptedException, IOException { + BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); + possibleCreateTable(options, context.element().getKey()); + context.output(context.element()); + } + })); } private void possibleCreateTable(BigQueryOptions options, TableDestination tableDestination) @@ -70,8 +96,7 @@ private void possibleCreateTable(BigQueryOptions options, TableDestination table String tableSpec = tableDestination.getTableSpec(); TableReference tableReference = tableDestination.getTableReference(); String tableDescription = tableDestination.getTableDescription(); - if (createDisposition != createDisposition.CREATE_NEVER - && !createdTables.contains(tableSpec)) { + if (createDisposition != createDisposition.CREATE_NEVER && !createdTables.contains(tableSpec)) { synchronized (createdTables) { // Another thread may have succeeded in creating the table in the meanwhile, so // check again. This check isn't needed for correctness, but we add it to prevent @@ -92,6 +117,8 @@ private void possibleCreateTable(BigQueryOptions options, TableDestination table } } + /** This method is used by the testing fake to clear static state. */ + @VisibleForTesting static void clearCreatedTables() { synchronized (createdTables) { createdTables.clear(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java index da3a70a861be..90d41a0778bb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/GenerateShardedTable.java @@ -39,8 +39,7 @@ class GenerateShardedTable extends DoFn, @ProcessElement public void processElement(ProcessContext context, BoundedWindow window) throws IOException { ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); - // We output on keys 0-50 to ensure that there's enough batching for - // BigQuery. + // We output on keys 0-numShards. String tableSpec = context.element().getKey().getTableSpec(); context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, numShards)), context.element().getValue())); 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 771241793757..a8bdb43bcee6 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 @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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.TableReference; @@ -23,6 +23,8 @@ import java.io.IOException; import org.apache.beam.sdk.options.BigQueryOptions; 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.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; @@ -30,37 +32,49 @@ import org.apache.beam.sdk.values.ValueInSingleWindow; /** - * Prepare an input {@link PCollection} for writing to BigQuery. Use the table-reference - * function to determine which tables each element is written to, and format the element into a - * {@link TableRow} using the user-supplied format function. + * Prepare an input {@link PCollection} for writing to BigQuery. Use the table function to determine + * 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 DoFn> { +public class PrepareWrite + extends PTransform, PCollection>> { private SerializableFunction, TableDestination> tableFunction; private SerializableFunction formatFunction; - public PrepareWrite(SerializableFunction, TableDestination> tableFunction, - SerializableFunction formatFunction) { + public PrepareWrite( + SerializableFunction, TableDestination> tableFunction, + SerializableFunction formatFunction) { this.tableFunction = tableFunction; this.formatFunction = formatFunction; } - @ProcessElement - public void processElement(ProcessContext context, BoundedWindow window) throws IOException { - TableDestination tableDestination = tableSpecFromWindowedValue( - context.getPipelineOptions().as(BigQueryOptions.class), - ValueInSingleWindow.of(context.element(), context.timestamp(), window, context.pane())); - TableRow tableRow = formatFunction.apply(context.element()); - context.output(KV.of(tableDestination, tableRow)); + @Override + public PCollection> expand(PCollection input) { + return input.apply( + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext context, BoundedWindow window) + throws IOException { + TableDestination tableDestination = + tableSpecFromWindowedValue( + context.getPipelineOptions().as(BigQueryOptions.class), + ValueInSingleWindow.of( + context.element(), context.timestamp(), window, context.pane())); + TableRow tableRow = formatFunction.apply(context.element()); + context.output(KV.of(tableDestination, tableRow)); + } + })); } - private TableDestination tableSpecFromWindowedValue(BigQueryOptions options, - ValueInSingleWindow value) { + 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()); + 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/ShardedKey.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java index 09b4fbf01191..c2b739f999e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ShardedKey.java @@ -25,6 +25,7 @@ * A key and a shard number. */ class ShardedKey implements Serializable { + private static final long serialVersionUID = 1L; private final K key; private final int shardNumber; 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 ced1d662cdf5..efd9c313614a 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 @@ -22,15 +22,10 @@ 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.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; 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.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -38,8 +33,8 @@ * PTransform that performs streaming BigQuery write. To increase consistency, * it leverages BigQuery best effort de-dup mechanism. */ -class StreamingInserts extends PTransform>, - WriteResult> { +public class StreamingInserts extends + PTransform>, WriteResult> { private final Write write; private static class ConstantSchemaFunction implements @@ -74,36 +69,11 @@ public WriteResult expand(PCollection> input) { SerializableFunction schemaFunction = new ConstantSchemaFunction(write.getSchema()); - // A naive implementation would be to simply stream data directly to BigQuery. - // However, this could occasionally lead to duplicated data, e.g., when - // a VM that runs this code is restarted and the code is re-run. + PCollection> writes = input + .apply("CreateTables", new CreateTables(write.getCreateDisposition(), schemaFunction) + .withTestServices(write.getBigQueryServices())); - // The above risk is mitigated in this implementation by relying on - // BigQuery built-in best effort de-dup mechanism. - - // To use this mechanism, each input TableRow is tagged with a generated - // unique id, which is then passed to BigQuery and used to ignore duplicates. - PCollection, TableRowInfo>> tagged = input - .apply("CreateTables", ParDo.of(new CreateTables(write.getCreateDisposition(), - write.getBigQueryServices(), schemaFunction))) - // We create 50 keys per BigQuery table to generate output on. This is few enough that we - // get good batching into BigQuery's insert calls, and enough that we can max out the - // streaming insert quota. - .apply("ShardTableWrites", ParDo.of(new GenerateShardedTable(50))) - .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowJsonCoder.of())) - .apply("TagWithUniqueIds", ParDo.of(new TagWithUniqueIds())); - - // To prevent having the same TableRow processed more than once with regenerated - // different unique ids, this implementation relies on "checkpointing", which is - // achieved as a side effect of having StreamingWriteFn immediately follow a GBK, - // performed by Reshuffle. - tagged - .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of())) - .apply(Reshuffle., TableRowInfo>of()) - .apply("StreamingWrite", - ParDo.of( - new StreamingWriteFn(write.getBigQueryServices()))); - - return WriteResult.in(input.getPipeline()); + return writes.apply(new StreamingWriteTables() + .withTestServices(write.getBigQueryServices())); } } 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 new file mode 100644 index 000000000000..4ddc1df947ed --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java @@ -0,0 +1,86 @@ +/* + * 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.TableRow; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.TableRowJsonCoder; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.Reshuffle; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * This transform takes in key-value pairs of {@link TableRow} entries and the + * {@link TableDestination} it should be written to. The BigQuery streaming-write service is used + * to stream these writes to the appropriate table. + * + *

This transform assumes that all destination tables already exist by the time it sees a write + * for that table. + */ +public class StreamingWriteTables extends PTransform< + PCollection>, WriteResult> { + private BigQueryServices bigQueryServices; + + public StreamingWriteTables() { + this(new BigQueryServicesImpl()); + } + + private StreamingWriteTables(BigQueryServices bigQueryServices) { + this.bigQueryServices = bigQueryServices; + } + + StreamingWriteTables withTestServices(BigQueryServices bigQueryServices) { + return new StreamingWriteTables(bigQueryServices); + } + + @Override + public WriteResult expand(PCollection> input) { + // A naive implementation would be to simply stream data directly to BigQuery. + // However, this could occasionally lead to duplicated data, e.g., when + // a VM that runs this code is restarted and the code is re-run. + + // The above risk is mitigated in this implementation by relying on + // BigQuery built-in best effort de-dup mechanism. + + // To use this mechanism, each input TableRow is tagged with a generated + // unique id, which is then passed to BigQuery and used to ignore duplicates + // We create 50 keys per BigQuery table to generate output on. This is few enough that we + // get good batching into BigQuery's insert calls, and enough that we can max out the + // streaming insert quota. + PCollection, TableRowInfo>> tagged = + input.apply("ShardTableWrites", ParDo.of + (new GenerateShardedTable(50))) + .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowJsonCoder.of())) + .apply("TagWithUniqueIds", ParDo.of(new TagWithUniqueIds())); + + // To prevent having the same TableRow processed more than once with regenerated + // different unique ids, this implementation relies on "checkpointing", which is + // achieved as a side effect of having StreamingWriteFn immediately follow a GBK, + // performed by Reshuffle. + tagged + .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of())) + .apply(Reshuffle., TableRowInfo>of()) + .apply("StreamingWrite", + ParDo.of( + new StreamingWriteFn(bigQueryServices))); + return WriteResult.in(input.getPipeline()); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index 36e14019bda8..962e2cde7f4d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -27,6 +27,7 @@ * Encapsulates a BigQuery table destination. */ public class TableDestination implements Serializable { + private static final long serialVersionUID = 1L; private final String tableSpec; private final String tableDescription; 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 fa24700472ab..262a00d12d18 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 @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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; @@ -26,20 +26,18 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; -/** - * A coder for {@link TableDestination} objects. - */ +/** A coder for {@link TableDestination} objects. */ public class TableDestinationCoder extends AtomicCoder { private static final TableDestinationCoder INSTANCE = new TableDestinationCoder(); - + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); @JsonCreator public static TableDestinationCoder of() { - return INSTANCE; - } + return INSTANCE; + } @Override - public void encode(TableDestination value, OutputStream outStream, Context context) + public void encode(TableDestination value, OutputStream outStream, Context context) throws IOException { if (value == null) { throw new CoderException("cannot encode a null value"); @@ -50,15 +48,13 @@ public void encode(TableDestination value, OutputStream outStream, Context conte @Override public TableDestination decode(InputStream inStream, Context context) throws IOException { - return new TableDestination( - stringCoder.decode(inStream, context.nested()), - stringCoder.decode(inStream, context.nested())); - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - return; - } + return new TableDestination( + stringCoder.decode(inStream, context.nested()), + stringCoder.decode(inStream, context.nested())); + } - StringUtf8Coder stringCoder = StringUtf8Coder.of(); + @Override + public void verifyDeterministic() throws NonDeterministicException { + return; + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java index ee8f46696865..91ef404bd039 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowWriter.java @@ -32,9 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Writes {@TableRow} objects out to a file. Used when doing batch load jobs into BigQuery. - */ +/** Writes {@TableRow} objects out to a file. Used when doing batch load jobs into BigQuery. */ class TableRowWriter { private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); @@ -47,16 +45,18 @@ class TableRowWriter { protected String mimeType = MimeTypes.TEXT; private CountingOutputStream out; - public class Result { - String filename; - long byteSize; + public static final class Result { + final String filename; + final long byteSize; + public Result(String filename, long byteSize) { this.filename = filename; this.byteSize = byteSize; } } + TableRowWriter(String basename) { - this.tempFilePrefix = basename; + this.tempFilePrefix = basename; } public final void open(String uId) throws Exception { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java index 737978483327..284691e49f55 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java @@ -28,15 +28,14 @@ import org.apache.beam.sdk.values.KV; /** - * Fn that tags each table row with a unique id and destination table. - * To avoid calling UUID.randomUUID() for each element, which can be costly, - * a randomUUID is generated only once per bucket of data. The actual unique - * id is created by concatenating this randomUUID with a sequential number. + * Fn that tags each table row with a unique id and destination table. To avoid calling + * UUID.randomUUID() for each element, which can be costly, a randomUUID is generated only once per + * bucket of data. The actual unique id is created by concatenating this randomUUID with a + * sequential number. */ @VisibleForTesting class TagWithUniqueIds extends DoFn, TableRow>, KV, TableRowInfo>> { - private transient String randomUUID; private transient long sequenceNo = 0L; @@ -51,8 +50,9 @@ public void processElement(ProcessContext context, BoundedWindow window) throws String uniqueId = randomUUID + sequenceNo++; // We output on keys 0-50 to ensure that there's enough batching for // BigQuery. - context.output(KV.of(context.element().getKey(), - new TableRowInfo(context.element().getValue(), uniqueId))); + context.output( + KV.of( + context.element().getKey(), new TableRowInfo(context.element().getValue(), uniqueId))); } @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 869e68a069fa..a25cc90da3f3 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 @@ -51,10 +51,11 @@ class WriteBundlesToFiles extends DoFn, WriteBund * 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 class Result implements Serializable { - public String filename; - public Long fileByteSize; - public TableDestination tableDestination; + 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 Result(String filename, Long fileByteSize, TableDestination tableDestination) { this.filename = filename; @@ -68,6 +69,9 @@ public Result(String filename, Long fileByteSize, TableDestination tableDestinat */ public static class ResultCoder extends AtomicCoder { private static final ResultCoder INSTANCE = new ResultCoder(); + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); + private static final VarLongCoder longCoder = VarLongCoder.of(); + private static final TableDestinationCoder tableDestinationCoder = TableDestinationCoder.of(); public static ResultCoder of() { return INSTANCE; @@ -87,18 +91,15 @@ public void encode(Result value, OutputStream outStream, Context context) @Override public Result decode(InputStream inStream, Context context) throws IOException { - return new Result(stringCoder.decode(inStream, context.nested()), - longCoder.decode(inStream, context.nested()), - tableDestinationCoder.decode(inStream, context.nested())); + 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); } @Override public void verifyDeterministic() throws NonDeterministicException { } - - StringUtf8Coder stringCoder = StringUtf8Coder.of(); - VarLongCoder longCoder = VarLongCoder.of(); - TableDestinationCoder tableDestinationCoder = TableDestinationCoder.of(); } WriteBundlesToFiles(String tempFilePrefix) { @@ -107,6 +108,8 @@ public void verifyDeterministic() throws NonDeterministicException { @StartBundle public void startBundle(Context c) { + // This must be done each bundle, as by default the {@link DoFn} might be reused between + // bundles. this.writers = Maps.newHashMap(); } 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 9c48b8233aea..9414909acd45 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 @@ -44,7 +44,65 @@ class WritePartition extends DoFn, List< private TupleTag, List>> multiPartitionsTag; private TupleTag, List>> singlePartitionTag; - public WritePartition( + private static class PartitionData { + private int numFiles = 0; + private long byteSize = 0; + private List filenames = Lists.newArrayList(); + + int getNumFiles() { + return numFiles; + } + + void addFiles(int numFiles) { + this.numFiles += numFiles; + } + + long getByteSize() { + return byteSize; + } + + void addBytes(long numBytes) { + this.byteSize += numBytes; + } + + List getFilenames() { + return filenames; + } + + void addFilename(String filename) { + filenames.add(filename); + } + + // Check to see whether we can add to this partition without exceeding the maximum partition + // size. + boolean canAccept(int numFiles, long numBytes) { + return this.numFiles + numFiles <= Write.MAX_NUM_FILES + && this.byteSize + numBytes <= Write.MAX_SIZE_BYTES; + } + } + + private static class DestinationData { + private List partitions = Lists.newArrayList(); + + DestinationData() { + // Always start out with a single empty partition. + partitions.add(new PartitionData()); + } + + List getPartitions() { + return partitions; + } + + PartitionData getLatestPartition() { + return partitions.get(partitions.size() - 1); + } + + void addPartition(PartitionData partition) { + partitions.add(partition); + } + } + + WritePartition( ValueProvider singletonOutputJsonTableRef, String singletonOutputTableDescription, PCollectionView> resultsView, @@ -76,54 +134,41 @@ public void processElement(ProcessContext c) throws Exception { } - long partitionId = 0; - Map currNumFilesMap = Maps.newHashMap(); - Map currSizeBytesMap = Maps.newHashMap(); - Map>> currResultsMap = Maps.newHashMap(); - for (int i = 0; i < results.size(); ++i) { - WriteBundlesToFiles.Result fileResult = results.get(i); + Map currentResults = Maps.newHashMap(); + for (WriteBundlesToFiles.Result fileResult : results) { TableDestination tableDestination = fileResult.tableDestination; - List> partitions = currResultsMap.get(tableDestination); - if (partitions == null) { - partitions = Lists.newArrayList(); - partitions.add(Lists.newArrayList()); - currResultsMap.put(tableDestination, partitions); + DestinationData destinationData = currentResults.get(tableDestination); + if (destinationData == null) { + destinationData = new DestinationData(); + currentResults.put(tableDestination, destinationData); } - int currNumFiles = getOrDefault(currNumFilesMap, tableDestination, 0); - long currSizeBytes = getOrDefault(currSizeBytesMap, tableDestination, 0L); - if (currNumFiles + 1 > Write.MAX_NUM_FILES - || currSizeBytes + fileResult.fileByteSize > Write.MAX_SIZE_BYTES) { - // Add a new partition for this table. - partitions.add(Lists.newArrayList()); - // c.sideOutput(multiPartitionsTag, KV.of(++partitionId, currResults)); - currNumFiles = 0; - currSizeBytes = 0; - currNumFilesMap.remove(tableDestination); - currSizeBytesMap.remove(tableDestination); + + PartitionData latestPartition = destinationData.getLatestPartition(); + if (!latestPartition.canAccept(1, fileResult.fileByteSize)) { + // Too much data, roll over to a new partition. + latestPartition = new PartitionData(); + destinationData.addPartition(latestPartition); } - currNumFilesMap.put(tableDestination, currNumFiles + 1); - currSizeBytesMap.put(tableDestination, currSizeBytes + fileResult.fileByteSize); - // Always add to the most recent partition for this table. - partitions.get(partitions.size() - 1).add(fileResult.filename); + latestPartition.addFilename(fileResult.filename); + latestPartition.addFiles(1); + latestPartition.addBytes(fileResult.fileByteSize); } - for (Map.Entry>> entry : currResultsMap.entrySet()) { + // 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(); - List> partitions = entry.getValue(); + 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 = - (partitions.size() == 1) ? singlePartitionTag : multiPartitionsTag; - for (int i = 0; i < partitions.size(); ++i) { - c.output(outputTag, KV.of(ShardedKey.of(tableDestination, i + 1), partitions.get(i))); + (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())); } } } - - private T getOrDefault(Map map, TableDestination tableDestination, - T defaultValue) { - if (map.containsKey(tableDestination)) { - return map.get(tableDestination); - } else { - return defaultValue; - } - } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index 752e7d3b63b8..9b1c989f96b4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -89,8 +89,9 @@ public void processElement(ProcessContext c) throws Exception { } // Make sure each destination table gets a unique job id. - String jobIdPrefix = String.format( - c.sideInput(jobIdToken) + "0x%08x", finalTableDestination.hashCode()); + String jobIdPrefix = BigQueryHelpers.createJobId( + c.sideInput(jobIdToken), finalTableDestination, -1); + copy( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), 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 f7fe87b9747d..4a6cd2b80aa0 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 @@ -57,11 +57,15 @@ /** * Writes partitions to BigQuery tables. * - *

The input is a list of files corresponding to a partition of a table. These files are + *

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 the final table to the temporary tables for each partition of that table. + * 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, Iterable>>, +class WriteTables extends DoFn, List>, KV> { private static final Logger LOG = LoggerFactory.getLogger(WriteTables.class); @@ -94,10 +98,9 @@ public WriteTables( public void processElement(ProcessContext c) throws Exception { TableDestination tableDestination = c.element().getKey().getKey(); Integer partition = c.element().getKey().getShardNumber(); - List partitionFiles = Lists.newArrayList(c.element().getValue()).get(0); - // Job ID must be different for each partition of each table. - String jobIdPrefix = String.format( - c.sideInput(jobIdToken) + "_0x%08x_%05d", tableDestination.hashCode(), partition); + List partitionFiles = Lists.newArrayList(c.element().getValue()); + String jobIdPrefix = BigQueryHelpers.createJobId( + c.sideInput(jobIdToken), tableDestination, partition); TableReference ref = tableDestination.getTableReference(); if (!singlePartition) { 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 f10be13c85e1..d0004e41b7ad 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 @@ -71,6 +71,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.TableRowJsonCoder; @@ -122,7 +123,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; @@ -607,13 +607,11 @@ public PartitionedGlobalWindow decode(InputStream inStream, Context context) } @Test - @Category(NeedsRunner.class) public void testStreamingWriteWithDynamicTables() throws Exception { testWriteWithDynamicTables(true); } @Test - @Category(NeedsRunner.class) public void testBatchWriteWithDynamicTables() throws Exception { testWriteWithDynamicTables(false); } @@ -842,7 +840,7 @@ public void testBuildWrite() { BigQueryIO.writeTableRows().to("foo.com:project:somedataset.sometable"); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -894,7 +892,7 @@ public void testBuildWriteWithoutValidation() { null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, - null, + "", false); } @@ -905,7 +903,7 @@ public void testBuildWriteDefaultProject() { checkWriteObject( write, null, "somedataset", "sometable", null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, - null); + ""); } @Test @@ -917,7 +915,7 @@ public void testBuildWriteWithTableReference() { BigQueryIO.Write write = BigQueryIO.writeTableRows().to(table); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -927,7 +925,7 @@ public void testBuildWriteWithSchema() { BigQueryIO.write().to("foo.com:project:somedataset.sometable").withSchema(schema); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, null); + schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -937,7 +935,7 @@ public void testBuildWriteWithCreateDispositionNever() { .withCreateDisposition(CreateDisposition.CREATE_NEVER); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY, null); + null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -947,7 +945,7 @@ public void testBuildWriteWithCreateDispositionIfNeeded() { .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -957,7 +955,7 @@ public void testBuildWriteWithWriteDispositionTruncate() { .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE, ""); } @Test @@ -967,7 +965,7 @@ public void testBuildWriteWithWriteDispositionAppend() { .withWriteDisposition(WriteDisposition.WRITE_APPEND); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND, ""); } @Test @@ -977,7 +975,7 @@ public void testBuildWriteWithWriteDispositionEmpty() { .withWriteDisposition(WriteDisposition.WRITE_EMPTY); checkWriteObject( write, "foo.com:project", "somedataset", "sometable", - null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, null); + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, ""); } @Test @@ -1359,7 +1357,6 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { SourceTestUtils.assertSplitAtFractionBehavior( bqSource, 2, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); - List> sources = bqSource.split(100, options); assertEquals(2, sources.size()); BoundedSource actual = sources.get(0); @@ -1626,9 +1623,11 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file TupleTag, List>> singlePartitionTag = new TupleTag, List>>("singlePartitionTag") {}; + PCollection filesPCollection = + p.apply(Create.of(files).withType(new TypeDescriptor() {})); PCollectionView> resultsView = PCollectionViews.iterableView( - p, + filesPCollection, WindowingStrategy.globalDefault(), WriteBundlesToFiles.ResultCoder.of()); @@ -1699,14 +1698,12 @@ public void testWriteTables() throws Exception { Path baseDir = Files.createTempDirectory(tempFolder, "testWriteTables"); - List, Iterable>>> 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); for (int j = 0; j < numPartitions; ++j) { - String tempTableId = String.format( - jobIdToken + "_0x%08x_%05d", tableDestination.hashCode(), j); + String tempTableId = BigQueryHelpers.createJobId(jobIdToken, tableDestination, j); List filesPerPartition = Lists.newArrayList(); for (int k = 0; k < numFilesPerPartition; ++k) { String filename = Paths.get(baseDir.toString(), @@ -1721,7 +1718,7 @@ public void testWriteTables() throws Exception { filesPerPartition.add(filename); } partitions.add(KV.of(ShardedKey.of(tableDestination, j), - (Iterable>) Collections.singleton(filesPerPartition))); + filesPerPartition)); List expectedTables = expectedTempTables.get(tableDestination); if (expectedTables == null) { @@ -1735,11 +1732,6 @@ public void testWriteTables() throws Exception { } } - PCollection expectedTempTablesPCollection = p.apply(Create.of(expectedTempTables)); - PCollectionView> tempTablesView = PCollectionViews.iterableView( - expectedTempTablesPCollection, - WindowingStrategy.globalDefault(), - StringUtf8Coder.of()); PCollection jobIdTokenCollection = p.apply("CreateJobId", Create.of("jobId")); PCollectionView jobIdTokenView = jobIdTokenCollection.apply(View.asSingleton()); @@ -1753,10 +1745,10 @@ public void testWriteTables() throws Exception { CreateDisposition.CREATE_IF_NEEDED, null); - DoFnTester, Iterable>>, + DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); - for (KV, Iterable>> partition : partitions) { + for (KV, List> partition : partitions) { tester.processElement(partition); } @@ -1848,11 +1840,27 @@ public void testWriteRename() throws Exception { } } + PCollection> tempTablesPCollection = + p.apply(Create.of(tempTables) + .withCoder(KvCoder.of(TableDestinationCoder.of(), + IterableCoder.of(StringUtf8Coder.of())))) + .apply(ParDo.of(new DoFn>, + KV>() { + @ProcessElement + public void processElement(ProcessContext c) { + TableDestination tableDestination = c.element().getKey(); + for (String tempTable : c.element().getValue()) { + c.output(KV.of(tableDestination, tempTable)); + } + } + })); + PCollectionView>> tempTablesView = PCollectionViews.multimapView( - p, + tempTablesPCollection, WindowingStrategy.globalDefault(), - KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of())); + KvCoder.of(TableDestinationCoder.of(), + StringUtf8Coder.of())); PCollection jobIdTokenCollection = p.apply("CreateJobId", Create.of("jobId")); PCollectionView jobIdTokenView =