From 4e8d7fbf7dfef6bcef9d7f01ce986c93f26ed958 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 1 Feb 2022 10:13:36 -0800 Subject: [PATCH] Fix additional nullness errors in BigQueryIO --- .../beam/examples/kotlin/snippets/Snippets.kt | 4 +- .../org/apache/beam/sdk/coders/VoidCoder.java | 2 +- .../apache/beam/sdk/state/ReadableState.java | 2 - .../apache/beam/sdk/state/ReadableStates.java | 5 +- .../org/apache/beam/sdk/transforms/DoFn.java | 9 +++ .../apache/beam/sdk/util/Preconditions.java | 35 ++++++++++ .../values/FailsafeValueInSingleWindow.java | 8 +-- .../java/org/apache/beam/sdk/values/KV.java | 8 ++- .../beam/sdk/values/ValueInSingleWindow.java | 14 ++-- .../sdk/io/gcp/bigquery/AvroRowWriter.java | 6 +- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 14 ++-- .../gcp/bigquery/BatchedStreamingWrite.java | 29 ++++---- .../io/gcp/bigquery/BigQueryAvroUtils.java | 6 +- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 35 +++++++--- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../io/gcp/bigquery/BigQueryIOMetadata.java | 10 ++- .../io/gcp/bigquery/BigQueryQueryHelper.java | 19 +++--- .../gcp/bigquery/BigQueryQuerySourceDef.java | 3 +- .../gcp/bigquery/BigQueryResourceNaming.java | 7 +- .../sdk/io/gcp/bigquery/BigQueryServices.java | 3 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 25 ++++--- .../io/gcp/bigquery/BigQuerySourceBase.java | 28 ++++---- .../bigquery/BigQueryStorageArrowReader.java | 6 +- .../bigquery/BigQueryStorageAvroReader.java | 20 ++++-- .../bigquery/BigQueryStorageQuerySource.java | 15 ++--- .../bigquery/BigQueryStorageSourceBase.java | 11 +-- .../bigquery/BigQueryStorageStreamSource.java | 57 ++++++++++------ .../bigquery/BigQueryStorageTableSource.java | 22 +++--- .../io/gcp/bigquery/BigQueryTableSource.java | 18 +++-- .../gcp/bigquery/BigQueryTableSourceDef.java | 8 +-- .../sdk/io/gcp/bigquery/BigQueryUtils.java | 32 +++++---- .../gcp/bigquery/CreateTableDestinations.java | 7 +- .../io/gcp/bigquery/CreateTableHelpers.java | 31 +++++---- .../sdk/io/gcp/bigquery/CreateTables.java | 17 ++--- .../io/gcp/bigquery/DynamicDestinations.java | 12 ++-- .../bigquery/DynamicDestinationsHelpers.java | 56 +++++++++------- .../sdk/io/gcp/bigquery/ErrorContainer.java | 10 +-- .../sdk/io/gcp/bigquery/PrepareWrite.java | 11 ++- .../sdk/io/gcp/bigquery/ReifyAsIterable.java | 6 +- .../sdk/io/gcp/bigquery/RetryManager.java | 16 ++--- .../sdk/io/gcp/bigquery/RowWriterFactory.java | 23 +++++-- .../bigquery/StorageApiConvertMessages.java | 5 +- .../StorageApiDynamicDestinations.java | 4 +- .../StorageApiDynamicDestinationsBeamRow.java | 4 +- ...StorageApiDynamicDestinationsTableRow.java | 7 +- .../StorageApiFinalizeWritesDoFn.java | 23 ++++--- .../StorageApiFlushAndFinalizeDoFn.java | 12 ++-- .../StorageApiWriteUnshardedRecords.java | 67 +++++++++++-------- .../StorageApiWritesShardedRecords.java | 18 ++--- .../sdk/io/gcp/bigquery/StreamingInserts.java | 12 ++-- .../io/gcp/bigquery/StreamingWriteTables.java | 26 +++---- .../sdk/io/gcp/bigquery/TableDestination.java | 11 ++- .../gcp/bigquery/TableDestinationCoder.java | 7 +- .../gcp/bigquery/TableDestinationCoderV2.java | 7 +- .../gcp/bigquery/TableDestinationCoderV3.java | 10 +-- .../bigquery/TableRowToStorageApiProto.java | 18 +++-- .../sdk/io/gcp/bigquery/TagWithUniqueIds.java | 7 +- .../sdk/io/gcp/bigquery/TestBigQuery.java | 30 +++++++-- .../TwoLevelMessageConverterCache.java | 6 +- .../gcp/bigquery/UpdateSchemaDestination.java | 2 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 37 ++++++---- .../sdk/io/gcp/bigquery/WritePartition.java | 16 ++--- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 20 +++--- .../beam/sdk/io/gcp/bigquery/WriteResult.java | 49 +++++++------- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 21 +++--- .../bigquery/BigQueryServicesImplTest.java | 12 +++- 66 files changed, 624 insertions(+), 459 deletions(-) diff --git a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt index 3fdc71d0b58a..d2e2a5883371 100644 --- a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt +++ b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt @@ -269,8 +269,8 @@ object Snippets { BigQueryIO.write() .to( object : DynamicDestinations() { - override fun getDestination(elem: ValueInSingleWindow): Long? { - return elem.value!!.year + override fun getDestination(elem: ValueInSingleWindow?): Long { + return elem!!.value!!.year } override fun getTable(destination: Long?): TableDestination { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java index dab7e64ae972..e323627a1ba6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java @@ -23,7 +23,7 @@ import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link Coder} for {@link Void}. Uses zero bytes per {@link Void}. */ -public class VoidCoder extends AtomicCoder { +public class VoidCoder extends AtomicCoder<@Nullable Void> { public static VoidCoder of() { return INSTANCE; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java index 1023b3c649ed..aa88d440618e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link State} that can be read via {@link #read()}. @@ -42,7 +41,6 @@ public interface ReadableState { * of the returned object should not modify state without going through the appropriate state * interface, and modification to the state should not be mirrored in the returned object. */ - @Nullable T read(); /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java index 0ad9afd208e0..769354752f89 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java @@ -18,17 +18,16 @@ package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Internal; -import org.checkerframework.checker.nullness.qual.Nullable; /** For internal use only; no backwards-compatibility guarantees. */ @Internal public class ReadableStates { /** A {@link ReadableState} constructed from a constant value, hence immediately available. */ - public static ReadableState immediate(final @Nullable T value) { + public static ReadableState immediate(final T value) { return new ReadableState() { @Override - public @Nullable T read() { + public T read() { return value; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 47eac66c9af8..ebde314dfd76 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; import org.joda.time.Duration; import org.joda.time.Instant; @@ -98,6 +99,7 @@ public abstract class StartBundleContext { * Returns the {@code PipelineOptions} specified with the {@link * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. */ + @Pure public abstract PipelineOptions getPipelineOptions(); } @@ -107,6 +109,7 @@ public abstract class FinishBundleContext { * Returns the {@code PipelineOptions} specified with the {@link * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. */ + @Pure public abstract PipelineOptions getPipelineOptions(); /** @@ -141,6 +144,7 @@ public abstract class WindowedContext { * Returns the {@code PipelineOptions} specified with the {@link * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. */ + @Pure public abstract PipelineOptions getPipelineOptions(); /** @@ -240,6 +244,7 @@ public abstract class ProcessContext extends WindowedContext { *

The element will not be changed -- it is safe to cache, etc. without copying. * Implementation of {@link DoFn.ProcessElement} method should not mutate the element. */ + @Pure public abstract InputT element(); /** @@ -248,6 +253,7 @@ public abstract class ProcessContext extends WindowedContext { * @throws IllegalArgumentException if this is not a side input * @see ParDo.SingleOutput#withSideInputs */ + @Pure public abstract T sideInput(PCollectionView view); /** @@ -255,6 +261,7 @@ public abstract class ProcessContext extends WindowedContext { * *

See {@link Window} for more information. */ + @Pure public abstract Instant timestamp(); /** @@ -264,6 +271,7 @@ public abstract class ProcessContext extends WindowedContext { *

Generally all data is in a single, uninteresting pane unless custom triggering and/or late * data has been explicitly requested. See {@link Window} for more information. */ + @Pure public abstract PaneInfo pane(); } @@ -287,6 +295,7 @@ public abstract class OnTimerContext extends WindowedContext { public abstract class OnWindowExpirationContext extends WindowedContext { /** Returns the window in which the window expiration is firing. */ + @Pure public abstract BoundedWindow window(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java index 2651d75420c1..a1c555fe57e4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java @@ -437,4 +437,39 @@ public class Preconditions { } return obj; } + + /** + * Ensures that a piece of state passed as a parameter to the calling method is not null. + * + * @param reference an object reference + * @return the non-null reference that was validated + * @throws IllegalStateException if {@code reference} is null + */ + @CanIgnoreReturnValue + @EnsuresNonNull("#1") + public static T checkStateNotNull(@Nullable T reference) { + if (reference == null) { + throw new IllegalStateException(); + } + return reference; + } + + /** + * Ensures that a piece of state passed as a parameter to the calling method is not null. + * + * @param reference an object reference + * @param errorMessage the exception message to use if the check fails; will be converted to a + * string using {@link String#valueOf(Object)} + * @return the non-null reference that was validated + * @throws IllegalStateException if {@code reference} is null + */ + @CanIgnoreReturnValue + @EnsuresNonNull("#1") + public static T checkStateNotNull( + @Nullable T reference, @Nullable Object errorMessage) { + if (reference == null) { + throw new IllegalStateException(String.valueOf(errorMessage)); + } + return reference; + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java index 7ddbebf1aa98..ac9b81996fda 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -38,12 +37,9 @@ */ @AutoValue @Internal -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public abstract class FailsafeValueInSingleWindow { /** Returns the value of this {@code FailsafeValueInSingleWindow}. */ - public abstract @Nullable T getValue(); + public abstract T getValue(); /** Returns the timestamp of this {@code FailsafeValueInSingleWindow}. */ public abstract Instant getTimestamp(); @@ -55,7 +51,7 @@ public abstract class FailsafeValueInSingleWindow { public abstract PaneInfo getPane(); /** Returns the failsafe value of this {@code FailsafeValueInSingleWindow}. */ - public abstract @Nullable ErrorT getFailsafeValue(); + public abstract ErrorT getFailsafeValue(); public static FailsafeValueInSingleWindow of( T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo, ErrorT failsafeValue) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java index 97d276c2b90a..0c095ade1555 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.transforms.SerializableComparator; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; /** * An immutable key/value pair. @@ -37,19 +38,20 @@ * @param the type of the key * @param the type of the value */ -public class KV implements Serializable { +public class KV implements Serializable { /** Returns a {@link KV} with the given key and value. */ - public static KV of( - K key, V value) { + public static KV of(K key, V value) { return new KV<>(key, value); } /** Returns the key of this {@link KV}. */ + @Pure public K getKey() { return key; } /** Returns the value of this {@link KV}. */ + @Pure public V getValue() { return value; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java index 968d4941d6bc..aed94541fe2b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java @@ -38,12 +38,18 @@ */ @AutoValue @Internal -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public abstract class ValueInSingleWindow { /** Returns the value of this {@code ValueInSingleWindow}. */ - public abstract @Nullable T getValue(); + @SuppressWarnings("nullness") + public T getValue() { + return getNullableValue(); + }; + + /** + * Workaround for autovalue code generation, which does not allow type variables to be + * instantiated with nullable actual parameters. + */ + protected abstract @Nullable T getNullableValue(); /** Returns the timestamp of this {@code ValueInSingleWindow}. */ public abstract Instant getTimestamp(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java index 540c196cab07..0b64a65c4503 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java @@ -24,14 +24,14 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.MimeTypes; -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class AvroRowWriter extends BigQueryRowWriter { private final DataFileWriter writer; private final Schema schema; private final SerializableFunction, AvroT> toAvroRecord; + @SuppressWarnings({ + "nullness" // calling superclass method in constructor flagged as error; TODO: fix + }) AvroRowWriter( String basename, Schema schema, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 8679926359fe..e5124b0047a9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -62,6 +62,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -80,9 +81,6 @@ import org.slf4j.LoggerFactory; /** PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BatchLoads extends PTransform>, WriteResult> { private static final Logger LOG = LoggerFactory.getLogger(BatchLoads.class); @@ -141,12 +139,12 @@ class BatchLoads private int maxFilesPerPartition; private long maxBytesPerPartition; private int numFileShards; - private Duration triggeringFrequency; + private @Nullable Duration triggeringFrequency; private ValueProvider customGcsTempLocation; - private ValueProvider loadJobProjectId; + private @Nullable ValueProvider loadJobProjectId; private final Coder elementCoder; private final RowWriterFactory rowWriterFactory; - private final String kmsKey; + private final @Nullable String kmsKey; private final boolean clusteringEnabled; private final String tempDataset; @@ -242,7 +240,8 @@ void setMaxBytesPerPartition(long maxBytesPerPartition) { } @Override - public void validate(PipelineOptions options) { + public void validate(@Nullable PipelineOptions maybeOptions) { + PipelineOptions options = Preconditions.checkArgumentNotNull(maybeOptions); // We will use a BigQuery load job -- validate the temp location. String tempLocation; if (customGcsTempLocation == null) { @@ -274,6 +273,7 @@ public void validate(PipelineOptions options) { // Expand the pipeline when the user has requested periodically-triggered file writes. private WriteResult expandTriggered(PCollection> input) { + Duration triggeringFrequency = Preconditions.checkStateNotNull(this.triggeringFrequency); Pipeline p = input.getPipeline(); final PCollectionView loadJobIdPrefixView = createJobIdPrefixView(p, JobType.LOAD); final PCollectionView tempLoadJobIdPrefixView = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java index 015a5e0dd79b..5e611d6c22d0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java @@ -59,13 +59,11 @@ 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.collect.ImmutableSet; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.joda.time.Duration; import org.joda.time.Instant; /** PTransform to perform batched streaming BigQuery write. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BatchedStreamingWrite extends PTransform>>, PCollectionTuple> { private static final TupleTag mainOutputTag = new TupleTag<>("mainOutput"); @@ -79,8 +77,8 @@ class BatchedStreamingWrite private final boolean skipInvalidRows; private final boolean ignoreUnknownValues; private final boolean ignoreInsertIds; - private final SerializableFunction toTableRow; - private final SerializableFunction toFailsafeTableRow; + private final @Nullable SerializableFunction toTableRow; + private final @Nullable SerializableFunction toFailsafeTableRow; private final Set allowedMetricUrns; /** Tracks bytes written, exposed as "ByteCount" Counter. */ @@ -98,8 +96,8 @@ public BatchedStreamingWrite( boolean skipInvalidRows, boolean ignoreUnknownValues, boolean ignoreInsertIds, - SerializableFunction toTableRow, - SerializableFunction toFailsafeTableRow) { + @Nullable SerializableFunction toTableRow, + @Nullable SerializableFunction toFailsafeTableRow) { this.bqServices = bqServices; this.retryPolicy = retryPolicy; this.failedOutputTag = failedOutputTag; @@ -123,8 +121,8 @@ private BatchedStreamingWrite( boolean skipInvalidRows, boolean ignoreUnknownValues, boolean ignoreInsertIds, - SerializableFunction toTableRow, - SerializableFunction toFailsafeTableRow, + @Nullable SerializableFunction toTableRow, + @Nullable SerializableFunction toFailsafeTableRow, boolean batchViaStateful) { this.bqServices = bqServices; this.retryPolicy = retryPolicy; @@ -212,10 +210,11 @@ public PCollectionTuple expand(PCollection>> i private class BatchAndInsertElements extends DoFn>, Void> { /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */ - private transient Map>> tableRows; + private transient @Nullable Map>> + tableRows = null; /** The list of unique ids for each BigQuery table row. */ - private transient Map> uniqueIdsForTableRows; + private transient @Nullable Map> uniqueIdsForTableRows = null; private transient @Nullable DatasetService datasetService; @@ -235,11 +234,14 @@ public void startBundle() { /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */ @ProcessElement + @RequiresNonNull({"tableRows", "uniqueIdsForTableRows", "toTableRow", "toFailsafeTableRow"}) public void processElement( @Element KV> element, @Timestamp Instant timestamp, BoundedWindow window, PaneInfo pane) { + Map>> tableRows = this.tableRows; + Map> uniqueIdsForTableRows = this.uniqueIdsForTableRows; String tableSpec = element.getKey(); TableRow tableRow = toTableRow.apply(element.getValue().tableRow); TableRow failsafeTableRow = toFailsafeTableRow.apply(element.getValue().tableRow); @@ -253,7 +255,10 @@ public void processElement( /** Writes the accumulated rows into BigQuery with streaming API. */ @FinishBundle + @RequiresNonNull({"tableRows", "uniqueIdsForTableRows"}) public void finishBundle(FinishBundleContext context) throws Exception { + Map>> tableRows = this.tableRows; + Map> uniqueIdsForTableRows = this.uniqueIdsForTableRows; List> failedInserts = Lists.newArrayList(); List> successfulInserts = Lists.newArrayList(); BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); @@ -364,7 +369,7 @@ private void flushRows( DatasetService datasetService, TableReference tableReference, List> tableRows, - List uniqueIds, + @Nullable List uniqueIds, List> failedInserts, List> successfulInserts) throws InterruptedException { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java index 220489d7cbc8..b36779dd6b08 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java @@ -59,9 +59,6 @@ *

These utilities are based on the Avro * 1.8.1 specification. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryAvroUtils { /** @@ -417,6 +414,9 @@ static Schema toGenericAvroSchema(String schemaName, List fiel avroFields); } + @SuppressWarnings({ + "nullness" // Avro library not annotated + }) private static Field convertField(TableFieldSchema bigQueryField) { ImmutableCollection avroTypes = BIG_QUERY_TO_AVRO_TYPES.get(bigQueryField.getType()); if (avroTypes.isEmpty()) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 10009b6fff62..54d2e8140094 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -47,15 +47,14 @@ import org.apache.beam.sdk.util.FluentBackoff; 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.collect.Lists; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.PolyNull; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** A set of helper functions and classes used by {@link BigQueryIO}. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class BigQueryHelpers { private static final String RESOURCE_NOT_FOUND_ERROR = "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline" @@ -91,9 +90,10 @@ public RetryJobIdResult(RetryJobId jobId, boolean shouldRetry) { static class PendingJobManager { private static class JobInfo { private final PendingJob pendingJob; - private final @Nullable SerializableFunction onSuccess; + private final SerializableFunction onSuccess; - public JobInfo(PendingJob pendingJob, SerializableFunction onSuccess) { + public JobInfo( + PendingJob pendingJob, SerializableFunction onSuccess) { this.pendingJob = pendingJob; this.onSuccess = onSuccess; } @@ -118,7 +118,7 @@ public JobInfo(PendingJob pendingJob, SerializableFunction onSuccess) { + PendingJob pendingJob, SerializableFunction onSuccess) { this.pendingJobs.add(new JobInfo(pendingJob, onSuccess)); return this; } @@ -173,7 +173,7 @@ static class PendingJob { private final int maxRetries; private int currentAttempt; RetryJobId currentJobId; - Job lastJobAttempted; + @Nullable Job lastJobAttempted = null; boolean started; PendingJob( @@ -396,6 +396,9 @@ static List getOrCreateMapListValue(Map> map, K key) { * *

If the project id is omitted, the default project id is used. */ + @SuppressWarnings({ + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) + }) public static TableReference parseTableSpec(String tableSpec) { Matcher match = BigQueryIO.TABLE_SPEC.matcher(tableSpec); if (!match.matches()) { @@ -414,6 +417,9 @@ public static TableReference parseTableSpec(String tableSpec) { return ref.setDatasetId(match.group("DATASET")).setTableId(match.group("TABLE")); } + @SuppressWarnings({ + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) + }) public static TableReference parseTableUrn(String tableUrn) { Matcher match = BigQueryIO.TABLE_URN_SPEC.matcher(tableUrn); if (!match.matches()) { @@ -435,6 +441,9 @@ public static String stripPartitionDecorator(String tableSpec) { return (index == -1) ? tableSpec : tableSpec.substring(0, index); } + @SuppressWarnings({ + "nullness" // The BigQuery API library is documented to accept nulls but is not annotated + }) static String jobToPrettyString(@Nullable Job job) throws IOException { if (job != null && job.getConfiguration().getLoad() != null) { // Removing schema and sourceUris from error messages for load jobs since these fields can be @@ -465,7 +474,7 @@ static Status parseStatus(@Nullable Job job) { } } - public static String toJsonString(Object item) { + public static @PolyNull String toJsonString(@PolyNull Object item) { if (item == null) { return null; } @@ -478,12 +487,18 @@ public static String toJsonString(Object item) { } } - public static T fromJsonString(String json, Class clazz) { + public static @PolyNull T fromJsonString(@PolyNull String json, Class clazz) { if (json == null) { return null; } try { - return BigQueryIO.JSON_FACTORY.fromString(json, clazz); + // If T is Void then this ends up null, otherwise it is not; kind of a tough invariant + @SuppressWarnings({ + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) + }) + @NonNull + T result = BigQueryIO.JSON_FACTORY.fromString(json, clazz); + return result; } catch (IOException e) { throw new RuntimeException( String.format("Cannot deserialize %s from a JSON string: %s.", clazz, json), e); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 6596d33be099..463a5c80f87b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -466,7 +466,7 @@ * here. */ @SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10608) }) public class BigQueryIO { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java index 663a853fec45..ee64a7ab9ddb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java @@ -21,16 +21,14 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.beam.sdk.extensions.gcp.util.GceMetadataUtil; +import org.checkerframework.checker.nullness.qual.Nullable; /** Metadata class for BigQueryIO. i.e. to use as BQ job labels. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) final class BigQueryIOMetadata { - private String beamJobId; + private @Nullable String beamJobId; - private BigQueryIOMetadata(String beamJobId) { + private BigQueryIOMetadata(@Nullable String beamJobId) { this.beamJobId = beamJobId; } @@ -66,7 +64,7 @@ public Map addAdditionalJobLabels(Map jobLabels) /* * Returns the beam job id. Can be null if it is not running on Dataflow. */ - public String getBeamJobId() { + public @Nullable String getBeamJobId() { return this.beamJobId; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java index a42eea200524..ee9b794880e1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java @@ -47,9 +47,6 @@ *

This object is not serializable, and its state can be safely discarded across serialization * boundaries for any associated source objects. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryQueryHelper { private static final Integer JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; @@ -59,13 +56,17 @@ class BigQueryQueryHelper { public static JobStatistics dryRunQueryIfNeeded( BigQueryServices bqServices, BigQueryOptions options, - AtomicReference dryRunJobStats, + AtomicReference<@Nullable JobStatistics> dryRunJobStats, String query, Boolean flattenResults, Boolean useLegacySql, @Nullable String location) throws InterruptedException, IOException { - if (dryRunJobStats.get() == null) { + @Nullable JobStatistics maybeJobStatistics = dryRunJobStats.get(); + + if (maybeJobStatistics != null) { + return maybeJobStatistics; + } else { JobStatistics jobStatistics = bqServices .getJobService(options) @@ -76,15 +77,14 @@ public static JobStatistics dryRunQueryIfNeeded( createBasicQueryConfig(query, flattenResults, useLegacySql), location); dryRunJobStats.compareAndSet(null, jobStatistics); + return jobStatistics; } - - return dryRunJobStats.get(); } public static TableReference executeQuery( BigQueryServices bqServices, BigQueryOptions options, - AtomicReference dryRunJobStats, + AtomicReference<@Nullable JobStatistics> dryRunJobStats, String stepUuid, String query, Boolean flattenResults, @@ -95,7 +95,7 @@ public static TableReference executeQuery( @Nullable String kmsKey) throws InterruptedException, IOException { // Step 1: Find the effective location of the query. - String effectiveLocation = location; + @Nullable String effectiveLocation = location; try (DatasetService tableService = bqServices.getDatasetService(options)) { if (effectiveLocation == null) { List referencedTables = @@ -162,6 +162,7 @@ public static TableReference executeQuery( queryResultTable, queryJobId); + @SuppressWarnings("nullness") // setLocation is not annotated, but does accept nulls JobReference jobReference = new JobReference() .setProjectId( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java index 41c9caf25565..1d06c819ccf6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +50,7 @@ class BigQueryQuerySourceDef implements BigQuerySourceDef { private final String tempDatasetId; private final String kmsKey; - private transient AtomicReference dryRunJobStats; + private transient AtomicReference<@Nullable JobStatistics> dryRunJobStats; static BigQueryQuerySourceDef create( BigQueryServices bqServices, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java index 7eae6fef33b5..fd1b18dfb921 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java @@ -22,6 +22,7 @@ import com.google.api.services.bigquery.model.TableReference; import java.util.Optional; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing; +import org.checkerframework.checker.nullness.qual.Nullable; /** * This class contains utilities to standardize how resources are named by BigQueryIO. @@ -37,9 +38,6 @@ * *

This class has no backwards compatibility guaantees. It is considered internal. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryResourceNaming { /** @@ -86,7 +84,8 @@ public enum JobType { * the parameter will be ignored. * @return */ - static String createJobIdPrefix(String jobName, String stepUuid, JobType type, String random) { + static String createJobIdPrefix( + String jobName, String stepUuid, JobType type, @Nullable String random) { jobName = jobName.replaceAll("-", ""); String result = BIGQUERY_JOB_TEMPLATE diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 598431407d14..dd4da2bd7fc3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -97,7 +97,8 @@ void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException; /** Dry runs the query in the given project. */ - JobStatistics dryRunQuery(String projectId, JobConfigurationQuery queryConfig, String location) + JobStatistics dryRunQuery( + String projectId, JobConfigurationQuery queryConfig, @Nullable String location) throws InterruptedException, IOException; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 0185b5a40693..73b13670f94f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -93,6 +93,7 @@ import io.grpc.protobuf.ProtoUtils; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -143,9 +144,10 @@ * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery * service. */ -@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402) - -// TODO(https://issues.apache.org/jira/browse/BEAM-10402) +@SuppressWarnings({ + "nullness", // TODO(https://issues.apache.org/jira/browse/BEAM-10608) + "keyfor" +}) class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); @@ -415,6 +417,7 @@ public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedExcep } @VisibleForTesting + @Nullable Job pollJob(JobReference jobRef, Sleeper sleeper, BackOff backoff) throws InterruptedException { do { try { @@ -459,7 +462,7 @@ private static String formatBqStatusCommand(String projectId, String jobId) { @Override public JobStatistics dryRunQuery( - String projectId, JobConfigurationQuery queryConfig, String location) + String projectId, JobConfigurationQuery queryConfig, @Nullable String location) throws InterruptedException, IOException { JobReference jobRef = new JobReference().setLocation(location).setProjectId(projectId); Job job = @@ -490,7 +493,7 @@ public Job getJob(JobReference jobRef) throws IOException, InterruptedException } @VisibleForTesting - public Job getJob(JobReference jobRef, Sleeper sleeper, BackOff backoff) + public @Nullable Job getJob(JobReference jobRef, Sleeper sleeper, BackOff backoff) throws IOException, InterruptedException { String jobId = jobRef.getJobId(); Exception lastException; @@ -544,7 +547,7 @@ static class DatasetServiceImpl implements DatasetService { private final ApiErrorExtractor errorExtractor; private final Bigquery client; - @Nullable private final BigQueryWriteClient newWriteClient; + private final @Nullable BigQueryWriteClient newWriteClient; private final PipelineOptions options; private final long maxRowsPerBatch; private final long maxRowBatchSize; @@ -552,7 +555,7 @@ static class DatasetServiceImpl implements DatasetService { private final Counter throttlingMsecs = Metrics.counter(DatasetServiceImpl.class, "throttling-msecs"); - private BoundedExecutorService executor; + private @Nullable BoundedExecutorService executor; private final BigQueryIOMetadata bqIOMetadata; @VisibleForTesting @@ -607,7 +610,7 @@ private DatasetServiceImpl(BigQueryOptions bqOptions) { @Override public @Nullable Table getTable(TableReference tableRef) throws IOException, InterruptedException { - return getTable(tableRef, null); + return getTable(tableRef, Collections.emptyList()); } @Override @@ -619,14 +622,14 @@ private DatasetServiceImpl(BigQueryOptions bqOptions) { @VisibleForTesting @Nullable Table getTable( - TableReference ref, @Nullable List selectedFields, BackOff backoff, Sleeper sleeper) + TableReference ref, List selectedFields, BackOff backoff, Sleeper sleeper) throws IOException, InterruptedException { Tables.Get get = client .tables() .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()) .setPrettyPrint(false); - if (selectedFields != null && !selectedFields.isEmpty()) { + if (!selectedFields.isEmpty()) { get.setSelectedFields(String.join(",", selectedFields)); } try { @@ -1238,7 +1241,7 @@ public long insertAll( successfulRows); } - protected static GoogleJsonError.ErrorInfo getErrorInfo(IOException e) { + protected static GoogleJsonError.@Nullable ErrorInfo getErrorInfo(IOException e) { if (!(e instanceof GoogleJsonResponseException)) { return null; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index ec5c7176a199..de42bc0998f2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.io.FileSystems.match; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.resolveTempLocation; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -64,9 +64,6 @@ * * ... */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) abstract class BigQuerySourceBase extends BoundedSource { private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceBase.class); @@ -76,7 +73,7 @@ abstract class BigQuerySourceBase extends BoundedSource { protected final String stepUuid; protected final BigQueryServices bqServices; - private transient List> cachedSplitResult; + private transient @Nullable List> cachedSplitResult = null; private SerializableFunction parseFn; private Coder coder; private final boolean useAvroLogicalTypes; @@ -87,24 +84,26 @@ abstract class BigQuerySourceBase extends BoundedSource { Coder coder, SerializableFunction parseFn, boolean useAvroLogicalTypes) { - this.stepUuid = checkNotNull(stepUuid, "stepUuid"); - this.bqServices = checkNotNull(bqServices, "bqServices"); - this.coder = checkNotNull(coder, "coder"); - this.parseFn = checkNotNull(parseFn, "parseFn"); + this.stepUuid = checkArgumentNotNull(stepUuid, "stepUuid"); + this.bqServices = checkArgumentNotNull(bqServices, "bqServices"); + this.coder = checkArgumentNotNull(coder, "coder"); + this.parseFn = checkArgumentNotNull(parseFn, "parseFn"); this.useAvroLogicalTypes = useAvroLogicalTypes; } protected static class ExtractResult { public final TableSchema schema; public final List extractedFiles; - public List metadata = null; + public @Nullable List metadata = null; public ExtractResult(TableSchema schema, List extractedFiles) { this(schema, extractedFiles, null); } public ExtractResult( - TableSchema schema, List extractedFiles, List metadata) { + TableSchema schema, + List extractedFiles, + @Nullable List metadata) { this.schema = schema; this.extractedFiles = extractedFiles; this.metadata = metadata; @@ -167,7 +166,7 @@ public List> split(long desiredBundleSizeBytes, PipelineOptions } } cleanupTempResource(options.as(BigQueryOptions.class)); - cachedSplitResult = checkNotNull(createSources(res.extractedFiles, res.schema, res.metadata)); + cachedSplitResult = createSources(res.extractedFiles, res.schema, res.metadata); } return cachedSplitResult; } @@ -228,7 +227,8 @@ private List executeExtract( return BigQueryIO.getExtractFilePaths(extractDestinationDir, extractJob); } - private static class TableSchemaFunction implements Serializable, Function { + private static class TableSchemaFunction + implements Serializable, Function<@Nullable String, @Nullable TableSchema> { @Override public @Nullable TableSchema apply(@Nullable String input) { return BigQueryHelpers.fromJsonString(input, TableSchema.class); @@ -236,7 +236,7 @@ private static class TableSchemaFunction implements Serializable, Function> createSources( - List files, TableSchema schema, List metadata) + List files, TableSchema schema, @Nullable List metadata) throws IOException, InterruptedException { final String jsonSchema = BigQueryIO.JSON_FACTORY.toString(schema); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java index 92d23bf4b883..468dcbf5f2ed 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.Row; -@SuppressWarnings("nullness") class BigQueryStorageArrowReader implements BigQueryStorageReader { private @Nullable RecordBatchRowIterator recordBatchIterator; @@ -50,12 +49,13 @@ public void processReadRowsResponse(ReadRowsResponse readRowsResponse) throws IO com.google.cloud.bigquery.storage.v1.ArrowRecordBatch recordBatch = readRowsResponse.getArrowRecordBatch(); rowCount = recordBatch.getRowCount(); - this.alloc = new RootAllocator(Long.MAX_VALUE); InputStream input = protoSchema.getSerializedSchema().newInput(); Schema arrowSchema = ArrowConversion.arrowSchemaFromInput(input); + RootAllocator alloc = new RootAllocator(Long.MAX_VALUE); + this.alloc = alloc; this.recordBatchIterator = ArrowConversion.rowsFromSerializedRecordBatch( - arrowSchema, recordBatch.getSerializedRecordBatch().newInput(), this.alloc); + arrowSchema, recordBatch.getSerializedRecordBatch().newInput(), alloc); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageAvroReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageAvroReader.java index 1670c399f4e4..486a5dac638d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageAvroReader.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageAvroReader.java @@ -27,14 +27,15 @@ import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DatumReader; import org.apache.avro.io.DecoderFactory; +import org.apache.beam.sdk.util.Preconditions; +import org.checkerframework.checker.nullness.qual.Nullable; -@SuppressWarnings({"nullness"}) class BigQueryStorageAvroReader implements BigQueryStorageReader { private final Schema avroSchema; private final DatumReader datumReader; - private BinaryDecoder decoder; - private GenericRecord record; + private @Nullable BinaryDecoder decoder; + private @Nullable GenericRecord record; private long rowCount; BigQueryStorageAvroReader(ReadSession readSession) { @@ -49,9 +50,13 @@ record = null; public void processReadRowsResponse(ReadRowsResponse readRowsResponse) { AvroRows avroRows = readRowsResponse.getAvroRows(); rowCount = avroRows.getRowCount(); - decoder = + @SuppressWarnings({ + "nullness" // reused decoder can be null but avro not annotated + }) + BinaryDecoder newDecoder = DecoderFactory.get() .binaryDecoder(avroRows.getSerializedBinaryRows().toByteArray(), decoder); + decoder = newDecoder; } @Override @@ -61,7 +66,12 @@ public long getRowCount() { @Override public GenericRecord readSingleRecord() throws IOException { - record = datumReader.read(record, decoder); + Preconditions.checkStateNotNull(decoder); + @SuppressWarnings({ + "nullness" // reused record can be null but avro not annotated + }) + GenericRecord newRecord = datumReader.read(record, decoder); + record = newRecord; return record; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java index ee7f02ef6689..906c66c79cec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java @@ -36,9 +36,6 @@ import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link org.apache.beam.sdk.io.Source} representing reading the results of a query. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryStorageQuerySource extends BigQueryStorageSourceBase { public static BigQueryStorageQuerySource create( @@ -100,11 +97,11 @@ public static BigQueryStorageQuerySource create( private final Boolean flattenResults; private final Boolean useLegacySql; private final QueryPriority priority; - private final String location; - private final String queryTempDataset; - private final String kmsKey; + private final @Nullable String location; + private final @Nullable String queryTempDataset; + private final @Nullable String kmsKey; - private transient AtomicReference dryRunJobStats; + private transient AtomicReference<@Nullable JobStatistics> dryRunJobStats; private BigQueryStorageQuerySource( String stepUuid, @@ -115,7 +112,7 @@ private BigQueryStorageQuerySource( @Nullable String location, @Nullable String queryTempDataset, @Nullable String kmsKey, - DataFormat format, + @Nullable DataFormat format, SerializableFunction parseFn, Coder outputCoder, BigQueryServices bqServices) { @@ -160,7 +157,7 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { } @Override - protected Table getTargetTable(BigQueryOptions options) throws Exception { + protected @Nullable Table getTargetTable(BigQueryOptions options) throws Exception { TableReference queryResultTable = BigQueryQueryHelper.executeQuery( bqServices, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java index 10ff3f49311c..3c8335ac82b5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -63,7 +64,7 @@ abstract class BigQueryStorageSourceBase extends BoundedSource { */ private static final int MIN_SPLIT_COUNT = 10; - protected final DataFormat format; + protected final @Nullable DataFormat format; protected final @Nullable ValueProvider> selectedFieldsProvider; protected final @Nullable ValueProvider rowRestrictionProvider; protected final SerializableFunction parseFn; @@ -71,7 +72,7 @@ abstract class BigQueryStorageSourceBase extends BoundedSource { protected final BigQueryServices bqServices; BigQueryStorageSourceBase( - DataFormat format, + @Nullable DataFormat format, @Nullable ValueProvider> selectedFieldsProvider, @Nullable ValueProvider rowRestrictionProvider, SerializableFunction parseFn, @@ -89,7 +90,7 @@ abstract class BigQueryStorageSourceBase extends BoundedSource { * Returns the table to read from at split time. This is currently never an anonymous table, but * it can be a named table which was created to hold the results of a query. */ - protected abstract Table getTargetTable(BigQueryOptions options) throws Exception; + protected abstract @Nullable Table getTargetTable(BigQueryOptions options) throws Exception; protected abstract @Nullable String getTargetTableId(BigQueryOptions options) throws Exception; @@ -102,7 +103,7 @@ public Coder getOutputCoder() { public List> split( long desiredBundleSizeBytes, PipelineOptions options) throws Exception { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - Table targetTable = getTargetTable(bqOptions); + @Nullable Table targetTable = getTargetTable(bqOptions); ReadSession.Builder readSessionBuilder = ReadSession.newBuilder(); if (targetTable != null) { @@ -180,6 +181,8 @@ public List> split( "data is not in a supported dataFormat: " + readSession.getDataFormat()); } + Preconditions.checkStateNotNull( + targetTable); // TODO: this is inconsistent with method above, where it can be null TableSchema trimmedSchema = BigQueryAvroUtils.trimBigQueryTableSchema(targetTable.getSchema(), sessionSchema); List> sources = Lists.newArrayList(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java index 9a2ee3b02dad..6c2ed9d54da4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.fromJsonString; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.toJsonString; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import com.google.api.gax.rpc.ApiException; import com.google.api.gax.rpc.FailedPreconditionException; @@ -44,15 +44,14 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** A {@link org.apache.beam.sdk.io.Source} representing a single stream in a read session. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryStorageStreamSource extends BoundedSource { private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSource.class); @@ -67,7 +66,7 @@ public static BigQueryStorageStreamSource create( return new BigQueryStorageStreamSource<>( readSession, readStream, - toJsonString(checkNotNull(tableSchema, "tableSchema")), + toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), parseFn, outputCoder, bqServices); @@ -96,12 +95,12 @@ private BigQueryStorageStreamSource( SerializableFunction parseFn, Coder outputCoder, BigQueryServices bqServices) { - this.readSession = checkNotNull(readSession, "readSession"); - this.readStream = checkNotNull(readStream, "stream"); - this.jsonTableSchema = checkNotNull(jsonTableSchema, "jsonTableSchema"); - this.parseFn = checkNotNull(parseFn, "parseFn"); - this.outputCoder = checkNotNull(outputCoder, "outputCoder"); - this.bqServices = checkNotNull(bqServices, "bqServices"); + this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); + this.readStream = Preconditions.checkArgumentNotNull(readStream, "stream"); + this.jsonTableSchema = Preconditions.checkArgumentNotNull(jsonTableSchema, "jsonTableSchema"); + this.parseFn = Preconditions.checkArgumentNotNull(parseFn, "parseFn"); + this.outputCoder = Preconditions.checkArgumentNotNull(outputCoder, "outputCoder"); + this.bqServices = Preconditions.checkArgumentNotNull(bqServices, "bqServices"); } @Override @@ -152,9 +151,9 @@ public static class BigQueryStorageStreamReader extends BoundedSource.Bounded private final TableSchema tableSchema; private BigQueryStorageStreamSource source; - private BigQueryServerStream responseStream; - private Iterator responseIterator; - private T current; + private @Nullable BigQueryServerStream responseStream = null; + private @Nullable Iterator responseIterator = null; + private @Nullable T current = null; private long currentOffset; // Values used for progress reporting. @@ -165,8 +164,8 @@ public static class BigQueryStorageStreamReader extends BoundedSource.Bounded private long rowsConsumedFromCurrentResponse; private long totalRowsInCurrentResponse; - private TableReference tableReference; - private ServiceCallMetric serviceCallMetric; + private @Nullable TableReference tableReference; + private @Nullable ServiceCallMetric serviceCallMetric; private BigQueryStorageStreamReader( BigQueryStorageStreamSource source, BigQueryOptions options) throws IOException { @@ -194,19 +193,22 @@ public synchronized boolean start() throws IOException { tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); + LOG.info("Started BigQuery Storage API read from stream {}.", source.readStream.getName()); responseStream = storageClient.readRows(request, source.readSession.getTable()); responseIterator = responseStream.iterator(); - LOG.info("Started BigQuery Storage API read from stream {}.", source.readStream.getName()); return readNextRecord(); } @Override public synchronized boolean advance() throws IOException { + Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } + @RequiresNonNull("responseIterator") private synchronized boolean readNextRecord() throws IOException { + Iterator responseIterator = this.responseIterator; while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { fractionConsumed = 1d; @@ -235,17 +237,17 @@ private synchronized boolean readNextRecord() throws IOException { totalRowsInCurrentResponse = response.getRowCount(); rowsConsumedFromCurrentResponse = 0L; - Preconditions.checkArgument( + checkArgument( totalRowsInCurrentResponse >= 0, "Row count from current response (%s) must be non-negative.", totalRowsInCurrentResponse); - Preconditions.checkArgument( + checkArgument( 0f <= progressAtResponseStart && progressAtResponseStart <= 1f, "Progress at response start (%s) is not in the range [0.0, 1.0].", progressAtResponseStart); - Preconditions.checkArgument( + checkArgument( 0f <= progressAtResponseEnd && progressAtResponseEnd <= 1f, "Progress at response end (%s) is not in the range [0.0, 1.0].", progressAtResponseEnd); @@ -275,11 +277,18 @@ private synchronized boolean readNextRecord() throws IOException { @Override public T getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } return current; } @Override public synchronized void close() { + // Because superclass cannot have preconditions around these variables, cannot use + // @RequiresNonNull + Preconditions.checkStateNotNull(storageClient); + Preconditions.checkStateNotNull(reader); storageClient.close(); reader.close(); } @@ -290,7 +299,11 @@ public synchronized BigQueryStorageStreamSource getCurrentSource() { } @Override - public BoundedSource splitAtFraction(double fraction) { + public @Nullable BoundedSource splitAtFraction(double fraction) { + // Because superclass cannot have preconditions around these variables, cannot use + // @RequiresNonNull + Preconditions.checkStateNotNull(responseStream); + BigQueryServerStream responseStream = this.responseStream; Metrics.counter(BigQueryStorageStreamReader.class, "split-at-fraction-calls").inc(); LOG.debug( "Received BigQuery Storage API split request for stream {} at fraction {}.", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java index dcb3dfdbe1e5..017fcd6c7e7d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java @@ -39,9 +39,6 @@ import org.slf4j.LoggerFactory; /** A {@link org.apache.beam.sdk.io.Source} representing reading from a table. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class BigQueryStorageTableSource extends BigQueryStorageSourceBase { private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageTableSource.class); @@ -49,7 +46,7 @@ public class BigQueryStorageTableSource extends BigQueryStorageSourceBase private final ValueProvider tableReferenceProvider; private final boolean projectionPushdownApplied; - private transient AtomicReference cachedTable; + private transient AtomicReference<@Nullable Table> cachedTable; public static BigQueryStorageTableSource create( ValueProvider tableRefProvider, @@ -91,7 +88,7 @@ public static BigQueryStorageTableSource create( private BigQueryStorageTableSource( ValueProvider tableRefProvider, - DataFormat format, + @Nullable DataFormat format, @Nullable ValueProvider> selectedFields, @Nullable ValueProvider rowRestriction, SerializableFunction parseFn, @@ -165,8 +162,11 @@ protected String getTargetTableId(BigQueryOptions options) throws Exception { } @Override - protected @Nullable Table getTargetTable(BigQueryOptions options) throws Exception { - if (cachedTable.get() == null) { + protected Table getTargetTable(BigQueryOptions options) throws Exception { + Table maybeTable = cachedTable.get(); + if (maybeTable != null) { + return maybeTable; + } else { TableReference tableReference = tableReferenceProvider.get(); if (Strings.isNullOrEmpty(tableReference.getProjectId())) { checkState( @@ -185,11 +185,13 @@ protected String getTargetTableId(BigQueryOptions options) throws Exception { : options.getBigQueryProject()); } try (DatasetService datasetService = bqServices.getDatasetService(options)) { - Table table = datasetService.getTable(tableReference); + Table table = bqServices.getDatasetService(options).getTable(tableReference); + if (table == null) { + throw new IllegalArgumentException("Table not found" + table); + } cachedTable.compareAndSet(null, table); + return table; } } - - return cachedTable.get(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java index 7f16baf5c1af..f944ae673b63 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java @@ -27,12 +27,10 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link BigQuerySourceBase} for reading BigQuery tables. */ @VisibleForTesting -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryTableSource extends BigQuerySourceBase { static BigQueryTableSource create( @@ -47,7 +45,7 @@ static BigQueryTableSource create( } private final BigQueryTableSourceDef tableDef; - private final AtomicReference tableSizeBytes; + private final AtomicReference<@Nullable Long> tableSizeBytes; private BigQueryTableSource( String stepUuid, @@ -68,11 +66,19 @@ protected TableReference getTableToExtract(BigQueryOptions bqOptions) throws IOE @Override public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - if (tableSizeBytes.get() == null) { + Long maybeNumBytes = tableSizeBytes.get(); + if (maybeNumBytes != null) { + return maybeNumBytes; + } else { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); TableReference tableRef = tableDef.getTableReference(bqOptions); try (DatasetService datasetService = bqServices.getDatasetService(bqOptions)) { Table table = datasetService.getTable(tableRef); + + if (table == null) { + throw new IllegalStateException("Table not found: " + table); + } + Long numBytes = table.getNumBytes(); if (table.getStreamingBuffer() != null && table.getStreamingBuffer().getEstimatedBytes() != null) { @@ -80,9 +86,9 @@ public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws E } tableSizeBytes.compareAndSet(null, numBytes); + return numBytes; } } - return tableSizeBytes.get(); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java index 9d86afb95cae..72de3f56e6c3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; @@ -30,13 +31,11 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class BigQueryTableSourceDef implements BigQuerySourceDef { private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableSourceDef.class); @@ -109,7 +108,8 @@ public Schema getBeamSchema(BigQueryOptions bqOptions) { try { try (DatasetService datasetService = bqServices.getDatasetService(bqOptions)) { TableReference tableRef = getTableReference(bqOptions); - TableSchema tableSchema = datasetService.getTable(tableRef).getSchema(); + Table table = datasetService.getTable(tableRef); + TableSchema tableSchema = Preconditions.checkStateNotNull(table).getSchema(); return BigQueryUtils.fromTableSchema(tableSchema); } } catch (Exception e) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index d8d83c9f611f..371bf6951f2f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -66,6 +66,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; @@ -84,7 +85,7 @@ /** Utility methods for BigQuery related operations. */ @SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10608) }) public class BigQueryUtils { @@ -226,8 +227,8 @@ public abstract static class Builder { .put(TypeName.BYTES, StandardSQLTypeName.BYTES) .build(); - private static final Map> JSON_VALUE_PARSERS = - ImmutableMap.>builder() + private static final Map> JSON_VALUE_PARSERS = + ImmutableMap.>builder() .put(TypeName.BYTE, Byte::valueOf) .put(TypeName.INT16, Short::valueOf) .put(TypeName.INT32, Integer::valueOf) @@ -275,11 +276,13 @@ public abstract static class Builder { static StandardSQLTypeName toStandardSQLTypeName(FieldType fieldType) { StandardSQLTypeName ret; if (fieldType.getTypeName().isLogicalType()) { - ret = BEAM_TO_BIGQUERY_LOGICAL_MAPPING.get(fieldType.getLogicalType().getIdentifier()); + Schema.LogicalType logicalType = + Preconditions.checkArgumentNotNull(fieldType.getLogicalType()); + ret = BEAM_TO_BIGQUERY_LOGICAL_MAPPING.get(logicalType.getIdentifier()); if (ret == null) { throw new IllegalArgumentException( "Cannot convert Beam logical type: " - + fieldType.getLogicalType().getIdentifier() + + logicalType.getIdentifier() + " to BigQuery type."); } } else { @@ -390,21 +393,23 @@ private static List toTableFieldSchema(Schema schema) { field.setMode(Mode.REQUIRED.toString()); } if (type.getTypeName().isCollectionType()) { - type = type.getCollectionElementType(); + type = Preconditions.checkArgumentNotNull(type.getCollectionElementType()); if (type.getTypeName().isCollectionType() || type.getTypeName().isMapType()) { throw new IllegalArgumentException("Array of collection is not supported in BigQuery."); } field.setMode(Mode.REPEATED.toString()); } if (TypeName.ROW == type.getTypeName()) { - Schema subType = type.getRowSchema(); + Schema subType = Preconditions.checkArgumentNotNull(type.getRowSchema()); field.setFields(toTableFieldSchema(subType)); } if (TypeName.MAP == type.getTypeName()) { + FieldType mapKeyType = Preconditions.checkArgumentNotNull(type.getMapKeyType()); + FieldType mapValueType = Preconditions.checkArgumentNotNull(type.getMapValueType()); Schema mapSchema = Schema.builder() - .addField(BIGQUERY_MAP_KEY_FIELD_NAME, type.getMapKeyType()) - .addField(BIGQUERY_MAP_VALUE_FIELD_NAME, type.getMapValueType()) + .addField(BIGQUERY_MAP_KEY_FIELD_NAME, mapKeyType) + .addField(BIGQUERY_MAP_VALUE_FIELD_NAME, mapValueType) .build(); type = FieldType.row(mapSchema); field.setFields(toTableFieldSchema(mapSchema)); @@ -667,7 +672,7 @@ public static Row toBeamRow(Schema rowSchema, TableSchema bqSchema, TableRow jso .collect(toRow(rowSchema)); } - private static Object toBeamValue(FieldType fieldType, Object jsonBQValue) { + private static @Nullable Object toBeamValue(FieldType fieldType, Object jsonBQValue) { if (jsonBQValue instanceof String || jsonBQValue instanceof Number || jsonBQValue instanceof Boolean) { @@ -974,8 +979,8 @@ private static Object convertAvroNumeric(Object value) { return null; } - private static ServiceCallMetric callMetricForMethod( - TableReference tableReference, String method) { + private static @Nullable ServiceCallMetric callMetricForMethod( + @Nullable TableReference tableReference, String method) { if (tableReference != null) { // TODO(ajamato): Add Ptransform label. Populate it as empty for now to prevent the // SpecMonitoringInfoValidator from dropping the MonitoringInfo. @@ -1006,7 +1011,8 @@ private static ServiceCallMetric callMetricForMethod( * elements directly from BigQuery in a process-wide metric. Such as: calls to readRows, * splitReadStream, createReadSession. */ - public static ServiceCallMetric readCallMetric(TableReference tableReference) { + public static @Nullable ServiceCallMetric readCallMetric( + @Nullable TableReference tableReference) { return callMetricForMethod(tableReference, "BigQueryBatchRead"); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java index 860ad4a1d6fb..0c2babdeef87 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java @@ -22,7 +22,6 @@ import com.google.api.services.bigquery.model.TableSchema; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.transforms.DoFn; @@ -34,6 +33,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; 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.checkerframework.checker.nullness.qual.Nullable; /** * Creates any tables needed before performing writes to the tables. This is a side-effect {@link @@ -45,7 +45,7 @@ public class CreateTableDestinations private final CreateDisposition createDisposition; private final BigQueryServices bqServices; private final DynamicDestinations dynamicDestinations; - @Nullable private final String kmsKey; + private final @Nullable String kmsKey; public CreateTableDestinations( CreateDisposition createDisposition, @@ -110,7 +110,8 @@ public void processElement( dest); @Nullable Coder destinationCoder = dynamicDestinations.getDestinationCoder(); - Supplier schemaSupplier = () -> dynamicDestinations.getSchema(dest); + Supplier<@Nullable TableSchema> schemaSupplier = + () -> dynamicDestinations.getSchema(dest); return CreateTableHelpers.possiblyCreateTable( context, tableDestination1, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java index aa7a4168a371..403554b263c4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java @@ -19,21 +19,24 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import com.google.api.services.bigquery.model.Clustering; import com.google.api.services.bigquery.model.EncryptionConfiguration; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; +import com.google.api.services.bigquery.model.TimePartitioning; import java.util.Collections; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.Preconditions; 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.Strings; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; +import org.checkerframework.checker.nullness.qual.Nullable; public class CreateTableHelpers { /** @@ -46,7 +49,7 @@ public class CreateTableHelpers { static TableDestination possiblyCreateTable( DoFn.ProcessContext context, TableDestination tableDestination, - Supplier schemaSupplier, + Supplier<@Nullable TableSchema> schemaSupplier, CreateDisposition createDisposition, @Nullable Coder tableDestinationCoder, @Nullable String kmsKey, @@ -97,10 +100,9 @@ static TableDestination possiblyCreateTable( return tableDestination; } - @SuppressWarnings({"nullness"}) private static void tryCreateTable( DoFn.ProcessContext context, - Supplier schemaSupplier, + Supplier<@Nullable TableSchema> schemaSupplier, TableDestination tableDestination, CreateDisposition createDisposition, String tableSpec, @@ -112,8 +114,8 @@ private static void tryCreateTable( bqServices.getDatasetService(context.getPipelineOptions().as(BigQueryOptions.class))) { if (datasetService.getTable(tableReference) == null) { TableSchema tableSchema = schemaSupplier.get(); - checkArgument( - tableSchema != null, + Preconditions.checkArgumentNotNull( + tableSchema, "Unless create disposition is %s, a schema must be specified, i.e. " + "DynamicDestinations.getSchema() may not return null. " + "However, create disposition is %s, and " @@ -122,13 +124,18 @@ private static void tryCreateTable( createDisposition, tableDestination); Table table = new Table().setTableReference(tableReference).setSchema(tableSchema); - if (tableDestination.getTableDescription() != null) { - table = table.setDescription(tableDestination.getTableDescription()); + + String tableDescription = tableDestination.getTableDescription(); + if (tableDescription != null) { + table = table.setDescription(tableDescription); } - if (tableDestination.getTimePartitioning() != null) { - table.setTimePartitioning(tableDestination.getTimePartitioning()); - if (tableDestination.getClustering() != null) { - table.setClustering(tableDestination.getClustering()); + + TimePartitioning timePartitioning = tableDestination.getTimePartitioning(); + if (timePartitioning != null) { + table.setTimePartitioning(timePartitioning); + Clustering clustering = tableDestination.getClustering(); + if (clustering != null) { + table.setClustering(clustering); } } if (kmsKey != null) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 8eef0677f820..278f49511c0a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -33,21 +34,19 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; 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.checkerframework.checker.nullness.qual.Nullable; /** * Creates any tables needed before performing streaming writes to the tables. This is a side-effect * {@link DoFn}, and returns the original collection unchanged. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class CreateTables extends PTransform< PCollection>, PCollection>> { private final CreateDisposition createDisposition; private final BigQueryServices bqServices; private final DynamicDestinations dynamicDestinations; - private final String kmsKey; + private final @Nullable String kmsKey; /** * The list of tables created so far, so we don't try the creation each time. @@ -64,14 +63,14 @@ private CreateTables( CreateDisposition createDisposition, BigQueryServices bqServices, DynamicDestinations dynamicDestinations, - String kmsKey) { + @Nullable String kmsKey) { this.createDisposition = createDisposition; this.bqServices = bqServices; this.dynamicDestinations = dynamicDestinations; this.kmsKey = kmsKey; } - CreateTables withKmsKey(String kmsKey) { + CreateTables withKmsKey(@Nullable String kmsKey) { return new CreateTables<>(createDisposition, bqServices, dynamicDestinations, kmsKey); } @@ -90,7 +89,7 @@ public PCollection> expand( private class CreateTablesFn extends DoFn, KV> { - private Map destinations; + private @Nullable Map destinations = null; @StartBundle public void startBundle() { @@ -100,6 +99,7 @@ public void startBundle() { @ProcessElement public void processElement(ProcessContext context) { dynamicDestinations.setSideInputAccessorFromProcessContext(context); + Preconditions.checkStateNotNull(destinations); TableDestination tableDestination = destinations.computeIfAbsent( context.element().getKey(), @@ -111,7 +111,8 @@ public void processElement(ProcessContext context) { + "but %s returned null for destination %s", dynamicDestinations, dest); - Supplier schemaSupplier = () -> dynamicDestinations.getSchema(dest); + Supplier<@Nullable TableSchema> schemaSupplier = + () -> dynamicDestinations.getSchema(dest); return CreateTableHelpers.possiblyCreateTable( context, tableDestination1, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java index 8f2b123ff3d5..f10cfc0e24c5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java @@ -73,9 +73,6 @@ * be a compact type with an efficient coder, as these objects may be used as a key in a {@link * org.apache.beam.sdk.transforms.GroupByKey}. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public abstract class DynamicDestinations implements Serializable { interface SideInputAccessor { SideInputT sideInput(PCollectionView view); @@ -120,6 +117,9 @@ protected final SideInputT sideInput(PCollectionView vi "View %s not declared in getSideInputs() (%s)", view, getSideInputs()); + if (sideInputAccessor == null) { + throw new IllegalStateException("sideInputAccessor (transient field) is null"); + } return sideInputAccessor.sideInput(view); } @@ -132,7 +132,7 @@ void setSideInputAccessorFromProcessContext(DoFn.ProcessContext context) { * Returns an object that represents at a high level which table is being written to. May not * return null. */ - public abstract DestinationT getDestination(ValueInSingleWindow element); + public abstract DestinationT getDestination(@Nullable ValueInSingleWindow element); /** * Returns the coder for {@link DestinationT}. If this is not overridden, then {@link BigQueryIO} @@ -151,8 +151,8 @@ void setSideInputAccessorFromProcessContext(DoFn.ProcessContext context) { */ public abstract TableDestination getTable(DestinationT destination); - /** Returns the table schema for the destination. May not return null. */ - public abstract TableSchema getSchema(DestinationT destination); + /** Returns the table schema for the destination. */ + public abstract @Nullable TableSchema getSchema(DestinationT destination); // Gets the destination coder. If the user does not provide one, try to find one in the coder // registry. If no coder can be found, throws CannotProvideCoderException. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 0f6b4ccc8b2b..e96bfe707078 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -35,11 +35,13 @@ import org.apache.beam.sdk.extensions.gcp.util.BackOffAdapter; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableSpec; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; @@ -50,9 +52,6 @@ import org.slf4j.LoggerFactory; /** Contains some useful helper instances of {@link DynamicDestinations}. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class DynamicDestinationsHelpers { private static final Logger LOG = LoggerFactory.getLogger(DynamicDestinationsHelpers.class); @@ -85,14 +84,14 @@ static ConstantTableDestinations fromJsonTableRef( } @Override - public TableDestination getDestination(ValueInSingleWindow element) { + public TableDestination getDestination(@Nullable ValueInSingleWindow element) { String tableSpec = this.tableSpec.get(); checkArgument(tableSpec != null, "tableSpec can not be null"); return new TableDestination(tableSpec, tableDescription); } @Override - public TableSchema getSchema(TableDestination destination) { + public @Nullable TableSchema getSchema(TableDestination destination) { return null; } @@ -113,18 +112,19 @@ public Coder getDestinationCoder() { /** Returns tables based on a user-supplied function. */ static class TableFunctionDestinations extends DynamicDestinations { - private final SerializableFunction, TableDestination> tableFunction; + private final SerializableFunction<@Nullable ValueInSingleWindow, TableDestination> + tableFunction; private final boolean clusteringEnabled; TableFunctionDestinations( - SerializableFunction, TableDestination> tableFunction, + SerializableFunction<@Nullable ValueInSingleWindow, TableDestination> tableFunction, boolean clusteringEnabled) { this.tableFunction = tableFunction; this.clusteringEnabled = clusteringEnabled; } @Override - public TableDestination getDestination(ValueInSingleWindow element) { + public TableDestination getDestination(@Nullable ValueInSingleWindow element) { TableDestination res = tableFunction.apply(element); checkArgument( res != null, @@ -135,7 +135,7 @@ public TableDestination getDestination(ValueInSingleWindow element) { } @Override - public TableSchema getSchema(TableDestination destination) { + public @Nullable TableSchema getSchema(TableDestination destination) { return null; } @@ -168,12 +168,12 @@ static class DelegatingDynamicDestinations } @Override - public DestinationT getDestination(ValueInSingleWindow element) { + public DestinationT getDestination(@Nullable ValueInSingleWindow element) { return inner.getDestination(element); } @Override - public TableSchema getSchema(DestinationT destination) { + public @Nullable TableSchema getSchema(DestinationT destination) { return inner.getSchema(destination); } @@ -183,7 +183,7 @@ public TableDestination getTable(DestinationT destination) { } @Override - public Coder getDestinationCoder() { + public @Nullable Coder getDestinationCoder() { return inner.getDestinationCoder(); } @@ -217,12 +217,12 @@ public String toString() { /** Returns the same schema for every table. */ static class ConstantSchemaDestinations extends DelegatingDynamicDestinations { - private final @Nullable ValueProvider jsonSchema; + private final ValueProvider jsonSchema; ConstantSchemaDestinations( DynamicDestinations inner, ValueProvider jsonSchema) { super(inner); - checkArgument(jsonSchema != null, "jsonSchema can not be null"); + Preconditions.checkArgumentNotNull(jsonSchema, "jsonSchema can not be null"); this.jsonSchema = jsonSchema; } @@ -245,7 +245,7 @@ public String toString() { static class ConstantTimePartitioningDestinations extends DelegatingDynamicDestinations { - private final @Nullable ValueProvider jsonTimePartitioning; + private final ValueProvider jsonTimePartitioning; private final @Nullable ValueProvider jsonClustering; ConstantTimePartitioningDestinations( @@ -253,16 +253,18 @@ static class ConstantTimePartitioningDestinations ValueProvider jsonTimePartitioning, ValueProvider jsonClustering) { super(inner); - checkArgument(jsonTimePartitioning != null, "jsonTimePartitioning provider can not be null"); + Preconditions.checkArgumentNotNull( + jsonTimePartitioning, "jsonTimePartitioning provider can not be null"); if (jsonTimePartitioning.isAccessible()) { - checkArgument(jsonTimePartitioning.get() != null, "jsonTimePartitioning can not be null"); + Preconditions.checkArgumentNotNull( + jsonTimePartitioning.get(), "jsonTimePartitioning can not be null"); } this.jsonTimePartitioning = jsonTimePartitioning; this.jsonClustering = jsonClustering; } @Override - public TableDestination getDestination(ValueInSingleWindow element) { + public TableDestination getDestination(@Nullable ValueInSingleWindow element) { TableDestination destination = super.getDestination(element); String partitioning = this.jsonTimePartitioning.get(); checkArgument(partitioning != null, "jsonTimePartitioning can not be null"); @@ -320,8 +322,8 @@ public List> getSideInputs() { public TableSchema getSchema(TableDestination destination) { Map mapValue = sideInput(schemaView); String schema = mapValue.get(destination.getTableSpec()); - checkArgument( - schema != null, + Preconditions.checkArgumentNotNull( + schema, "Schema view must contain data for every destination used, " + "but view %s does not contain data for table destination %s " + "produced by %s", @@ -355,7 +357,7 @@ private MatchTableDynamicDestinations( this.bqServices = bqServices; } - private Table getBigQueryTable(TableReference tableReference) { + private @Nullable Table getBigQueryTable(TableReference tableReference) { BackOff backoff = BackOffAdapter.toGcpBackOff( FluentBackoff.DEFAULT @@ -366,7 +368,11 @@ private Table getBigQueryTable(TableReference tableReference) { try { do { try { - BigQueryOptions bqOptions = getPipelineOptions().as(BigQueryOptions.class); + PipelineOptions options = getPipelineOptions(); + if (options == null) { + throw new IllegalStateException("pipeline options cannot be null"); + } + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); if (tableReference.getProjectId() == null) { tableReference.setProjectId( bqOptions.getBigQueryProject() == null @@ -415,11 +421,11 @@ public TableDestination getTable(DestinationT destination) { } } - /** Returns the table schema for the destination. May not return null. */ + /** Returns the table schema for the destination. */ @Override - public TableSchema getSchema(DestinationT destination) { + public @Nullable TableSchema getSchema(DestinationT destination) { TableDestination wrappedDestination = super.getTable(destination); - Table existingTable = getBigQueryTable(wrappedDestination.getTableReference()); + @Nullable Table existingTable = getBigQueryTable(wrappedDestination.getTableReference()); if (existingTable == null) { return super.getSchema(destination); } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java index 0e74c79639c5..652c91a4a78c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java @@ -22,6 +22,7 @@ import com.google.api.services.bigquery.model.TableRow; import java.io.Serializable; import java.util.List; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.FailsafeValueInSingleWindow; import org.apache.beam.sdk.values.ValueInSingleWindow; @@ -30,9 +31,6 @@ * * @param */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public interface ErrorContainer extends Serializable { void add( List> failedInserts, @@ -44,8 +42,10 @@ void add( (failedInserts, error, ref, tableRow) -> failedInserts.add( ValueInSingleWindow.of( - tableRow.getFailsafeValue(), tableRow.getTimestamp(), - tableRow.getWindow(), tableRow.getPane())); + Preconditions.checkArgumentNotNull(tableRow.getFailsafeValue()), + tableRow.getTimestamp(), + tableRow.getWindow(), + tableRow.getPane())); ErrorContainer BIG_QUERY_INSERT_ERROR_ERROR_CONTAINER = (failedInserts, error, ref, tableRow) -> { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java index 07570a9b9289..1e00e877b24a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java @@ -27,9 +27,11 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.checkerframework.checker.nullness.qual.NonNull; import org.joda.time.Instant; /** @@ -37,10 +39,7 @@ * which tables each element is written to, and format the element into a {@link TableRow} using the * user-supplied format function. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) -public class PrepareWrite +public class PrepareWrite extends PTransform, PCollection>> { private DynamicDestinations dynamicDestinations; private SerializableFunction formatFunction; @@ -77,8 +76,8 @@ public void processElement( dynamicDestinations, element); OutputT outputValue = formatFunction.apply(element); - checkArgument( - outputValue != null, + Preconditions.checkArgumentNotNull( + outputValue, "formatFunction may not return null, but %s returned null on element %s", formatFunction, element); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java index 58be37f426c8..8a4bf390586f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java @@ -26,21 +26,19 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.checkerframework.checker.nullness.qual.Nullable; /** * This transforms turns a side input into a singleton PCollection that can be used as the main * input for another transform. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class ReifyAsIterable extends PTransform, PCollection>> { @Override public PCollection> expand(PCollection input) { final PCollectionView> view = input.apply(View.asIterable()); return input .getPipeline() - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply(Create.<@Nullable Void>of((@Nullable Void) null).withCoder(VoidCoder.of())) .apply( ParDo.of( new DoFn>() { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java index dd86f84cbfb6..34a64c1baeb9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + import com.google.api.core.ApiFuture; import com.google.api.core.ApiFutureCallback; import com.google.api.core.ApiFutures; @@ -34,8 +36,8 @@ import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.Sleeper; -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.Queues; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; @@ -111,7 +113,7 @@ public void setResult(@Nullable ResultT result) { private final Function hasSucceeded; @Nullable private ApiFuture future = null; @Nullable private Callback callback = null; - @Nullable ContextT context = null; + ContextT context; public Operation( Function> runOperation, @@ -126,15 +128,14 @@ public Operation( this.context = context; } - @SuppressWarnings({"nullness"}) void run(Executor executor) { this.future = runOperation.apply(context); this.callback = new Callback<>(hasSucceeded); ApiFutures.addCallback(future, callback, executor); } - @SuppressWarnings({"nullness"}) boolean await() throws Exception { + Preconditions.checkStateNotNull(callback); callback.await(); return callback.getFailed(); } @@ -251,13 +252,12 @@ void run(boolean await) throws Exception { } } - @SuppressWarnings({"nullness"}) void await() throws Exception { while (!this.operations.isEmpty()) { Operation operation = this.operations.element(); boolean failed = operation.await(); if (failed) { - Throwable failure = operation.callback.getFailure(); + Throwable failure = Preconditions.checkStateNotNull(operation.callback).getFailure(); operation.context.setError(failure); RetryType retryType = operation.onError.apply( @@ -265,7 +265,7 @@ void await() throws Exception { if (retryType == RetryType.DONT_RETRY) { operations.clear(); } else { - Preconditions.checkState(RetryType.RETRY_ALL_OPERATIONS == retryType); + checkState(RetryType.RETRY_ALL_OPERATIONS == retryType); if (!BackOffUtils.next(Sleeper.DEFAULT, backoff)) { throw new RuntimeException(failure); } @@ -276,7 +276,7 @@ void await() throws Exception { run(false); } } else { - operation.context.setResult(operation.future.get()); + operation.context.setResult(Preconditions.checkStateNotNull(operation.future).get()); operation.onSuccess.accept(operation.context); operations.remove(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java index 22f11ccee8a0..ad851706d8b0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java @@ -23,10 +23,8 @@ import org.apache.avro.Schema; import org.apache.avro.io.DatumWriter; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.checkerframework.checker.nullness.qual.Nullable; -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) abstract class RowWriterFactory implements Serializable { private RowWriterFactory() {} @@ -101,14 +99,14 @@ static final class AvroRowWriterFactory private final SerializableFunction, AvroT> toAvro; private final SerializableFunction> writerFactory; - private final SerializableFunction schemaFactory; - private final DynamicDestinations dynamicDestinations; + private final @Nullable SerializableFunction schemaFactory; + private final @Nullable DynamicDestinations dynamicDestinations; private AvroRowWriterFactory( SerializableFunction, AvroT> toAvro, SerializableFunction> writerFactory, - SerializableFunction schemaFactory, - DynamicDestinations dynamicDestinations) { + @Nullable SerializableFunction schemaFactory, + @Nullable DynamicDestinations dynamicDestinations) { this.toAvro = toAvro; this.writerFactory = writerFactory; this.schemaFactory = schemaFactory; @@ -129,7 +127,18 @@ OutputType getOutputType() { @Override BigQueryRowWriter createRowWriter(String tempFilePrefix, DestinationT destination) throws Exception { + if (dynamicDestinations == null) { + throw new IllegalStateException( + "createRowWriter called when dynamicDestinations is null; forgot to call prepare()?"); + } + if (schemaFactory == null) { + throw new IllegalStateException( + "createRowWriter called when schemaFactory is null; forgot to call prepare()?"); + } TableSchema tableSchema = dynamicDestinations.getSchema(destination); + if (tableSchema == null) { + throw new IllegalStateException("dynamicDestinations.getSchema returned null"); + } Schema avroSchema = schemaFactory.apply(tableSchema); return new AvroRowWriter<>(tempFilePrefix, avroSchema, toAvro, writerFactory); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java index f6faf2a9c072..a14409788b2d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java @@ -19,7 +19,6 @@ import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; @@ -32,6 +31,8 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A transform that converts messages to protocol buffers in preparation for writing to BigQuery. @@ -81,7 +82,7 @@ public PCollectionTuple expand(PCollection> input) { return result; } - public static class ConvertMessagesDoFn + public static class ConvertMessagesDoFn extends DoFn, KV> { private final StorageApiDynamicDestinations dynamicDestinations; private TwoLevelMessageConverterCache messageConverters; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java index 2e0ce6a38dab..53d640a00f27 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java @@ -67,7 +67,7 @@ public abstract MessageConverter getMessageConverter( DestinationT destination, DatasetService datasetService) throws Exception; @Override - public DestinationT getDestination(ValueInSingleWindow element) { + public DestinationT getDestination(@Nullable ValueInSingleWindow element) { return inner.getDestination(element); } @@ -82,7 +82,7 @@ public TableDestination getTable(DestinationT destination) { } @Override - public TableSchema getSchema(DestinationT destination) { + public @Nullable TableSchema getSchema(DestinationT destination) { return inner.getSchema(destination); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java index 941620891f2d..5f85cc1eb1b1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.Row; +import org.checkerframework.checker.nullness.qual.NonNull; -@SuppressWarnings({"nullness"}) /** Storage API DynamicDestinations used when the input is a Beam Row. */ -class StorageApiDynamicDestinationsBeamRow +class StorageApiDynamicDestinationsBeamRow extends StorageApiDynamicDestinations { private final Schema schema; private final SerializableFunction toRow; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java index c486b19e11a9..b33b220de5af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java @@ -28,12 +28,13 @@ import org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto.SchemaTooNarrowException; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings({"nullness"}) -public class StorageApiDynamicDestinationsTableRow +public class StorageApiDynamicDestinationsTableRow extends StorageApiDynamicDestinations { private final SerializableFunction formatFunction; private final CreateDisposition createDisposition; @@ -69,7 +70,7 @@ static void clearSchemaCache() throws ExecutionException, InterruptedException { public MessageConverter getMessageConverter( DestinationT destination, DatasetService datasetService) throws Exception { return new MessageConverter() { - TableSchema tableSchema; + @Nullable TableSchema tableSchema; TableRowToStorageApiProto.SchemaInformation schemaInformation; Descriptor descriptor; long descriptorHash; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java index 2744603dff2f..3a854f5152f3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.Map; import java.util.Set; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; @@ -33,11 +32,13 @@ import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; 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.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,7 +95,6 @@ public void startBundle() throws IOException { } @ProcessElement - @SuppressWarnings({"nullness"}) public void process(PipelineOptions pipelineOptions, @Element KV element) throws Exception { String tableId = element.getKey(); @@ -109,11 +109,9 @@ public void process(PipelineOptions pipelineOptions, @Element KV return datasetService.finalizeWriteStream(streamId); }, contexts -> { - LOG.error( - "Finalize of stream " - + streamId - + " failed with " - + Iterables.getFirst(contexts, null).getError()); + RetryManager.Operation.Context firstContext = + Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)); + LOG.error("Finalize of stream " + streamId + " failed with " + firstContext.getError()); finalizeOperationsFailed.inc(); return RetryType.RETRY_ALL_OPERATIONS; }, @@ -127,7 +125,6 @@ public void process(PipelineOptions pipelineOptions, @Element KV } @FinishBundle - @SuppressWarnings({"nullness"}) public void finishBundle(PipelineOptions pipelineOptions) throws Exception { DatasetService datasetService = getDatasetService(pipelineOptions); for (Map.Entry> entry : commitStreams.entrySet()) { @@ -139,19 +136,24 @@ public void finishBundle(PipelineOptions pipelineOptions) throws Exception { new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3); retryManager.addOperation( c -> { + @SuppressWarnings({ + "nullness" // unsure why s is inferred to be @Nullable + }) Iterable streamsToCommit = Iterables.filter(streamNames, s -> !alreadyCommittedStreams.contains(s)); batchCommitOperationsSent.inc(); return datasetService.commitWriteStreams(tableId, streamsToCommit); }, contexts -> { + RetryManager.Operation.Context firstContext = + Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)); LOG.error( "BatchCommit failed. tableId " + tableId + " streamNames " + streamNames + " error: " - + Iterables.getFirst(contexts, null).getError()); + + firstContext.getError()); batchCommitOperationsFailed.inc(); return RetryType.RETRY_ALL_OPERATIONS; }, @@ -167,6 +169,9 @@ public void finishBundle(PipelineOptions pipelineOptions) throws Exception { alreadyCommittedStreams.add(storageError.getEntity()); } } + @SuppressWarnings({ + "nullness" // unsure why s is inferred to be @Nullable + }) Iterable streamsToCommit = Iterables.filter(streamNames, s -> !alreadyCommittedStreams.contains(s)); // If there are no more streams left to commit, then report this operation as having diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java index 719e1ddb97b7..c28ef2d5eacd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaCreate; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.joda.time.Duration; @@ -130,7 +131,6 @@ public void onTeardown() { } } - @SuppressWarnings({"nullness"}) @ProcessElement public void process(PipelineOptions pipelineOptions, @Element KV element) throws Exception { @@ -155,7 +155,8 @@ public void process(PipelineOptions pipelineOptions, @Element KV { - Throwable error = Iterables.getFirst(contexts, null).getError(); + Throwable error = + Preconditions.checkArgumentNotNull(Iterables.getFirst(contexts, null)).getError(); LOG.warn( "Flush of stream " + streamId + " to offset " + offset + " failed with " + error); flushOperationsFailed.inc(); @@ -208,9 +209,12 @@ public void process(PipelineOptions pipelineOptions, @Element KV firstContext = Iterables.getFirst(contexts, null); + @Nullable Throwable error = firstContext == null ? null : firstContext.getError(); if (error instanceof ApiException) { Code statusCode = ((ApiException) error).getStatusCode().getCode(); if (statusCode.equals(Code.NOT_FOUND)) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 7995660af58a..2e627e86c74c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -35,7 +35,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; @@ -54,25 +53,27 @@ import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -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.Cache; 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.RemovalNotification; 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.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings({"nullness", "FutureReturnValueIgnored"}) /** * Write records to the Storage API using a standard batch approach. PENDING streams are used, which * do not become visible until they are finalized and committed. Each input bundle to the DoFn * creates a stream per output table, appends all records in the bundle to the stream, and schedules * a finalize/commit operation at the end. */ +@SuppressWarnings({"FutureReturnValueIgnored"}) public class StorageApiWriteUnshardedRecords extends PTransform>, PCollection> { private static final Logger LOG = LoggerFactory.getLogger(StorageApiWriteUnshardedRecords.class); @@ -93,7 +94,7 @@ public class StorageApiWriteUnshardedRecords .removalListener( (RemovalNotification removal) -> { LOG.info("Expiring append client for " + removal.getKey()); - @Nullable final StreamAppendClient streamAppendClient = removal.getValue(); + final @Nullable StreamAppendClient streamAppendClient = removal.getValue(); // Close the writer in a different thread so as not to block the main one. runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close); }) @@ -151,7 +152,7 @@ public PCollection expand(PCollection + static class WriteRecordsDoFn extends DoFn, KV> { private final Counter forcedFlushes = Metrics.counter(WriteRecordsDoFn.class, "forcedFlushes"); @@ -162,7 +163,7 @@ class DestinationState { private @Nullable StreamAppendClient streamAppendClient = null; private long currentOffset = 0; private List pendingMessages; - private transient @Nullable DatasetService datasetService; + private transient @Nullable DatasetService maybeDatasetService; private final Counter recordsAppended = Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); private final Counter appendFailures = @@ -173,7 +174,7 @@ class DestinationState { Metrics.distribution(WriteRecordsDoFn.class, "streamWriterWaitSeconds"); private final boolean useDefaultStream; private DescriptorWrapper descriptorWrapper; - private Instant nextCacheTickle; + private Instant nextCacheTickle = Instant.MAX; private final int clientNumber; public DestinationState( @@ -185,7 +186,7 @@ public DestinationState( this.tableUrn = tableUrn; this.messageConverter = messageConverter; this.pendingMessages = Lists.newArrayList(); - this.datasetService = datasetService; + this.maybeDatasetService = datasetService; this.useDefaultStream = useDefaultStream; this.descriptorWrapper = messageConverter.getSchemaDescriptor(); this.clientNumber = new Random().nextInt(streamAppendClientCount); @@ -214,7 +215,7 @@ String createStreamIfNeeded() { try { if (!useDefaultStream) { this.streamName = - Preconditions.checkNotNull(datasetService) + Preconditions.checkStateNotNull(maybeDatasetService) .createWriteStream(tableUrn, Type.PENDING) .getName(); } else { @@ -227,27 +228,30 @@ String createStreamIfNeeded() { } StreamAppendClient generateClient() throws Exception { - return datasetService.getStreamAppendClient(streamName, descriptorWrapper.descriptor); + Preconditions.checkStateNotNull(maybeDatasetService); + return maybeDatasetService.getStreamAppendClient(streamName, descriptorWrapper.descriptor); } StreamAppendClient getStreamAppendClient(boolean lookupCache) { try { - if (streamAppendClient == null) { + if (this.streamAppendClient == null) { createStreamIfNeeded(); + final StreamAppendClient newStreamAppendClient; synchronized (APPEND_CLIENTS) { if (lookupCache) { - this.streamAppendClient = + newStreamAppendClient = APPEND_CLIENTS.get( getStreamAppendClientCacheEntryKey(), () -> generateClient()); } else { - this.streamAppendClient = generateClient(); + newStreamAppendClient = generateClient(); // override the clients in the cache - APPEND_CLIENTS.put(getStreamAppendClientCacheEntryKey(), streamAppendClient); + APPEND_CLIENTS.put(getStreamAppendClientCacheEntryKey(), newStreamAppendClient); } - this.streamAppendClient.pin(); + newStreamAppendClient.pin(); } this.currentOffset = 0; nextCacheTickle = Instant.now().plus(java.time.Duration.ofMinutes(1)); + this.streamAppendClient = newStreamAppendClient; } return streamAppendClient; } catch (Exception e) { @@ -317,7 +321,6 @@ void addMessage(StorageApiWritePayload payload) throws Exception { pendingMessages.add(ByteString.copyFrom(payload.getPayload())); } - @SuppressWarnings({"nullness"}) void flush(RetryManager> retryManager) throws Exception { if (pendingMessages.isEmpty()) { @@ -369,12 +372,12 @@ void flush(RetryManager> retryMa String retrieveErrorDetails(Iterable> contexts) { return StreamSupport.stream(contexts.spliterator(), false) - .map(ctx -> ctx.getError()) + .map(ctx -> Preconditions.checkStateNotNull(ctx.getError())) .map( err -> String.format( "message: %s, stacktrace: %s", - err.toString(), + err, Lists.newArrayList(err.getStackTrace()).stream() .map(se -> se.toString()) .collect(Collectors.joining("\n")))) @@ -382,9 +385,9 @@ String retrieveErrorDetails(Iterable> contexts) { } } - private Map destinations = Maps.newHashMap(); + private @Nullable Map destinations = Maps.newHashMap(); private final TwoLevelMessageConverterCache messageConverters; - private transient @Nullable DatasetService datasetService; + private transient @Nullable DatasetService maybeDatasetService; private int numPendingRecords = 0; private int numPendingRecordBytes = 0; private final int flushThresholdBytes; @@ -429,6 +432,7 @@ void flushAll() throws Exception { RetryManager> retryManager = new RetryManager<>(Duration.standardSeconds(1), Duration.standardSeconds(10), 1000); + Preconditions.checkStateNotNull(destinations); for (DestinationState destinationState : destinations.values()) { destinationState.flush(retryManager); } @@ -437,10 +441,12 @@ void flushAll() throws Exception { numPendingRecordBytes = 0; } - private void initializeDatasetService(PipelineOptions pipelineOptions) { - if (datasetService == null) { - datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + private DatasetService initializeDatasetService(PipelineOptions pipelineOptions) { + if (maybeDatasetService == null) { + maybeDatasetService = + bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); } + return maybeDatasetService; } @StartBundle @@ -479,11 +485,12 @@ public void process( PipelineOptions pipelineOptions, @Element KV element) throws Exception { - initializeDatasetService(pipelineOptions); + DatasetService initializedDatasetService = initializeDatasetService(pipelineOptions); dynamicDestinations.setSideInputAccessorFromProcessContext(c); + Preconditions.checkStateNotNull(destinations); DestinationState state = destinations.computeIfAbsent( - element.getKey(), k -> createDestinationState(c, k, datasetService)); + element.getKey(), k -> createDestinationState(c, k, initializedDatasetService)); flushIfNecessary(); state.addMessage(element.getValue()); ++numPendingRecords; @@ -493,6 +500,8 @@ public void process( @FinishBundle public void finishBundle(FinishBundleContext context) throws Exception { flushAll(); + final Map destinations = + Preconditions.checkStateNotNull(this.destinations); for (DestinationState state : destinations.values()) { if (!useDefaultStream) { context.output( @@ -503,16 +512,16 @@ public void finishBundle(FinishBundleContext context) throws Exception { state.teardown(); } destinations.clear(); - destinations = null; + this.destinations = null; } @Teardown public void teardown() { destinations = null; try { - if (datasetService != null) { - datasetService.close(); - datasetService = null; + if (maybeDatasetService != null) { + maybeDatasetService.close(); + maybeDatasetService = null; } } catch (Exception e) { throw new RuntimeException(e); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index c9a070fbd8f3..ef128918fc1f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -37,7 +37,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -70,12 +69,12 @@ import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; -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.base.Strings; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; @@ -83,6 +82,8 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; 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.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,7 +93,7 @@ "FutureReturnValueIgnored", "unused" // TODO(BEAM-13271): Remove when new version of errorprone is released (2.11.0) }) -public class StorageApiWritesShardedRecords +public class StorageApiWritesShardedRecords extends PTransform< PCollection, Iterable>>, PCollection> { @@ -112,7 +113,7 @@ public class StorageApiWritesShardedRecords .expireAfterAccess(5, TimeUnit.MINUTES) .removalListener( (RemovalNotification removal) -> { - @Nullable final StreamAppendClient streamAppendClient = removal.getValue(); + final @Nullable StreamAppendClient streamAppendClient = removal.getValue(); // Close the writer in a different thread so as not to block the main one. runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close); }) @@ -244,7 +245,6 @@ public void startBundle() throws IOException { // Get the current stream for this key. If there is no current stream, create one and store the // stream name in // persistent state. - @SuppressWarnings({"nullness"}) String getOrCreateStream( String tableId, ValueState streamName, @@ -286,7 +286,6 @@ public void onTeardown() { } } - @SuppressWarnings({"nullness"}) @ProcessElement public void process( ProcessContext c, @@ -346,7 +345,7 @@ public void process( class AppendRowsContext extends RetryManager.Operation.Context { final ShardedKey key; String streamName = ""; - StreamAppendClient client = null; + @Nullable StreamAppendClient client = null; long offset = -1; long numRows = 0; long tryIteration = 0; @@ -440,7 +439,7 @@ public String toString() { failedContexts -> { // The first context is always the one that fails. AppendRowsContext failedContext = - Preconditions.checkNotNull(Iterables.getFirst(failedContexts, null)); + Preconditions.checkStateNotNull(Iterables.getFirst(failedContexts, null)); // Invalidate the StreamWriter and force a new one to be created. LOG.error( "Got error " + failedContext.getError() + " closing " + failedContext.streamName); @@ -449,7 +448,8 @@ public String toString() { boolean explicitStreamFinalized = failedContext.getError() instanceof StreamFinalizedException; - Status.Code statusCode = Status.fromThrowable(failedContext.getError()).getCode(); + Throwable error = Preconditions.checkStateNotNull(failedContext.getError()); + Status.Code statusCode = Status.fromThrowable(error).getCode(); // This means that the offset we have stored does not match the current end of // the stream in the Storage API. Usually this happens because a crash or a bundle // failure diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 79fbdac1c95a..c9cefdc5e0ca 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -24,14 +24,12 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.checkerframework.checker.nullness.qual.Nullable; /** * PTransform that performs streaming BigQuery write. To increase consistency, it leverages * BigQuery's best effort de-dup mechanism. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class StreamingInserts extends PTransform>, WriteResult> { private BigQueryServices bigQueryServices; @@ -43,11 +41,11 @@ public class StreamingInserts private final boolean ignoreUnknownValues; private final boolean ignoreInsertIds; private final boolean autoSharding; - private final String kmsKey; + private final @Nullable String kmsKey; private final Coder elementCoder; private final SerializableFunction toTableRow; private final SerializableFunction toFailsafeTableRow; - private final SerializableFunction deterministicRecordIdFn; + private final @Nullable SerializableFunction deterministicRecordIdFn; /** Constructor. */ public StreamingInserts( @@ -87,8 +85,8 @@ private StreamingInserts( Coder elementCoder, SerializableFunction toTableRow, SerializableFunction toFailsafeTableRow, - SerializableFunction deterministicRecordIdFn, - String kmsKey) { + @Nullable SerializableFunction deterministicRecordIdFn, + @Nullable String kmsKey) { this.createDisposition = createDisposition; this.dynamicDestinations = dynamicDestinations; this.bigQueryServices = bigQueryServices; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java index 13fee6b0c7f5..22dd04105132 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java @@ -32,11 +32,14 @@ import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.ShardedKey; import org.apache.beam.sdk.values.TupleTag; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.RequiresNonNull; /** * This transform takes in key-value pairs of {@link TableRow} entries and the {@link @@ -46,9 +49,6 @@ *

This transform assumes that all destination tables already exist by the time it sees a write * for that table. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class StreamingWriteTables extends PTransform>, WriteResult> { private BigQueryServices bigQueryServices; @@ -59,10 +59,10 @@ public class StreamingWriteTables private final boolean ignoreUnknownValues; private final boolean ignoreInsertIds; private final boolean autoSharding; - private final Coder elementCoder; - private final SerializableFunction toTableRow; - private final SerializableFunction toFailsafeTableRow; - private final SerializableFunction deterministicRecordIdFn; + private final @Nullable Coder elementCoder; + private final @Nullable SerializableFunction toTableRow; + private final @Nullable SerializableFunction toFailsafeTableRow; + private final @Nullable SerializableFunction deterministicRecordIdFn; public StreamingWriteTables() { this( @@ -87,10 +87,10 @@ private StreamingWriteTables( boolean ignoreUnknownValues, boolean ignoreInsertIds, boolean autoSharding, - Coder elementCoder, - SerializableFunction toTableRow, - SerializableFunction toFailsafeTableRow, - SerializableFunction deterministicRecordIdFn) { + @Nullable Coder elementCoder, + @Nullable SerializableFunction toTableRow, + @Nullable SerializableFunction toFailsafeTableRow, + @Nullable SerializableFunction deterministicRecordIdFn) { this.bigQueryServices = bigQueryServices; this.retryPolicy = retryPolicy; this.extendedErrorInfo = extendedErrorInfo; @@ -257,7 +257,7 @@ StreamingWriteTables withToFailsafeTableRow( } StreamingWriteTables withDeterministicRecordIdFn( - SerializableFunction deterministicRecordIdFn) { + @Nullable SerializableFunction deterministicRecordIdFn) { return new StreamingWriteTables<>( bigQueryServices, retryPolicy, @@ -274,6 +274,7 @@ StreamingWriteTables withDeterministicRecordIdFn( @Override public WriteResult expand(PCollection> input) { + Preconditions.checkStateNotNull(elementCoder); if (extendedErrorInfo) { TupleTag failedInsertsTag = new TupleTag<>(FAILED_INSERTS_TAG_ID); PCollectionTuple result = @@ -309,6 +310,7 @@ public WriteResult expand(PCollection> input) { } } + @RequiresNonNull({"elementCoder"}) private PCollectionTuple writeAndGetErrors( PCollection> input, TupleTag failedInsertsTag, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index ddd0b780c6ea..ef4e94b6b7e8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -25,9 +25,6 @@ import org.checkerframework.checker.nullness.qual.Nullable; /** Encapsulates a BigQuery table destination. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class TableDestination implements Serializable { private static final long serialVersionUID = 1L; private final String tableSpec; @@ -136,11 +133,11 @@ public TableReference getTableReference() { return BigQueryHelpers.parseTableSpec(tableSpec); } - public String getJsonTimePartitioning() { + public @Nullable String getJsonTimePartitioning() { return jsonTimePartitioning; } - public TimePartitioning getTimePartitioning() { + public @Nullable TimePartitioning getTimePartitioning() { if (jsonTimePartitioning == null) { return null; } else { @@ -148,11 +145,11 @@ public TimePartitioning getTimePartitioning() { } } - public String getJsonClustering() { + public @Nullable String getJsonClustering() { return jsonClustering; } - public Clustering getClustering() { + public @Nullable Clustering getClustering() { if (jsonClustering == null) { return null; } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java index 2cf4a1b9e8e9..c477a69c512a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -25,15 +25,14 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.checkerframework.checker.nullness.qual.Nullable; /** A coder for {@link TableDestination} objects. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class TableDestinationCoder extends AtomicCoder { private static final TableDestinationCoder INSTANCE = new TableDestinationCoder(); private static final Coder tableSpecCoder = StringUtf8Coder.of(); - private static final Coder tableDescriptionCoder = NullableCoder.of(StringUtf8Coder.of()); + private static final Coder<@Nullable String> tableDescriptionCoder = + NullableCoder.of(StringUtf8Coder.of()); private TableDestinationCoder() {} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java index bc6695aa3bb5..46ae3f47239e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link Coder} for {@link TableDestination} that includes time partitioning information. It is @@ -31,12 +32,10 @@ * old {@link TableDestinationCoder}) for compatibility reasons. The old coder is kept around for * the same compatibility reasons. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class TableDestinationCoderV2 extends AtomicCoder { private static final TableDestinationCoderV2 INSTANCE = new TableDestinationCoderV2(); - private static final Coder timePartitioningCoder = NullableCoder.of(StringUtf8Coder.of()); + private static final Coder<@Nullable String> timePartitioningCoder = + NullableCoder.of(StringUtf8Coder.of()); public static TableDestinationCoderV2 of() { return INSTANCE; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java index 43fa83675e8c..8b4bd8d34937 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link Coder} for {@link TableDestination} that includes time partitioning and clustering @@ -31,13 +32,12 @@ * options on {@link BigQueryIO.Write}, otherwise {@link TableDestinationCoderV2} will be used and * clustering information will be discarded. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class TableDestinationCoderV3 extends AtomicCoder { private static final TableDestinationCoderV3 INSTANCE = new TableDestinationCoderV3(); - private static final Coder timePartitioningCoder = NullableCoder.of(StringUtf8Coder.of()); - private static final Coder clusteringCoder = NullableCoder.of(StringUtf8Coder.of()); + private static final Coder<@Nullable String> timePartitioningCoder = + NullableCoder.of(StringUtf8Coder.of()); + private static final Coder<@Nullable String> clusteringCoder = + NullableCoder.of(StringUtf8Coder.of()); public static TableDestinationCoderV3 of() { return INSTANCE; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java index 8700cc5dd75b..7c1aefd4b858 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java @@ -51,7 +51,7 @@ import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import javax.annotation.Nullable; +import org.apache.beam.sdk.util.Preconditions; 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.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -59,6 +59,7 @@ 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.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Days; /** @@ -350,9 +351,7 @@ private static void fieldDescriptorFromTableField( descriptorBuilder.addField(fieldDescriptorBuilder.build()); } - @Nullable - @SuppressWarnings({"nullness"}) - private static Object messageValueFromFieldValue( + private static @Nullable Object messageValueFromFieldValue( SchemaInformation schemaInformation, FieldDescriptor fieldDescriptor, @Nullable Object bqValue, @@ -370,7 +369,7 @@ private static Object messageValueFromFieldValue( } if (fieldDescriptor.isRepeated()) { List listValue = (List) bqValue; - List protoList = Lists.newArrayListWithCapacity(listValue.size()); + List<@Nullable Object> protoList = Lists.newArrayListWithCapacity(listValue.size()); for (@Nullable Object o : listValue) { if (o != null) { // repeated field cannot contain null. protoList.add( @@ -385,11 +384,10 @@ private static Object messageValueFromFieldValue( } @VisibleForTesting - @Nullable - static Object singularFieldToProtoValue( + static @Nullable Object singularFieldToProtoValue( SchemaInformation schemaInformation, FieldDescriptor fieldDescriptor, - Object value, + @Nullable Object value, boolean ignoreUnknownValues) throws SchemaConversionException { switch (schemaInformation.getType()) { @@ -510,7 +508,7 @@ static Object singularFieldToProtoValue( case "STRING": case "JSON": case "GEOGRAPHY": - return value.toString(); + return Preconditions.checkArgumentNotNull(value).toString(); case "STRUCT": case "RECORD": if (value instanceof TableRow) { @@ -530,7 +528,7 @@ static Object singularFieldToProtoValue( "Unexpected value :" + value + ", type: " - + value.getClass() + + (value == null ? "null" : value.getClass()) + ". Table field name: " + schemaInformation.getFullName() + ", type: " diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java index 07576d93fe8b..7ba14d24aa54 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java @@ -32,12 +32,9 @@ * sequential number. */ @VisibleForTesting -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class TagWithUniqueIds extends DoFn, KV>> { - private transient String randomUUID; + private transient @Nullable String randomUUID = null; private transient long sequenceNo = 0L; private final @Nullable SerializableFunction elementToId; @@ -46,7 +43,7 @@ public TagWithUniqueIds() { elementToId = null; } - public TagWithUniqueIds(SerializableFunction elementToId) { + public TagWithUniqueIds(@Nullable SerializableFunction elementToId) { this.elementToId = elementToId; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java index a89e52cc8363..291ddb40cb59 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java @@ -47,8 +47,12 @@ import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.RequiresNonNull; +import org.checkerframework.dataflow.qual.SideEffectFree; import org.hamcrest.Matcher; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -65,17 +69,14 @@ * *

Deletes the table on test shutdown. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public class TestBigQuery implements TestRule { private static final DateTimeFormatter DATETIME_FORMAT = DateTimeFormat.forPattern("YYYY_MM_dd_HH_mm_ss_SSS"); private TestBigQueryOptions pipelineOptions; private Schema schema; - private Table table; - private BigQueryServices.DatasetService datasetService; + private @Nullable Table table = null; + private BigQueryServices.@Nullable DatasetService datasetService = null; /** * Creates an instance of this rule. @@ -125,7 +126,9 @@ private void initializeBigQuery(Description description) this.table = createTable(description); } + @RequiresNonNull("datasetService") private Table createTable(Description description) throws IOException, InterruptedException { + BigQueryServices.DatasetService datasetService = this.datasetService; TableReference tableReference = new TableReference() .setProjectId( @@ -135,7 +138,7 @@ private Table createTable(Description description) throws IOException, Interrupt .setDatasetId(pipelineOptions.getTargetDataset()) .setTableId(createRandomizedName(description)); - table = + Table newTable = new Table() .setTableReference(tableReference) .setSchema(BigQueryUtils.toTableSchema(schema)) @@ -153,7 +156,8 @@ private Table createTable(Description description) throws IOException, Interrupt + "It should have been cleaned up by the test rule."); } - datasetService.createTable(table); + datasetService.createTable(newTable); + table = newTable; return table; } @@ -195,7 +199,9 @@ static String createRandomizedName(Description description) throws IOException { return topicName.toString() + "_" + String.valueOf(randomNumber); } + @RequiresNonNull("table") public String tableSpec() { + Table table = this.table; return String.format( "%s:%s.%s", table.getTableReference().getProjectId(), @@ -203,12 +209,15 @@ public String tableSpec() { table.getTableReference().getTableId()); } + @RequiresNonNull("table") public TableReference tableReference() { return table.getTableReference(); } @Experimental(Kind.SCHEMAS) + @RequiresNonNull("table") public TableDataInsertAllResponse insertRows(Schema rowSchema, Row... rows) throws IOException { + Table table = this.table; List bqRows = Arrays.stream(rows) .map(row -> new Rows().setJson(BigQueryUtils.toTableRow(row))) @@ -236,6 +245,7 @@ public TableDataInsertAllResponse insertRows(Schema rowSchema, Row... rows) thro @Experimental(Kind.SCHEMAS) public List getFlatJsonRows(Schema rowSchema) { Bigquery bq = newBigQueryClient(pipelineOptions); + Preconditions.checkStateNotNull(this.table); return bqRowsToBeamRows(getSchema(bq), getTableRows(bq), rowSchema); } @@ -276,7 +286,10 @@ private List bqRowsToBeamRows( .collect(Collectors.toList()); } + @RequiresNonNull("table") + @SideEffectFree private TableSchema getSchema(Bigquery bq) { + Table table = this.table; try { return bq.tables() .get( @@ -293,7 +306,10 @@ private TableSchema getSchema(Bigquery bq) { } } + @RequiresNonNull("table") + @SideEffectFree private List getTableRows(Bigquery bq) { + Table table = this.table; try { return bq.tabledata() .list( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java index b0a6a3123202..cb0ea9af58ee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java @@ -23,6 +23,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; +import org.checkerframework.checker.nullness.qual.NonNull; /** * A cache for {@link MessageConverter} objects. @@ -31,7 +32,8 @@ * name. However since this object is stored in DoFns and many DoFns share the same * MessageConverters, we also store a static cache keyed by operation name. */ -class TwoLevelMessageConverterCache implements Serializable { +class TwoLevelMessageConverterCache + implements Serializable { final String operationName; TwoLevelMessageConverterCache(String operationName) { @@ -40,7 +42,6 @@ class TwoLevelMessageConverterCache implements Serializa // Cache MessageConverters since creating them can be expensive. Cache is keyed by transform name // and the destination. - @SuppressWarnings({"nullness"}) private static final Cache, MessageConverter> CACHED_MESSAGE_CONVERTERS = CacheBuilder.newBuilder().expireAfterAccess(java.time.Duration.ofMinutes(15)).build(); @@ -49,7 +50,6 @@ class TwoLevelMessageConverterCache implements Serializa // on every element. Since there will be multiple DoFn instances (and they may periodically be // recreated), we // still need the static cache to allow reuse. - @SuppressWarnings({"nullness"}) private final Cache> localMessageConverters = CacheBuilder.newBuilder().expireAfterAccess(java.time.Duration.ofMinutes(15)).build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java index 4ae1064bc431..4130b09556ed 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java @@ -90,7 +90,7 @@ public UpdateSchemaDestination( BigQueryIO.Write.CreateDisposition createDisposition, int maxRetryJobs, boolean ignoreUnknownValues, - String kmsKey, + @Nullable String kmsKey, String sourceFormat, boolean useAvroLogicalTypes, Set schemaUpdateOptions, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index a468108a93fa..d3e9ebd4e9c5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; - import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; import java.io.InputStream; @@ -37,12 +35,14 @@ import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.ShardedKey; import org.apache.beam.sdk.values.TupleTag; 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.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -53,10 +53,7 @@ * the element will be spilled into the output, and the {@link WriteGroupedRecordsToFiles} transform * will take care of writing it to a file. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) -class WriteBundlesToFiles +class WriteBundlesToFiles extends DoFn, Result> { // When we spill records, shard the output keys to prevent hotspots. Experiments running up to @@ -64,8 +61,9 @@ class WriteBundlesToFiles private static final int SPILLED_RECORD_SHARDING_FACTOR = 10; // Map from tablespec to a writer for that table. - private transient Map> writers; - private transient Map writerWindows; + private transient @Nullable Map> writers = null; + private transient @Nullable Map writerWindows = null; + private final PCollectionView tempFilePrefixView; private final TupleTag, ElementT>> unwrittenRecordsTag; private final int maxNumWritersPerBundle; @@ -84,7 +82,7 @@ static final class Result implements Serializable { public final DestinationT destination; public Result(String filename, Long fileByteSize, DestinationT destination) { - checkNotNull(destination); + Preconditions.checkArgumentNotNull(destination); this.filename = filename; this.fileByteSize = fileByteSize; this.destination = destination; @@ -179,17 +177,21 @@ public void verifyDeterministic() {} public void startBundle() { // This must be done for each bundle, as by default the {@link DoFn} might be reused between // bundles. - this.writers = Maps.newHashMap(); this.writerWindows = Maps.newHashMap(); + this.writers = Maps.newHashMap(); this.spilledShardNumber = ThreadLocalRandom.current().nextInt(SPILLED_RECORD_SHARDING_FACTOR); } BigQueryRowWriter createAndInsertWriter( DestinationT destination, String tempFilePrefix, BoundedWindow window) throws Exception { + Map writerWindows = + Preconditions.checkStateNotNull(this.writerWindows); + Map> writers = + Preconditions.checkStateNotNull(this.writers); BigQueryRowWriter writer = rowWriterFactory.createRowWriter(tempFilePrefix, destination); - writers.put(destination, writer); writerWindows.put(destination, window); + writers.put(destination, writer); return writer; } @@ -197,6 +199,8 @@ BigQueryRowWriter createAndInsertWriter( public void processElement( ProcessContext c, @Element KV element, BoundedWindow window) throws Exception { + Map> writers = + Preconditions.checkStateNotNull(this.writers); String tempFilePrefix = c.sideInput(tempFilePrefixView); DestinationT destination = c.element().getKey(); @@ -245,6 +249,11 @@ public void processElement( @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { + Map> writers = + Preconditions.checkStateNotNull(this.writers); + Map writerWindows = + Preconditions.checkStateNotNull(this.writerWindows); + List exceptionList = Lists.newArrayList(); for (BigQueryRowWriter writer : writers.values()) { try { @@ -266,10 +275,12 @@ public void finishBundle(FinishBundleContext c) throws Exception { DestinationT destination = entry.getKey(); BigQueryRowWriter writer = entry.getValue(); BigQueryRowWriter.Result result = writer.getResult(); + BoundedWindow window = writerWindows.get(destination); + Preconditions.checkStateNotNull(window); c.output( new Result<>(result.resourceId.toString(), result.byteSize, destination), - writerWindows.get(destination).maxTimestamp(), - writerWindows.get(destination)); + window.maxTimestamp(), + window); } catch (Exception e) { exceptionList.add(e); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java index e1e0566c1704..03cce2631e64 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java @@ -36,15 +36,11 @@ import org.apache.beam.sdk.values.TupleTag; 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.checkerframework.checker.nullness.qual.Nullable; /** * Partitions temporary files based on number of files and file sizes. Output key is a pair of * tablespec and the list of files corresponding to each partition of that table. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class WritePartition extends DoFn< Iterable>, @@ -81,8 +77,7 @@ public Result decode(InputStream inStream) throws IOException { private final long maxSizeBytes; private final RowWriterFactory rowWriterFactory; - private @Nullable TupleTag, WritePartition.Result>> - multiPartitionsTag; + private TupleTag, WritePartition.Result>> multiPartitionsTag; private TupleTag, WritePartition.Result>> singlePartitionTag; private static class PartitionData { @@ -225,8 +220,13 @@ public void processElement(ProcessContext c) throws Exception { // In the fast-path case where we only output one table, the transform loads it directly // to the final table. In this case, we output on a special TupleTag so the enclosing // transform knows to skip the rename step. - TupleTag, WritePartition.Result>> outputTag = - (destinationData.getPartitions().size() == 1) ? singlePartitionTag : multiPartitionsTag; + TupleTag, WritePartition.Result>> outputTag; + if (destinationData.getPartitions().size() == 1) { + outputTag = singlePartitionTag; + } else { + outputTag = multiPartitionsTag; + } + for (int i = 0; i < destinationData.getPartitions().size(); ++i) { PartitionData partitionData = destinationData.getPartitions().get(i); c.output( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index 354f8eff95d6..0dc7e6fbcb7f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.PendingJobManager; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; @@ -44,6 +43,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; 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.Multimap; +import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,9 +51,6 @@ * Copies temporary tables to destination table. The input element is an {@link Iterable} that * provides the list of all temporary tables created for a given {@link TableDestination}. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) class WriteRename extends DoFn>, TableDestination> { private static final Logger LOG = LoggerFactory.getLogger(WriteRename.class); @@ -67,8 +64,8 @@ class WriteRename private final WriteDisposition firstPaneWriteDisposition; private final CreateDisposition firstPaneCreateDisposition; private final int maxRetryJobs; - private final String kmsKey; - private final ValueProvider loadJobProjectId; + private final @Nullable String kmsKey; + private final @Nullable ValueProvider loadJobProjectId; private transient @Nullable DatasetService datasetService; private static class PendingJobData { @@ -97,8 +94,8 @@ public WriteRename( WriteDisposition writeDisposition, CreateDisposition createDisposition, int maxRetryJobs, - String kmsKey, - ValueProvider loadJobProjectId) { + @Nullable String kmsKey, + @Nullable ValueProvider loadJobProjectId) { this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.firstPaneWriteDisposition = writeDisposition; @@ -192,8 +189,9 @@ private PendingJobData startWriteRename( // trigger to handle the case where an earlier pane triggered the single-partition path. If this // happened, then the // table will already exist so we want to append to the table. + WriteTables.@Nullable Result firstTempTable = Iterables.getFirst(tempTableNames, null); boolean isFirstPane = - Iterables.getFirst(tempTableNames, null).isFirstPane() && c.pane().isFirst(); + firstTempTable != null && firstTempTable.isFirstPane() && c.pane().isFirst(); WriteDisposition writeDisposition = isFirstPane ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; CreateDisposition createDisposition = @@ -232,8 +230,8 @@ private BigQueryHelpers.PendingJob startCopy( List tempTables, WriteDisposition writeDisposition, CreateDisposition createDisposition, - String kmsKey, - ValueProvider loadJobProjectId) { + @Nullable String kmsKey, + @Nullable ValueProvider loadJobProjectId) { JobConfigurationTableCopy copyConfig = new JobConfigurationTableCopy() .setSourceTables(tempTables) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java index a742df150bf1..820deb8bbfb8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; - import com.google.api.services.bigquery.model.TableRow; import java.util.Map; import org.apache.beam.sdk.Pipeline; @@ -136,14 +134,14 @@ private WriteResult( * loaded using the batch load API. */ public PCollection getSuccessfulTableLoads() { - checkArgument( - successfulBatchInsertsTag != null, + Preconditions.checkArgumentNotNull( + successfulBatchInsertsTag, + "Cannot use getSuccessfulTableLoads because this WriteResult was not " + + "configured to produce them. Note: only batch loads produce successfulTableLoads."); + return Preconditions.checkArgumentNotNull( + successfulBatchInserts, "Cannot use getSuccessfulTableLoads because this WriteResult was not " + "configured to produce them. Note: only batch loads produce successfulTableLoads."); - if (successfulBatchInserts == null) { - throw new IllegalStateException("Unexpected null successfulBatchInserts collection."); - } - return successfulBatchInserts; } /** @@ -166,14 +164,14 @@ public PCollection getSuccessfulInserts() { * WriteResult#getFailedInsertsWithErr()} */ public PCollection getFailedInserts() { - checkArgument( - failedInsertsTag != null, + Preconditions.checkArgumentNotNull( + failedInsertsTag, + "Cannot use getFailedInserts as this WriteResult uses extended errors" + + " information. Use getFailedInsertsWithErr instead"); + return Preconditions.checkStateNotNull( + failedInserts, "Cannot use getFailedInserts as this WriteResult uses extended errors" + " information. Use getFailedInsertsWithErr instead"); - if (failedInserts == null) { - throw new IllegalStateException("Unexpected null failedInserts collection."); - } - return failedInserts; } /** @@ -184,24 +182,23 @@ public PCollection getFailedInserts() { * Otherwise use {@link WriteResult#getFailedInserts()} */ public PCollection getFailedInsertsWithErr() { - checkArgument( - failedInsertsWithErrTag != null, + Preconditions.checkArgumentNotNull( + failedInsertsWithErrTag, + "Cannot use getFailedInsertsWithErr as this WriteResult does not use" + + " extended errors. Use getFailedInserts instead"); + return Preconditions.checkArgumentNotNull( + failedInsertsWithErr, "Cannot use getFailedInsertsWithErr as this WriteResult does not use" + " extended errors. Use getFailedInserts instead"); - if (failedInsertsWithErr == null) { - throw new IllegalStateException("Unexpected null failedInsertsWithErr collection."); - } - return failedInsertsWithErr; } public PCollection getFailedStorageApiInserts() { - checkArgument( - failedStorageApiInsertsTag != null, + Preconditions.checkStateNotNull( + failedStorageApiInsertsTag, + "Cannot use getFailedStorageApiInserts as this insert didn't use the storage API."); + return Preconditions.checkStateNotNull( + failedStorageApiInserts, "Cannot use getFailedStorageApiInserts as this insert didn't use the storage API."); - if (failedStorageApiInserts == null) { - throw new IllegalStateException("Unexpected null failedStorageApiInserts collection."); - } - return failedStorageApiInserts; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index f30388b523cc..ce725c989b77 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -63,6 +63,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -93,10 +94,7 @@ * the data directly into the final table, skipping temporary tables. In this case, the output * {@link KV} maps the final table to itself. */ -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) -class WriteTables +class WriteTables extends PTransform< PCollection, WritePartition.Result>>, PCollection>> { @@ -139,7 +137,7 @@ public Result decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) private final List> sideInputs; private final TupleTag> mainOutputTag; private final TupleTag temporaryFilesTag; - private final ValueProvider loadJobProjectId; + private final @Nullable ValueProvider loadJobProjectId; private final int maxRetryJobs; private final boolean ignoreUnknownValues; private final @Nullable String kmsKey; @@ -206,8 +204,8 @@ public void processElement( BigQueryHelpers.fromJsonString(jsonSchemas.get(destination), TableSchema.class); } else { tableSchema = dynamicDestinations.getSchema(destination); - checkArgument( - tableSchema != null, + Preconditions.checkArgumentNotNull( + tableSchema, "Unless create disposition is %s, a schema must be specified, i.e. " + "DynamicDestinations.getSchema() may not return null. " + "However, create disposition is %s, and %s returned null for destination %s", @@ -399,7 +397,7 @@ public WriteTables( @Nullable ValueProvider loadJobProjectId, int maxRetryJobs, boolean ignoreUnknownValues, - String kmsKey, + @Nullable String kmsKey, String sourceFormat, boolean useAvroLogicalTypes, Set schemaUpdateOptions, @@ -460,13 +458,16 @@ private PendingJob startLoad( DatasetService datasetService, String jobIdPrefix, TableReference ref, - TimePartitioning timePartitioning, - Clustering clustering, + @Nullable TimePartitioning timePartitioning, + @Nullable Clustering clustering, @Nullable TableSchema schema, List gcsUris, WriteDisposition writeDisposition, CreateDisposition createDisposition, Set schemaUpdateOptions) { + @SuppressWarnings({ + "nullness" // nulls allowed in most fields but API client not annotated + }) JobConfigurationLoad loadConfig = new JobConfigurationLoad() .setDestinationTable(ref) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index 1d171705fcd4..6007d6a11bd5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -82,6 +82,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -471,7 +472,9 @@ public void testGetTableSucceeds() throws Exception { new BigQueryServicesImpl.DatasetServiceImpl( bigquery, null, PipelineOptionsFactory.create()); - Table table = datasetService.getTable(tableRef, null, BackOff.ZERO_BACKOFF, Sleeper.DEFAULT); + Table table = + datasetService.getTable( + tableRef, Collections.emptyList(), BackOff.ZERO_BACKOFF, Sleeper.DEFAULT); assertEquals(testTable, table); verifyAllResponsesAreRead(); @@ -494,7 +497,9 @@ public void testGetTableNotFound() throws IOException, InterruptedException { .setProjectId("projectId") .setDatasetId("datasetId") .setTableId("tableId"); - Table table = datasetService.getTable(tableRef, null, BackOff.ZERO_BACKOFF, Sleeper.DEFAULT); + Table table = + datasetService.getTable( + tableRef, Collections.emptyList(), BackOff.ZERO_BACKOFF, Sleeper.DEFAULT); assertNull(table); verifyAllResponsesAreRead(); @@ -520,7 +525,8 @@ public void testGetTableThrows() throws Exception { BigQueryServicesImpl.DatasetServiceImpl datasetService = new BigQueryServicesImpl.DatasetServiceImpl( bigquery, null, PipelineOptionsFactory.create()); - datasetService.getTable(tableRef, null, BackOff.STOP_BACKOFF, Sleeper.DEFAULT); + datasetService.getTable( + tableRef, Collections.emptyList(), BackOff.STOP_BACKOFF, Sleeper.DEFAULT); } @Test