-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Fix UpdateSchemaDestination breaking DynamicDestination in Bigquery BatchLoad #25410
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -71,6 +71,7 @@ public enum JobType { | |
| COPY, | ||
| EXPORT, | ||
| QUERY, | ||
| SCHEMA_UPDATE, | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -27,8 +27,8 @@ | |||
| import com.google.api.services.bigquery.model.TableSchema; | ||||
| import com.google.api.services.bigquery.model.TimePartitioning; | ||||
| import java.io.IOException; | ||||
| import java.util.ArrayList; | ||||
| import java.util.List; | ||||
| import java.util.Map; | ||||
| import java.util.Set; | ||||
| import java.util.stream.Collectors; | ||||
| import javax.annotation.Nullable; | ||||
|
|
@@ -41,9 +41,23 @@ | |||
| import org.apache.beam.sdk.values.PCollectionView; | ||||
| import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; | ||||
| import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; | ||||
| import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; | ||||
| import org.slf4j.Logger; | ||||
| import org.slf4j.LoggerFactory; | ||||
|
|
||||
| /** | ||||
| * Update destination schema based on data that is about to be copied into it. | ||||
| * | ||||
| * <p>Unlike load and query jobs, BigQuery copy jobs do not support schema field addition or | ||||
| * relaxation on the destination table. This DoFn fills that gap by updating the destination table | ||||
| * schemas to be compatible with the data coming from the source table so that schemaUpdateOptions | ||||
| * are respected regardless of whether data is loaded directly to the destination table or loaded | ||||
| * into temporary tables before being copied into the destination. | ||||
| * | ||||
| * <p>This transform takes as input a list of KV(destination, WriteTables.Result) and emits a list | ||||
| * of KV(TableDestination, WriteTables.Result) where the destination label is parsed and replaced to | ||||
| * TableDestination objects. | ||||
| */ | ||||
| @SuppressWarnings({"nullness"}) | ||||
| public class UpdateSchemaDestination<DestinationT> | ||||
| extends DoFn< | ||||
|
|
@@ -52,8 +66,8 @@ public class UpdateSchemaDestination<DestinationT> | |||
|
|
||||
| private static final Logger LOG = LoggerFactory.getLogger(UpdateSchemaDestination.class); | ||||
| private final BigQueryServices bqServices; | ||||
| private final PCollectionView<String> loadJobIdPrefixView; | ||||
| private final ValueProvider<String> loadJobProjectId; | ||||
| private final PCollectionView<String> zeroLoadJobIdPrefixView; | ||||
| private final @Nullable ValueProvider<String> loadJobProjectId; | ||||
| private transient @Nullable DatasetService datasetService; | ||||
| private final int maxRetryJobs; | ||||
| private final @Nullable String kmsKey; | ||||
|
|
@@ -78,11 +92,11 @@ public PendingJobData( | |||
| } | ||||
| } | ||||
|
|
||||
| private final List<UpdateSchemaDestination.PendingJobData> pendingJobs = Lists.newArrayList(); | ||||
| private final Map<DestinationT, PendingJobData> pendingJobs = Maps.newHashMap(); | ||||
|
|
||||
| public UpdateSchemaDestination( | ||||
| BigQueryServices bqServices, | ||||
| PCollectionView<String> loadJobIdPrefixView, | ||||
| PCollectionView<String> zeroLoadJobIdPrefixView, | ||||
| @Nullable ValueProvider<String> loadJobProjectId, | ||||
| BigQueryIO.Write.WriteDisposition writeDisposition, | ||||
| BigQueryIO.Write.CreateDisposition createDisposition, | ||||
|
|
@@ -91,7 +105,7 @@ public UpdateSchemaDestination( | |||
| Set<BigQueryIO.Write.SchemaUpdateOption> schemaUpdateOptions, | ||||
| DynamicDestinations<?, DestinationT> dynamicDestinations) { | ||||
| this.loadJobProjectId = loadJobProjectId; | ||||
| this.loadJobIdPrefixView = loadJobIdPrefixView; | ||||
| this.zeroLoadJobIdPrefixView = zeroLoadJobIdPrefixView; | ||||
| this.bqServices = bqServices; | ||||
| this.maxRetryJobs = maxRetryJobs; | ||||
| this.kmsKey = kmsKey; | ||||
|
|
@@ -106,7 +120,13 @@ public void startBundle(StartBundleContext c) { | |||
| pendingJobs.clear(); | ||||
| } | ||||
|
|
||||
| TableDestination getTableWithDefaultProject(DestinationT destination, BigQueryOptions options) { | ||||
| TableDestination getTableWithDefaultProject(DestinationT destination) { | ||||
| if (dynamicDestinations.getPipelineOptions() == null) { | ||||
| throw new IllegalStateException( | ||||
| "Unexpected null pipeline option for DynamicDestination object. " | ||||
| + "Need to call setSideInputAccessorFromProcessContext(context) before use it."); | ||||
| } | ||||
| BigQueryOptions options = dynamicDestinations.getPipelineOptions().as(BigQueryOptions.class); | ||||
| TableDestination tableDestination = dynamicDestinations.getTable(destination); | ||||
| TableReference tableReference = tableDestination.getTableReference(); | ||||
|
|
||||
|
|
@@ -127,25 +147,24 @@ public void processElement( | |||
| ProcessContext context, | ||||
| BoundedWindow window) | ||||
| throws IOException { | ||||
| DestinationT destination = null; | ||||
| BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); | ||||
| dynamicDestinations.setSideInputAccessorFromProcessContext(context); | ||||
| List<KV<TableDestination, WriteTables.Result>> outputs = Lists.newArrayList(); | ||||
| for (KV<DestinationT, WriteTables.Result> entry : element) { | ||||
| destination = entry.getKey(); | ||||
| if (destination != null) { | ||||
|
||||
| TableDestination tableDestination = dynamicDestinations.getTable(destination); |
and asking it return a nonNull tableDestination. (same call used in UpdateSchemaDestination). So make the behavior consistent to WriteTables here
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it make sense to have a GBK after
writeTempTables? Re your comment on the issue we'd end up with aPCollection<KV<DestinationT, Iterable<WriteTables.Result>>>. Also would protect writeTempTables against retry if UpdateSchemaDestination failsThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see this would need some changes in WriteRename as well, perhaps this could be an improvement in a separate PR.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
after leaving that comment I find using a gbk may break the use case when the final destination table gets updated more than once, e.g. in streaming file load. Of course careful windowing we can avoid it, however I did not look deep into it at this moment. So far I leave the overall structure of the pipeline unchanged.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ideally, this check would prevent that if it worked as intended:
beam/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java
Line 263 in feb248a
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah agree, ideally. Just not confident enough and want to keep the change limited to what necessary to fix bug (though change is already not minor)