Skip to content
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

[BEAM-13665] Fix SpannerIO NPE when projectID is not specified #16547

Merged
merged 1 commit into from
Jan 18, 2022
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 @@ -23,6 +23,7 @@
import com.google.cloud.spanner.Partition;
import com.google.cloud.spanner.ResultSet;
import com.google.cloud.spanner.SpannerException;
import com.google.cloud.spanner.SpannerOptions;
import com.google.cloud.spanner.Struct;
import com.google.cloud.spanner.TimestampBound;
import java.util.HashMap;
Expand Down Expand Up @@ -151,6 +152,7 @@ private static class ReadFromPartitionFn extends DoFn<Partition, Struct> {
private final PCollectionView<? extends Transaction> txView;

private transient SpannerAccessor spannerAccessor;
private transient String projectId;

public ReadFromPartitionFn(
SpannerConfig config, PCollectionView<? extends Transaction> txView) {
Expand All @@ -161,6 +163,10 @@ public ReadFromPartitionFn(
@Setup
public void setup() throws Exception {
spannerAccessor = SpannerAccessor.getOrCreate(config);
projectId =
this.config.getProjectId() == null
? SpannerOptions.getDefaultProjectId()
: this.config.getProjectId().get();
}

@Teardown
Expand All @@ -172,9 +178,7 @@ public void teardown() throws Exception {
public void processElement(ProcessContext c) throws Exception {
ServiceCallMetric serviceCallMetric =
createServiceCallMetric(
this.config.getProjectId().toString(),
this.config.getDatabaseId().toString(),
this.config.getInstanceId().toString());
projectId, this.config.getDatabaseId().get(), this.config.getInstanceId().get());
Transaction tx = c.sideInput(txView);

BatchReadOnlyTransaction batchTx =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1610,6 +1610,7 @@ static class WriteToSpannerFn extends DoFn<Iterable<MutationGroup>, Void> {

// Fluent Backoff is not serializable so create at runtime in setup().
private transient FluentBackoff bundleWriteBackoff;
private transient String projectId;

WriteToSpannerFn(
SpannerConfig spannerConfig, FailureMode failureMode, TupleTag<MutationGroup> failedTag) {
Expand All @@ -1625,6 +1626,11 @@ public void setup() {
FluentBackoff.DEFAULT
.withMaxCumulativeBackoff(spannerConfig.getMaxCumulativeBackoff().get())
.withInitialBackoff(spannerConfig.getMaxCumulativeBackoff().get().dividedBy(60));

projectId =
this.spannerConfig.getProjectId() == null
? SpannerOptions.getDefaultProjectId()
: this.spannerConfig.getProjectId().get();
}

@Teardown
Expand Down Expand Up @@ -1680,9 +1686,9 @@ private void spannerWriteWithRetryIfSchemaChange(Iterable<Mutation> batch)
for (int retry = 1; ; retry++) {
ServiceCallMetric serviceCallMetric =
createServiceCallMetric(
this.spannerConfig.getProjectId().toString(),
this.spannerConfig.getDatabaseId().toString(),
this.spannerConfig.getInstanceId().toString(),
projectId,
this.spannerConfig.getDatabaseId().get(),
this.spannerConfig.getInstanceId().get(),
"Write");
try {
spannerAccessor
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,12 +192,23 @@ private SpannerConfig getSpannerConfig() {
}

@Test
public void runRead() throws Exception {
public void runReadTestWithProjectId() throws Exception {
runReadTest(getSpannerConfig());
}

@Test
public void runReadTestWithDefaultProject() throws Exception {
runReadTest(
SpannerConfig.create()
.withInstanceId("123")
.withDatabaseId("aaa")
.withServiceFactory(serviceFactory));
}

private void runReadTest(SpannerConfig spannerConfig) throws Exception {
Timestamp timestamp = Timestamp.ofTimeMicroseconds(12345);
TimestampBound timestampBound = TimestampBound.ofReadTimestamp(timestamp);

SpannerConfig spannerConfig = getSpannerConfig();

PCollection<Struct> one =
pipeline.apply(
"read q",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,21 @@ public void singleMutationPipeline() throws Exception {
verifyBatches(batch(m(2L)));
}

@Test
public void singleMutationPipelineNoProjectId() throws Exception {
Mutation mutation = m(2L);
PCollection<Mutation> mutations = pipeline.apply(Create.of(mutation));

mutations.apply(
SpannerIO.write()
.withInstanceId("test-instance")
.withDatabaseId("test-database")
.withServiceFactory(serviceFactory));
pipeline.run();

verifyBatches(batch(m(2L)));
}

@Test
public void singleMutationGroupPipeline() throws Exception {
PCollection<MutationGroup> mutations =
Expand Down