Skip to content

Commit b4c3a4f

Browse files
authored
[BigQueryIO] fetch updated schema for newly created Storage API stream writers (#33231)
* add dynamic dest test * fix and add some tests * add to changes.md * fix whitespace * trigger postcommits * address comments
1 parent 40151ab commit b4c3a4f

File tree

8 files changed

+236
-34
lines changed

8 files changed

+236
-34
lines changed
Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
11
{
22
"comment": "Modify this file in a trivial way to cause this test suite to run",
3-
"modification": 2
3+
"modification": 3
44
}

CHANGES.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,7 @@
7979
## Bugfixes
8080

8181
* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
82+
* [BigQueryIO] Fixed an issue where Storage Write API sometimes doesn't pick up auto-schema updates ([#33231](https://github.com/apache/beam/pull/33231))
8283

8384
## Security Fixes
8485
* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)).

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -213,7 +213,7 @@ WriteStream createWriteStream(String tableUrn, WriteStream.Type type)
213213
throws IOException, InterruptedException;
214214

215215
@Nullable
216-
WriteStream getWriteStream(String writeStream);
216+
TableSchema getWriteStreamSchema(String writeStream);
217217

218218
/**
219219
* Create an append client for a given Storage API write stream. The stream must be created

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,7 @@
7676
import com.google.cloud.bigquery.storage.v1.FinalizeWriteStreamResponse;
7777
import com.google.cloud.bigquery.storage.v1.FlushRowsRequest;
7878
import com.google.cloud.bigquery.storage.v1.FlushRowsResponse;
79+
import com.google.cloud.bigquery.storage.v1.GetWriteStreamRequest;
7980
import com.google.cloud.bigquery.storage.v1.ProtoRows;
8081
import com.google.cloud.bigquery.storage.v1.ProtoSchema;
8182
import com.google.cloud.bigquery.storage.v1.ReadRowsRequest;
@@ -86,6 +87,7 @@
8687
import com.google.cloud.bigquery.storage.v1.StreamWriter;
8788
import com.google.cloud.bigquery.storage.v1.TableSchema;
8889
import com.google.cloud.bigquery.storage.v1.WriteStream;
90+
import com.google.cloud.bigquery.storage.v1.WriteStreamView;
8991
import com.google.cloud.hadoop.util.ApiErrorExtractor;
9092
import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
9193
import com.google.protobuf.DescriptorProtos;
@@ -1418,8 +1420,15 @@ public WriteStream createWriteStream(String tableUrn, WriteStream.Type type)
14181420
}
14191421

14201422
@Override
1421-
public @Nullable WriteStream getWriteStream(String writeStream) {
1422-
return newWriteClient.getWriteStream(writeStream);
1423+
public @Nullable TableSchema getWriteStreamSchema(String writeStream) {
1424+
@Nullable
1425+
WriteStream stream =
1426+
newWriteClient.getWriteStream(
1427+
GetWriteStreamRequest.newBuilder()
1428+
.setView(WriteStreamView.FULL)
1429+
.setName(writeStream)
1430+
.build());
1431+
return (stream != null && stream.hasTableSchema()) ? stream.getTableSchema() : null;
14231432
}
14241433

14251434
@Override

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@
2929
import com.google.cloud.bigquery.storage.v1.Exceptions;
3030
import com.google.cloud.bigquery.storage.v1.ProtoRows;
3131
import com.google.cloud.bigquery.storage.v1.TableSchema;
32-
import com.google.cloud.bigquery.storage.v1.WriteStream;
3332
import com.google.cloud.bigquery.storage.v1.WriteStream.Type;
3433
import com.google.protobuf.ByteString;
3534
import com.google.protobuf.DescriptorProtos;
@@ -475,15 +474,18 @@ SchemaAndDescriptor getCurrentTableSchema(String stream, @Nullable TableSchema u
475474
() -> {
476475
if (autoUpdateSchema) {
477476
@Nullable
478-
WriteStream writeStream =
477+
TableSchema streamSchema =
479478
Preconditions.checkStateNotNull(maybeWriteStreamService)
480-
.getWriteStream(streamName);
481-
if (writeStream != null && writeStream.hasTableSchema()) {
482-
TableSchema updatedFromStream = writeStream.getTableSchema();
483-
currentSchema.set(updatedFromStream);
484-
updated.set(true);
485-
LOG.debug(
486-
"Fetched updated schema for table {}:\n\t{}", tableUrn, updatedFromStream);
479+
.getWriteStreamSchema(streamName);
480+
if (streamSchema != null) {
481+
Optional<TableSchema> newSchema =
482+
TableSchemaUpdateUtils.getUpdatedSchema(initialTableSchema, streamSchema);
483+
if (newSchema.isPresent()) {
484+
currentSchema.set(newSchema.get());
485+
updated.set(true);
486+
LOG.debug(
487+
"Fetched updated schema for table {}:\n\t{}", tableUrn, newSchema.get());
488+
}
487489
}
488490
}
489491
return null;

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -531,6 +531,28 @@ public void process(
531531
element.getKey().getKey(), dynamicDestinations, datasetService);
532532
tableSchema = converter.getTableSchema();
533533
descriptor = converter.getDescriptor(false);
534+
535+
if (autoUpdateSchema) {
536+
// A StreamWriter ignores table schema updates that happen prior to its creation.
537+
// So before creating a StreamWriter below, we fetch the table schema to check if we
538+
// missed an update.
539+
// If so, use the new schema instead of the base schema
540+
@Nullable
541+
TableSchema streamSchema =
542+
MoreObjects.firstNonNull(
543+
writeStreamService.getWriteStreamSchema(getOrCreateStream.get()),
544+
TableSchema.getDefaultInstance());
545+
Optional<TableSchema> newSchema =
546+
TableSchemaUpdateUtils.getUpdatedSchema(tableSchema, streamSchema);
547+
548+
if (newSchema.isPresent()) {
549+
tableSchema = newSchema.get();
550+
descriptor =
551+
TableRowToStorageApiProto.descriptorSchemaFromTableSchema(
552+
tableSchema, true, false);
553+
updatedSchema.write(tableSchema);
554+
}
555+
}
534556
}
535557
AppendClientInfo info =
536558
AppendClientInfo.of(

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -590,11 +590,11 @@ public WriteStream createWriteStream(String tableUrn, Type type) throws Interrup
590590

591591
@Override
592592
@Nullable
593-
public WriteStream getWriteStream(String streamName) {
593+
public com.google.cloud.bigquery.storage.v1.TableSchema getWriteStreamSchema(String streamName) {
594594
synchronized (FakeDatasetService.class) {
595595
@Nullable Stream stream = writeStreams.get(streamName);
596596
if (stream != null) {
597-
return stream.toWriteStream();
597+
return stream.toWriteStream().getTableSchema();
598598
}
599599
}
600600
// TODO(relax): Return the exact error that BigQuery returns.

0 commit comments

Comments
 (0)