ahmedabu98 commented on code in PR #38058:
URL: https://github.com/apache/beam/pull/38058#discussion_r3125629964
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java:
##########
@@ -2513,6 +2515,180 @@ public void updateTableSchemaTest(boolean useSet)
throws Exception {
Iterables.concat(expectedDroppedValues, expectedFullValues),
TableRow.class)));
}
+ @Test
+ public void testAutoPatchTableSchemaTest() throws Exception {
+ assumeTrue(useStreaming);
+ assumeTrue(useStorageApi);
+
+ // Make sure that GroupIntoBatches does not buffer data.
+
p.getOptions().as(BigQueryOptions.class).setStorageApiAppendThresholdBytes(1);
+ p.getOptions().as(BigQueryOptions.class).setNumStorageWriteApiStreams(1);
+
p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(2);
+
+ BigQueryIO.Write.Method method =
+ useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE :
Method.STORAGE_WRITE_API;
+ p.enableAbandonedNodeEnforcement(false);
+
+ TableReference tableRef =
BigQueryHelpers.parseTableSpec("project-id:dataset-id.table");
+ TableSchema tableSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("number").setType("INT64"),
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED")));
+ fakeDatasetService.createTable(new
Table().setTableReference(tableRef).setSchema(tableSchema));
+
+ final int stride = 5;
+ Function<Integer, TableSchema> getUpdatedSchema =
+ currentStride -> {
+ TableSchema tableSchemaUpdated = new TableSchema();
Review Comment:
I'm noticing that this function only gets checked for `Integer.MAX_VALUE` to
validate the final schema.
Is there a way to view the table update history and validate that
incremental updates occurred?
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java:
##########
@@ -2513,6 +2515,180 @@ public void updateTableSchemaTest(boolean useSet)
throws Exception {
Iterables.concat(expectedDroppedValues, expectedFullValues),
TableRow.class)));
}
+ @Test
+ public void testAutoPatchTableSchemaTest() throws Exception {
+ assumeTrue(useStreaming);
+ assumeTrue(useStorageApi);
+
+ // Make sure that GroupIntoBatches does not buffer data.
+
p.getOptions().as(BigQueryOptions.class).setStorageApiAppendThresholdBytes(1);
+ p.getOptions().as(BigQueryOptions.class).setNumStorageWriteApiStreams(1);
+
p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(2);
+
+ BigQueryIO.Write.Method method =
+ useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE :
Method.STORAGE_WRITE_API;
+ p.enableAbandonedNodeEnforcement(false);
+
+ TableReference tableRef =
BigQueryHelpers.parseTableSpec("project-id:dataset-id.table");
+ TableSchema tableSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("number").setType("INT64"),
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED")));
+ fakeDatasetService.createTable(new
Table().setTableReference(tableRef).setSchema(tableSchema));
+
+ final int stride = 5;
+ Function<Integer, TableSchema> getUpdatedSchema =
+ currentStride -> {
+ TableSchema tableSchemaUpdated = new TableSchema();
+ tableSchemaUpdated.setFields(
+ Lists.newArrayList(
+ new
TableFieldSchema().setName("number").setType("INT64").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("name").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new1").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new2").setType("STRING").setMode("NULLABLE")));
+
+ if (currentStride >= 2) {
+ List<TableFieldSchema> nestedFields =
+ Lists.newArrayList(
+ new TableFieldSchema()
+ .setName("nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE"));
+ if (currentStride >= 3) {
+ List<TableFieldSchema> doubleNestedFields =
+ Lists.newArrayList(
+ new TableFieldSchema()
+ .setName("double_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"));
+ nestedFields.add(
+ new TableFieldSchema()
+ .setName("double_nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(doubleNestedFields));
+
+ List<TableFieldSchema> repeatedNestedFields =
+ Lists.newArrayList(
+ new TableFieldSchema()
+ .setName("repeated_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("repeated_nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE"));
+
+ nestedFields.add(
+ new TableFieldSchema()
+ .setName("repeated_nested")
+ .setType("STRUCT")
+ .setMode("REPEATED")
+ .setFields(repeatedNestedFields));
+ }
+ tableSchemaUpdated
+ .getFields()
+ .add(
+ new TableFieldSchema()
+ .setName("nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(nestedFields));
+ }
+ return tableSchemaUpdated;
+ };
+
+ IntFunction<TableRow> getRow =
+ (IntFunction<TableRow> & Serializable)
+ (int i) -> {
+ TableRow row = new TableRow().set("name", "name" +
i).set("number", Long.toString(i));
+ if (i < stride) {
+ row = row.set("req", "foo");
+ } else {
+ row = row.set("new1", "blah" + i);
+ row = row.set("new2", "baz" + i);
+
+ if (i >= 2 * stride) {
+ TableRow nested =
+ new TableRow()
+ .set("nested_field1", "nested1" + i)
+ .set("nested_field2", "nested2" + i);
+
+ if (i >= 3 * stride) {
+ TableRow doubleNested =
+ new TableRow().set("double_nested_field1",
"double_nested1" + i);
+ nested = nested.set("double_nested", doubleNested);
+
+ // Add a repeated struct to ensure that we capture this
code path as well.
+ TableRow repeatedNested1 =
+ new TableRow().set("repeated_nested_field1",
"repeated_nested1" + i);
+ TableRow repeatedNested2 =
+ new TableRow().set("repeated_nested_field2",
"repeated_nested2" + i);
+ nested =
+ nested.set(
+ "repeated_nested",
ImmutableList.of(repeatedNested1, repeatedNested2));
+ }
+ row.set("nested", nested);
+ }
+ }
+ return row;
+ };
+
+ TestStream.Builder<TableRow> testStream =
+ TestStream.create(TableRowJsonCoder.of()).advanceWatermarkTo(new
Instant(0));
+ List<TableRow> expectedRows = Lists.newArrayList();
+ for (int i = 0; i < 20; i += stride) {
+ for (int j = i; j < i + stride; ++j) {
+ TableRow tableRow = getRow.apply(j);
+ expectedRows.add(tableRow);
+ testStream = testStream.addElements(tableRow);
+ }
+ if (i > 0 && (i % 5) == 0) {
+ for (int n = 0; n < 5; ++n) {
+ testStream =
testStream.advanceProcessingTime(Duration.standardSeconds(2));
+ }
+ }
+ }
+ for (int i = 0; i < 5; ++i) {
+ testStream =
testStream.advanceProcessingTime(Duration.standardSeconds(2));
+ }
Review Comment:
Why is this needed?
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDataTriggeredSchemaUpdateIT.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull;
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+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.auto.value.AutoValue;
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import org.joda.time.Duration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Integration test for BigQueryIO Storage Write API auto schema upgrade
triggered by data.
+ *
+ * <p>Uses a Stateful DoFn to sequence elements ensuring base schema elements
are written before
+ * evolved schema elements, avoiding race conditions in distributed execution.
+ */
+@RunWith(JUnit4.class)
+public class StorageApiDataTriggeredSchemaUpdateIT {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(StorageApiDataTriggeredSchemaUpdateIT.class);
+
+ private static final BigqueryClient BQ_CLIENT =
+ new BigqueryClient("StorageApiDataTriggeredSchemaUpdateIT");
+ private static final String PROJECT =
+ TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+ private static final String BIG_QUERY_DATASET_ID =
+ "storage_api_data_triggered_schema_update_" + System.nanoTime();
+
+ private static String bigQueryLocation;
+
+ @Rule public TestName testName = new TestName();
+
+ @BeforeClass
+ public static void setUpTestEnvironment() throws IOException,
InterruptedException {
+ bigQueryLocation =
+
TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation();
+ BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null,
bigQueryLocation);
+ }
+
+ @AfterClass
+ public static void cleanUp() {
+ LOG.info("Cleaning up dataset {} and tables.", BIG_QUERY_DATASET_ID);
+ BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+ }
+
+ private String createTable(TableSchema tableSchema) throws IOException,
InterruptedException {
+ String tableId = testName.getMethodName().replace("[", "_").replace("]",
"_");
+ BQ_CLIENT.deleteTable(PROJECT, BIG_QUERY_DATASET_ID, tableId);
+ BQ_CLIENT.createNewTable(
+ PROJECT,
+ BIG_QUERY_DATASET_ID,
+ new Table()
+ .setSchema(tableSchema)
+ .setTableReference(
+ new TableReference()
+ .setTableId(tableId)
+ .setDatasetId(BIG_QUERY_DATASET_ID)
+ .setProjectId(PROJECT)));
+ return tableId;
+ }
+
+ static class SequenceRowsDoFn extends DoFn<KV<Integer, Integer>, TableRow> {
+ private static final String COUNTER = "counter";
+
+ @StateId(COUNTER)
+ @SuppressWarnings("unused")
+ private final StateSpec<ValueState<Integer>> counterSpec =
StateSpecs.value();
+
+ private final int stride;
+
+ public SequenceRowsDoFn(int stride) {
+ this.stride = stride;
+ }
+
+ @ProcessElement
+ public void processElement(ProcessContext c, @StateId(COUNTER)
ValueState<Integer> counter) {
+ int current = firstNonNull(counter.read(), 0);
+ c.output(getRow(current));
+ counter.write(++current);
+ }
+
+ TableRow getRow(int i) {
+ TableRow row = new TableRow().set("name", "name" + i).set("number",
Long.toString(i));
+ if (i < stride) {
+ row = row.set("req", "foo");
+ } else {
+ row = row.set("new1", "blah" + i);
+ row = row.set("new2", "baz" + i);
+
+ if (i >= 2 * stride) {
+ TableRow nested =
+ new TableRow()
+ .set("nested_field1", "nested1" + i)
+ .set("nested_field2", "nested2" + i);
+
+ if (i >= 3 * stride) {
+ TableRow doubleNested =
+ new TableRow().set("double_nested_field1", "double_nested1" +
i);
+ nested = nested.set("double_nested", doubleNested);
+
+ // Add a repeated struct to ensure that we capture this code path
as well.
+ TableRow repeatedNested1 =
+ new TableRow().set("repeated_nested_field1",
"repeated_nested1" + i);
+ TableRow repeatedNested2 =
+ new TableRow().set("repeated_nested_field2",
"repeated_nested2" + i);
+ nested =
+ nested.set("repeated_nested",
ImmutableList.of(repeatedNested1, repeatedNested2));
+ }
+ row.set("nested", nested);
+ }
+ }
+ return row;
+ };
+ }
+
+ @Test
+ public void testDataTriggeredSchemaUpgradeExactlyOnce() throws Exception {
+ runTest(Write.Method.STORAGE_WRITE_API);
+ }
+
+ @Test
+ public void testDataTriggeredSchemaUpgradeAtLeastOnce() throws Exception {
+ runTest(Write.Method.STORAGE_API_AT_LEAST_ONCE);
+ }
+
+ private void runTest(Write.Method method) throws Exception {
+ Pipeline p = Pipeline.create(TestPipeline.testingPipelineOptions());
+
p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(1);
+
+ TableSchema baseSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("number").setType("INT64"),
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED")));
+ TableSchema evolvedSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new
TableFieldSchema().setName("number").setType("INT64").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("name").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new1").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new2").setType("STRING").setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+ .setName("nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("double_nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+
.setName("double_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"))),
+ new TableFieldSchema()
+ .setName("repeated_nested")
+ .setType("STRUCT")
+ .setMode("REPEATED")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+
.setName("repeated_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+
.setName("repeated_nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE")))))));
+
+ String tableId = createTable(baseSchema);
+ String tableSpec = PROJECT + ":" + BIG_QUERY_DATASET_ID + "." + tableId;
+
+ List<Integer> dummyInputs = IntStream.range(0,
20).boxed().collect(Collectors.toList());
+
+ BigQueryIO.Write<TableRow> write =
+ BigQueryIO.writeTableRows()
+ .to(tableSpec)
+ .withMethod(method)
+ .withSchemaUpdateOptions(
+ ImmutableSet.of(
+ Write.SchemaUpdateOption.ALLOW_FIELD_ADDITION,
+ Write.SchemaUpdateOption.ALLOW_FIELD_RELAXATION))
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER)
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND);
+ if (method == Write.Method.STORAGE_WRITE_API) {
+ write =
+ write
+ .withTriggeringFrequency(Duration.standardSeconds(1))
+ .withNumStorageWriteApiStreams(2);
+ }
+
+ p.apply("Create Dummy Inputs", Create.of(dummyInputs))
+ .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED)
+ .apply("Add a dummy key", WithKeys.of(1))
+ .apply("Sequence Rows", ParDo.of(new SequenceRowsDoFn(5)))
+ .apply("Stream to BigQuery", write);
+
+ p.run().waitUntilFinish();
+
+ List<TableRow> response =
+ BQ_CLIENT.queryUnflattened(
+ String.format("SELECT * FROM [%s] ORDER BY number", tableSpec),
+ PROJECT,
+ true,
+ false,
+ bigQueryLocation);
+ System.err.println("FULL RESULT " + response);
Review Comment:
nit: cleanup
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchemaTest.java:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+
+import com.google.cloud.bigquery.storage.v1.TableFieldSchema;
+import com.google.cloud.bigquery.storage.v1.TableSchema;
+import com.google.protobuf.DescriptorProtos;
+import com.google.protobuf.Descriptors;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class UpgradeTableSchemaTest {
+
+ @Test
+ public void testNewErrorCollector() throws Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+
+ // Test exceptions that should be collected
+ TableRowToStorageApiProto.SchemaTooNarrowException tooNarrow =
+ new TableRowToStorageApiProto.SchemaTooNarrowException("field",
"error", false, false);
+ TableRowToStorageApiProto.SchemaMissingRequiredFieldException
missingRequired =
+ new
TableRowToStorageApiProto.SchemaMissingRequiredFieldException(Sets.newHashSet("field"));
+
+ collector.collect(tooNarrow);
+ collector.collect(missingRequired);
+
+ assertEquals(2, collector.getExceptions().size());
+
+ // Test exception that should not be collected
+ TableRowToStorageApiProto.SchemaConversionException randomException =
+ new TableRowToStorageApiProto.SchemaConversionException("generic
error") {};
+ assertThrows(
+ TableRowToStorageApiProto.SchemaConversionException.class,
+ () -> {
+ collector.collect(randomException);
+ });
+ }
+
+ @Test
+ public void testGetIncrementalSchema_SchemaTooNarrow() throws Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+ TableRowToStorageApiProto.SchemaTooNarrowException tooNarrow =
+ new TableRowToStorageApiProto.SchemaTooNarrowException("new_field",
"error", false, false);
+ collector.collect(tooNarrow);
+
+ TableSchema oldSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("old_field")
+ .setType(TableFieldSchema.Type.STRING))
+ .build();
+
+ TableSchema incrementalSchema =
UpgradeTableSchema.getIncrementalSchema(collector, oldSchema);
+ assertEquals(1, incrementalSchema.getFieldsCount());
+ assertEquals("new_field", incrementalSchema.getFields(0).getName());
+ assertEquals(TableFieldSchema.Mode.NULLABLE,
incrementalSchema.getFields(0).getMode());
+ assertEquals(TableFieldSchema.Type.STRING,
incrementalSchema.getFields(0).getType());
+ }
+
+ @Test
+ public void testGetIncrementalSchema_SchemaMissingRequiredField() throws
Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+ TableRowToStorageApiProto.SchemaMissingRequiredFieldException
missingRequired =
+ new TableRowToStorageApiProto.SchemaMissingRequiredFieldException(
+ Sets.newHashSet("required_field"));
+ collector.collect(missingRequired);
+
+ TableSchema oldSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("required_field")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .build();
+
+ TableSchema incrementalSchema =
UpgradeTableSchema.getIncrementalSchema(collector, oldSchema);
+ assertEquals(1, incrementalSchema.getFieldsCount());
+ assertEquals("required_field", incrementalSchema.getFields(0).getName());
+ assertEquals(TableFieldSchema.Mode.NULLABLE,
incrementalSchema.getFields(0).getMode());
Review Comment:
Also check that type is preserved
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java:
##########
@@ -2394,4 +2554,142 @@ public void testCdcFields() throws Exception {
assertEquals(
Long.toHexString(42L),
msg.getField(fieldDescriptors.get(StorageApiCDC.CHANGE_SQN_COLUMN)));
}
+
+ @Test
+ public void testTableSchemaHash() {
+ com.google.cloud.bigquery.storage.v1.TableSchema schema1 =
+ com.google.cloud.bigquery.storage.v1.TableSchema.newBuilder()
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field1")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.NULLABLE)
+ .build())
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field2")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .build();
+
+ com.google.cloud.bigquery.storage.v1.TableSchema schemaNameDiff =
+ com.google.cloud.bigquery.storage.v1.TableSchema.newBuilder()
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field1_diff")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.NULLABLE)
+ .build())
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field2")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .build();
+
+ com.google.cloud.bigquery.storage.v1.TableSchema schemaTypeDiff =
+ com.google.cloud.bigquery.storage.v1.TableSchema.newBuilder()
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field1")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.NULLABLE)
+ .build())
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field2")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .build();
+
+ com.google.cloud.bigquery.storage.v1.TableSchema schemaModeDiff =
+ com.google.cloud.bigquery.storage.v1.TableSchema.newBuilder()
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field1")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field2")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .build();
+
+ com.google.cloud.bigquery.storage.v1.TableSchema schemaOrderDiff =
+ com.google.cloud.bigquery.storage.v1.TableSchema.newBuilder()
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field2")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)
+ .build())
+ .addFields(
+
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+ .setName("field1")
+
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+
.setMode(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.NULLABLE)
+ .build())
+ .build();
+
+ byte[] hash1 = TableRowToStorageApiProto.tableSchemaHash(schema1);
+ byte[] hashNameDiff =
TableRowToStorageApiProto.tableSchemaHash(schemaNameDiff);
+ byte[] hashTypeDiff =
TableRowToStorageApiProto.tableSchemaHash(schemaTypeDiff);
+ byte[] hashModeDiff =
TableRowToStorageApiProto.tableSchemaHash(schemaModeDiff);
+ byte[] hashOrderDiff =
TableRowToStorageApiProto.tableSchemaHash(schemaOrderDiff);
+
Review Comment:
Maybe also add for sanity check
```suggestion
byte[] hash2 = TableRowToStorageApiProto.tableSchemaHash(schema1);
assertTrue(Arrays.equals(hash1, hash2));
```
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchemaTest.java:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+
+import com.google.cloud.bigquery.storage.v1.TableFieldSchema;
+import com.google.cloud.bigquery.storage.v1.TableSchema;
+import com.google.protobuf.DescriptorProtos;
+import com.google.protobuf.Descriptors;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class UpgradeTableSchemaTest {
+
+ @Test
+ public void testNewErrorCollector() throws Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+
+ // Test exceptions that should be collected
+ TableRowToStorageApiProto.SchemaTooNarrowException tooNarrow =
+ new TableRowToStorageApiProto.SchemaTooNarrowException("field",
"error", false, false);
+ TableRowToStorageApiProto.SchemaMissingRequiredFieldException
missingRequired =
+ new
TableRowToStorageApiProto.SchemaMissingRequiredFieldException(Sets.newHashSet("field"));
+
+ collector.collect(tooNarrow);
+ collector.collect(missingRequired);
+
+ assertEquals(2, collector.getExceptions().size());
+
+ // Test exception that should not be collected
+ TableRowToStorageApiProto.SchemaConversionException randomException =
+ new TableRowToStorageApiProto.SchemaConversionException("generic
error") {};
+ assertThrows(
+ TableRowToStorageApiProto.SchemaConversionException.class,
+ () -> {
+ collector.collect(randomException);
+ });
+ }
+
+ @Test
+ public void testGetIncrementalSchema_SchemaTooNarrow() throws Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+ TableRowToStorageApiProto.SchemaTooNarrowException tooNarrow =
+ new TableRowToStorageApiProto.SchemaTooNarrowException("new_field",
"error", false, false);
+ collector.collect(tooNarrow);
+
+ TableSchema oldSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("old_field")
+ .setType(TableFieldSchema.Type.STRING))
+ .build();
+
+ TableSchema incrementalSchema =
UpgradeTableSchema.getIncrementalSchema(collector, oldSchema);
+ assertEquals(1, incrementalSchema.getFieldsCount());
+ assertEquals("new_field", incrementalSchema.getFields(0).getName());
+ assertEquals(TableFieldSchema.Mode.NULLABLE,
incrementalSchema.getFields(0).getMode());
+ assertEquals(TableFieldSchema.Type.STRING,
incrementalSchema.getFields(0).getType());
+ }
+
+ @Test
+ public void testGetIncrementalSchema_SchemaMissingRequiredField() throws
Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+ TableRowToStorageApiProto.SchemaMissingRequiredFieldException
missingRequired =
+ new TableRowToStorageApiProto.SchemaMissingRequiredFieldException(
+ Sets.newHashSet("required_field"));
+ collector.collect(missingRequired);
+
+ TableSchema oldSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("required_field")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .build();
+
+ TableSchema incrementalSchema =
UpgradeTableSchema.getIncrementalSchema(collector, oldSchema);
+ assertEquals(1, incrementalSchema.getFieldsCount());
+ assertEquals("required_field", incrementalSchema.getFields(0).getName());
+ assertEquals(TableFieldSchema.Mode.NULLABLE,
incrementalSchema.getFields(0).getMode());
+ }
+
+ @Test
+ public void testGetIncrementalSchema_NestedFields() throws Exception {
+ TableRowToStorageApiProto.ErrorCollector collector =
UpgradeTableSchema.newErrorCollector();
+
+ // 1. Add a nested field
+ TableRowToStorageApiProto.SchemaTooNarrowException tooNarrow =
+ new TableRowToStorageApiProto.SchemaTooNarrowException(
+ "nested.new_field", "error", false, false);
+ collector.collect(tooNarrow);
+
+ // 2. Relax a nested field
+ TableRowToStorageApiProto.SchemaMissingRequiredFieldException
missingRequired =
+ new TableRowToStorageApiProto.SchemaMissingRequiredFieldException(
+ Sets.newHashSet("nested.required_field"));
+ collector.collect(missingRequired);
+
+ TableSchema oldSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("top")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("nested")
+ .setType(TableFieldSchema.Type.STRUCT)
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("required_field")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED)))
+ .build();
+
+ TableSchema incrementalSchema =
UpgradeTableSchema.getIncrementalSchema(collector, oldSchema);
+
+ TableSchema expected =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("nested")
+ .setType(TableFieldSchema.Type.STRUCT)
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("required_field")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("new_field")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE)))
+ .build();
+ assertEquals(expected, incrementalSchema);
+ }
+
+ @Test
+ public void testMergeSchemas() {
+ TableSchema schema1 =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f3")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .build();
+ TableSchema schema2 =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f2")
+ .setType(TableFieldSchema.Type.INT64)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .build();
+
+ TableSchema merged = UpgradeTableSchema.mergeSchemas(schema1, schema2);
+
+ TableSchema expectedSchema =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f3")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("f2")
+ .setType(TableFieldSchema.Type.INT64)
+ .setMode(TableFieldSchema.Mode.REQUIRED))
+ .build();
+ assertEquals(expectedSchema, merged);
+ assertEquals(3, merged.getFieldsCount());
+ }
+
+ @Test
+ public void testMergeSchemasConflict() {
+ TableSchema schema1 =
+ TableSchema.newBuilder()
+ .addFields(
+
TableFieldSchema.newBuilder().setName("f1").setType(TableFieldSchema.Type.STRING))
+ .build();
+ TableSchema schema2 =
+ TableSchema.newBuilder()
+ .addFields(
+
TableFieldSchema.newBuilder().setName("f1").setType(TableFieldSchema.Type.INT64))
+ .build();
+
+ assertThrows(
+ IllegalArgumentException.class,
+ () -> {
+ UpgradeTableSchema.mergeSchemas(schema1, schema2);
+ });
+ }
+
+ @Test
+ public void testMergeSchemasNested() {
+ TableSchema schema1 =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("nested")
+ .setType(TableFieldSchema.Type.STRUCT)
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("n1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.REQUIRED)))
+ .build();
+
+ TableSchema schema2 =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("nested")
+ .setType(TableFieldSchema.Type.STRUCT)
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("n1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("n2")
+ .setType(TableFieldSchema.Type.INT64)))
+ .build();
+
+ TableSchema merged = UpgradeTableSchema.mergeSchemas(schema1, schema2);
+
+ TableSchema expected =
+ TableSchema.newBuilder()
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("nested")
+ .setType(TableFieldSchema.Type.STRUCT)
+ .setMode(TableFieldSchema.Mode.NULLABLE)
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("n1")
+ .setType(TableFieldSchema.Type.STRING)
+ .setMode(TableFieldSchema.Mode.NULLABLE))
+ .addFields(
+ TableFieldSchema.newBuilder()
+ .setName("n2")
+ .setType(TableFieldSchema.Type.INT64)))
+ .build();
+ assertEquals(expected, merged);
+ assertEquals(1, merged.getFieldsCount());
+ }
+
+ @Test
+ public void testIsPayloadSchemaOutOfDate() throws Exception {
+ byte[] hash1 =
+
Hashing.goodFastHash(32).hashBytes("schema1".getBytes(StandardCharsets.UTF_8)).asBytes();
Review Comment:
Can we test cases where `UpgradeTableSchema.isPayloadSchemaOutOfDate`
returns true?
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java:
##########
@@ -2513,6 +2515,180 @@ public void updateTableSchemaTest(boolean useSet)
throws Exception {
Iterables.concat(expectedDroppedValues, expectedFullValues),
TableRow.class)));
}
+ @Test
+ public void testAutoPatchTableSchemaTest() throws Exception {
+ assumeTrue(useStreaming);
+ assumeTrue(useStorageApi);
+
+ // Make sure that GroupIntoBatches does not buffer data.
+
p.getOptions().as(BigQueryOptions.class).setStorageApiAppendThresholdBytes(1);
+ p.getOptions().as(BigQueryOptions.class).setNumStorageWriteApiStreams(1);
+
p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(2);
+
+ BigQueryIO.Write.Method method =
+ useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE :
Method.STORAGE_WRITE_API;
+ p.enableAbandonedNodeEnforcement(false);
+
+ TableReference tableRef =
BigQueryHelpers.parseTableSpec("project-id:dataset-id.table");
+ TableSchema tableSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("number").setType("INT64"),
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED")));
+ fakeDatasetService.createTable(new
Table().setTableReference(tableRef).setSchema(tableSchema));
+
+ final int stride = 5;
+ Function<Integer, TableSchema> getUpdatedSchema =
+ currentStride -> {
+ TableSchema tableSchemaUpdated = new TableSchema();
Review Comment:
Maybe `TableContainer` can maintain an ordered list of Table schemas
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDataTriggeredSchemaUpdateIT.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull;
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+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.auto.value.AutoValue;
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import org.joda.time.Duration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Integration test for BigQueryIO Storage Write API auto schema upgrade
triggered by data.
+ *
+ * <p>Uses a Stateful DoFn to sequence elements ensuring base schema elements
are written before
+ * evolved schema elements, avoiding race conditions in distributed execution.
+ */
+@RunWith(JUnit4.class)
+public class StorageApiDataTriggeredSchemaUpdateIT {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(StorageApiDataTriggeredSchemaUpdateIT.class);
+
+ private static final BigqueryClient BQ_CLIENT =
+ new BigqueryClient("StorageApiDataTriggeredSchemaUpdateIT");
+ private static final String PROJECT =
+ TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+ private static final String BIG_QUERY_DATASET_ID =
+ "storage_api_data_triggered_schema_update_" + System.nanoTime();
+
+ private static String bigQueryLocation;
+
+ @Rule public TestName testName = new TestName();
+
+ @BeforeClass
+ public static void setUpTestEnvironment() throws IOException,
InterruptedException {
+ bigQueryLocation =
+
TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation();
+ BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null,
bigQueryLocation);
+ }
+
+ @AfterClass
+ public static void cleanUp() {
+ LOG.info("Cleaning up dataset {} and tables.", BIG_QUERY_DATASET_ID);
+ BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+ }
+
+ private String createTable(TableSchema tableSchema) throws IOException,
InterruptedException {
+ String tableId = testName.getMethodName().replace("[", "_").replace("]",
"_");
+ BQ_CLIENT.deleteTable(PROJECT, BIG_QUERY_DATASET_ID, tableId);
+ BQ_CLIENT.createNewTable(
+ PROJECT,
+ BIG_QUERY_DATASET_ID,
+ new Table()
+ .setSchema(tableSchema)
+ .setTableReference(
+ new TableReference()
+ .setTableId(tableId)
+ .setDatasetId(BIG_QUERY_DATASET_ID)
+ .setProjectId(PROJECT)));
+ return tableId;
+ }
+
+ static class SequenceRowsDoFn extends DoFn<KV<Integer, Integer>, TableRow> {
+ private static final String COUNTER = "counter";
+
+ @StateId(COUNTER)
+ @SuppressWarnings("unused")
+ private final StateSpec<ValueState<Integer>> counterSpec =
StateSpecs.value();
+
+ private final int stride;
+
+ public SequenceRowsDoFn(int stride) {
+ this.stride = stride;
+ }
+
+ @ProcessElement
+ public void processElement(ProcessContext c, @StateId(COUNTER)
ValueState<Integer> counter) {
+ int current = firstNonNull(counter.read(), 0);
+ c.output(getRow(current));
+ counter.write(++current);
+ }
+
+ TableRow getRow(int i) {
+ TableRow row = new TableRow().set("name", "name" + i).set("number",
Long.toString(i));
+ if (i < stride) {
+ row = row.set("req", "foo");
+ } else {
+ row = row.set("new1", "blah" + i);
+ row = row.set("new2", "baz" + i);
+
+ if (i >= 2 * stride) {
+ TableRow nested =
+ new TableRow()
+ .set("nested_field1", "nested1" + i)
+ .set("nested_field2", "nested2" + i);
+
+ if (i >= 3 * stride) {
+ TableRow doubleNested =
+ new TableRow().set("double_nested_field1", "double_nested1" +
i);
+ nested = nested.set("double_nested", doubleNested);
+
+ // Add a repeated struct to ensure that we capture this code path
as well.
+ TableRow repeatedNested1 =
+ new TableRow().set("repeated_nested_field1",
"repeated_nested1" + i);
+ TableRow repeatedNested2 =
+ new TableRow().set("repeated_nested_field2",
"repeated_nested2" + i);
+ nested =
+ nested.set("repeated_nested",
ImmutableList.of(repeatedNested1, repeatedNested2));
+ }
+ row.set("nested", nested);
+ }
+ }
+ return row;
+ };
+ }
+
+ @Test
+ public void testDataTriggeredSchemaUpgradeExactlyOnce() throws Exception {
+ runTest(Write.Method.STORAGE_WRITE_API);
+ }
+
+ @Test
+ public void testDataTriggeredSchemaUpgradeAtLeastOnce() throws Exception {
+ runTest(Write.Method.STORAGE_API_AT_LEAST_ONCE);
+ }
+
+ private void runTest(Write.Method method) throws Exception {
+ Pipeline p = Pipeline.create(TestPipeline.testingPipelineOptions());
+
p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(1);
+
+ TableSchema baseSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema().setName("number").setType("INT64"),
+ new TableFieldSchema().setName("name").setType("STRING"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED")));
+ TableSchema evolvedSchema =
+ new TableSchema()
+ .setFields(
+ ImmutableList.of(
+ new
TableFieldSchema().setName("number").setType("INT64").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("name").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("req").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new1").setType("STRING").setMode("NULLABLE"),
+ new
TableFieldSchema().setName("new2").setType("STRING").setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+ .setName("nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+ .setName("double_nested")
+ .setType("STRUCT")
+ .setMode("NULLABLE")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+
.setName("double_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"))),
+ new TableFieldSchema()
+ .setName("repeated_nested")
+ .setType("STRUCT")
+ .setMode("REPEATED")
+ .setFields(
+ ImmutableList.of(
+ new TableFieldSchema()
+
.setName("repeated_nested_field1")
+ .setType("STRING")
+ .setMode("NULLABLE"),
+ new TableFieldSchema()
+
.setName("repeated_nested_field2")
+ .setType("STRING")
+ .setMode("NULLABLE")))))));
+
+ String tableId = createTable(baseSchema);
+ String tableSpec = PROJECT + ":" + BIG_QUERY_DATASET_ID + "." + tableId;
+
+ List<Integer> dummyInputs = IntStream.range(0,
20).boxed().collect(Collectors.toList());
+
+ BigQueryIO.Write<TableRow> write =
+ BigQueryIO.writeTableRows()
+ .to(tableSpec)
+ .withMethod(method)
+ .withSchemaUpdateOptions(
+ ImmutableSet.of(
+ Write.SchemaUpdateOption.ALLOW_FIELD_ADDITION,
+ Write.SchemaUpdateOption.ALLOW_FIELD_RELAXATION))
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER)
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND);
+ if (method == Write.Method.STORAGE_WRITE_API) {
+ write =
+ write
+ .withTriggeringFrequency(Duration.standardSeconds(1))
+ .withNumStorageWriteApiStreams(2);
+ }
+
+ p.apply("Create Dummy Inputs", Create.of(dummyInputs))
+ .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED)
+ .apply("Add a dummy key", WithKeys.of(1))
+ .apply("Sequence Rows", ParDo.of(new SequenceRowsDoFn(5)))
+ .apply("Stream to BigQuery", write);
+
+ p.run().waitUntilFinish();
+
+ List<TableRow> response =
+ BQ_CLIENT.queryUnflattened(
+ String.format("SELECT * FROM [%s] ORDER BY number", tableSpec),
+ PROJECT,
+ true,
+ false,
+ bigQueryLocation);
+ System.err.println("FULL RESULT " + response);
+
+ // Verification
+ verifyTableSchemaUpdated(tableSpec, evolvedSchema);
+ List<VerificationInfo> verifications =
+ ImmutableList.of(
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo("", 20),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo("req IS NULL",
15),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "new1 IS NOT NULL", 15),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "new2 IS NOT NULL", 15),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "nested.nested_field1 IS NOT NULL", 10),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "nested.nested_field2 IS NOT NULL", 10),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "nested.double_nested.double_nested_field1 IS NOT NULL", 5),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "nested.repeated_nested.repeated_nested_field1 IS NOT NULL",
5),
+ new
AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo(
+ "nested.repeated_nested.repeated_nested_field2 IS NOT NULL",
5));
+ verifyDataWritten(tableSpec, verifications);
+ }
+
+ private void verifyTableSchemaUpdated(String tableSpec, TableSchema
evolvedSchema)
+ throws IOException, InterruptedException {
+ Table table =
+ BQ_CLIENT.getTableResource(
+ PROJECT,
+ BIG_QUERY_DATASET_ID,
+ Iterables.getLast(
+
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter.on('.')
+ .split(tableSpec)));
+ assertEquals(
+ TableRowToStorageApiProto.schemaToProtoTableSchema(evolvedSchema),
+ TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema()));
+ }
Review Comment:
Same question from above: is there a way to validate that incremental
schemas occurred? Maybe harder to do in an integration test, but worth asking
the question
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]