Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,8 @@ private WriteResult expandTriggered(PCollection<KV<DestinationT, ElementT>> inpu
writeDisposition,
createDisposition,
maxRetryJobs,
kmsKey))
kmsKey,
loadJobProjectId))
.withSideInputs(copyJobIdPrefixView));
writeSinglePartition(partitions.get(singlePartitionTag), loadJobIdPrefixView);
return writeResult(p);
Expand Down Expand Up @@ -438,8 +439,9 @@ public WriteResult expandUntriggered(PCollection<KV<DestinationT, ElementT>> inp
writeDisposition,
createDisposition,
maxRetryJobs,
kmsKey))
.withSideInputs(loadJobIdPrefixView));
kmsKey,
loadJobProjectId))
.withSideInputs(copyJobIdPrefixView));
writeSinglePartition(partitions.get(singlePartitionTag), loadJobIdPrefixView);
return writeResult(p);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
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.PipelineOptions;
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.KV;
Expand Down Expand Up @@ -65,6 +66,7 @@ class WriteRename extends DoFn<Iterable<KV<TableDestination, WriteTables.Result>
private final CreateDisposition firstPaneCreateDisposition;
private final int maxRetryJobs;
private final String kmsKey;
private final ValueProvider<String> loadJobProjectId;
private @Nullable DatasetService datasetService;

private static class PendingJobData {
Expand All @@ -90,13 +92,15 @@ public WriteRename(
WriteDisposition writeDisposition,
CreateDisposition createDisposition,
int maxRetryJobs,
String kmsKey) {
String kmsKey,
ValueProvider<String> loadJobProjectId) {
this.bqServices = bqServices;
this.jobIdToken = jobIdToken;
this.firstPaneWriteDisposition = writeDisposition;
this.firstPaneCreateDisposition = createDisposition;
this.maxRetryJobs = maxRetryJobs;
this.kmsKey = kmsKey;
this.loadJobProjectId = loadJobProjectId;
}

@StartBundle
Expand Down Expand Up @@ -205,7 +209,8 @@ private PendingJobData startWriteRename(
tempTables,
writeDisposition,
createDisposition,
kmsKey);
kmsKey,
loadJobProjectId);
return new PendingJobData(retryJob, finalTableDestination, tempTables);
}

Expand All @@ -217,7 +222,8 @@ private BigQueryHelpers.PendingJob startCopy(
List<TableReference> tempTables,
WriteDisposition writeDisposition,
CreateDisposition createDisposition,
String kmsKey) {
String kmsKey,
ValueProvider<String> loadJobProjectId) {
JobConfigurationTableCopy copyConfig =
new JobConfigurationTableCopy()
.setSourceTables(tempTables)
Expand All @@ -232,7 +238,10 @@ private BigQueryHelpers.PendingJob startCopy(
String bqLocation =
BigQueryHelpers.getDatasetLocation(datasetService, ref.getProjectId(), ref.getDatasetId());

String projectId = ref.getProjectId();
String projectId =
loadJobProjectId == null || loadJobProjectId.get() == null
? ref.getProjectId()
: loadJobProjectId.get();
BigQueryHelpers.PendingJob retryJob =
new BigQueryHelpers.PendingJob(
jobId -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -488,7 +488,10 @@ private PendingJob startLoad(
loadConfig.setDestinationEncryptionConfiguration(
new EncryptionConfiguration().setKmsKeyName(kmsKey));
}
String projectId = loadJobProjectId == null ? ref.getProjectId() : loadJobProjectId.get();
String projectId =
loadJobProjectId == null || loadJobProjectId.get() == null
? ref.getProjectId()
: loadJobProjectId.get();
String bqLocation =
BigQueryHelpers.getDatasetLocation(datasetService, ref.getProjectId(), ref.getDatasetId());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2006,7 +2006,8 @@ public void testWriteRename() throws Exception {
BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED,
3,
"kms_key");
"kms_key",
null);

DoFnTester<Iterable<KV<TableDestination, WriteTables.Result>>, Void> tester =
DoFnTester.of(writeRename);
Expand Down