From cf790b50cc5c90c2bad9c6ab6e4779834a61e19e Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Mon, 19 Sep 2022 13:56:37 -0400 Subject: [PATCH] Do not use .get() on ValueProvider during pipeline creation --- CHANGES.md | 1 + .../beam/sdk/io/gcp/spanner/BatchSpannerRead.java | 6 +++--- .../beam/sdk/io/gcp/spanner/SpannerConfig.java | 13 ++++--------- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index e9bbd5f99124..0932d8ed8c34 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -60,6 +60,7 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Decreased TextSource CPU utilization by 2.3x (Java) ([#23193](https://github.com/apache/beam/issues/23193)). +* Fixed bug when using SpannerIO with RuntimeValueProvider options (Java) ([#22146](https://github.com/apache/beam/issues/22146)). ## New Features / Improvements diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java index f7715f9d9dc5..d1ac03b0b43a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + import com.google.auto.value.AutoValue; import com.google.cloud.spanner.BatchReadOnlyTransaction; import com.google.cloud.spanner.Options; @@ -37,7 +39,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LoadingCache; @@ -118,8 +119,7 @@ public GeneratePartitionsFn( SpannerConfig config, PCollectionView txView) { this.config = config; this.txView = txView; - Preconditions.checkNotNull(config.getRpcPriority()); - Preconditions.checkNotNull(config.getRpcPriority().get()); + checkNotNull(config.getRpcPriority()); } @Setup diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 2d3c15143c9a..05c8c8926d9c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -161,8 +160,7 @@ public SpannerConfig withProjectId(String projectId) { /** Specifies the Cloud Spanner instance ID. */ public SpannerConfig withInstanceId(ValueProvider instanceId) { - Preconditions.checkNotNull(instanceId); - Preconditions.checkNotNull(instanceId.get()); + checkNotNull(instanceId, "withInstanceId(instanceId) called with null input."); return toBuilder().setInstanceId(instanceId).build(); } @@ -173,8 +171,7 @@ public SpannerConfig withInstanceId(String instanceId) { /** Specifies the Cloud Spanner database ID. */ public SpannerConfig withDatabaseId(ValueProvider databaseId) { - Preconditions.checkNotNull(databaseId); - Preconditions.checkNotNull(databaseId.get()); + checkNotNull(databaseId, "withDatabaseId(databaseId) called with null input."); return toBuilder().setDatabaseId(databaseId).build(); } @@ -185,8 +182,7 @@ public SpannerConfig withDatabaseId(String databaseId) { /** Specifies the Cloud Spanner host. */ public SpannerConfig withHost(ValueProvider host) { - Preconditions.checkNotNull(host); - Preconditions.checkNotNull(host.get()); + checkNotNull(host, "withHost(host) called with null input."); return toBuilder().setHost(host).build(); } @@ -257,8 +253,7 @@ public SpannerConfig withRpcPriority(RpcPriority rpcPriority) { /** Specifies the RPC priority. */ public SpannerConfig withRpcPriority(ValueProvider rpcPriority) { - Preconditions.checkNotNull(rpcPriority); - Preconditions.checkNotNull(rpcPriority.get()); + checkNotNull(rpcPriority, "withRpcPriority(rpcPriority) called with null input."); return toBuilder().setRpcPriority(rpcPriority).build(); } }