From 3ec1d0845cb36c5f523725f2f13234870805a565 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 5 Dec 2022 18:20:50 +0000 Subject: [PATCH 01/32] ReadAPI Source v2 --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 205 ++++++++++- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 6 + .../bigquery/BigQueryStorageSourceBase.java | 62 +++- .../bigquery/BigQueryStorageSourceBaseV2.java | 5 + .../BigQueryStorageStreamSourceV2.java | 346 ++++++++++++++++++ .../bigquery/BigQueryIOStorageReadTest.java | 1 + 6 files changed, 608 insertions(+), 17 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java 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 fedc898fe973..37a471aa38c4 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 @@ -75,7 +75,9 @@ import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.AvroSource; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.fs.MoveOptions; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; @@ -1171,11 +1173,11 @@ public PCollection expand(PBegin input) { // if both toRowFn and fromRowFn values are set, enable Beam schema support Pipeline p = input.getPipeline(); + BigQueryOptions bqOptions = p.getOptions().as(BigQueryOptions.class); final BigQuerySourceDef sourceDef = createSourceDef(); Schema beamSchema = null; if (getTypeDescriptor() != null && getToBeamRowFn() != null && getFromBeamRowFn() != null) { - BigQueryOptions bqOptions = p.getOptions().as(BigQueryOptions.class); beamSchema = sourceDef.getBeamSchema(bqOptions); beamSchema = getFinalSchema(beamSchema, getSelectedFields()); } @@ -1183,7 +1185,7 @@ public PCollection expand(PBegin input) { final Coder coder = inferCoder(p.getCoderRegistry()); if (getMethod() == TypedRead.Method.DIRECT_READ) { - return expandForDirectRead(input, coder, beamSchema); + return expandForDirectRead(input, coder, beamSchema, bqOptions); } checkArgument( @@ -1361,7 +1363,7 @@ private static Schema getFinalSchema( } private PCollection expandForDirectRead( - PBegin input, Coder outputCoder, Schema beamSchema) { + PBegin input, Coder outputCoder, Schema beamSchema, BigQueryOptions bqOptions) { ValueProvider tableProvider = getTableProvider(); Pipeline p = input.getPipeline(); if (tableProvider != null) { @@ -1407,6 +1409,14 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // + // if (bqOptions.getUseReadAPISourceV2()) { + // return expandAnonForDirectReadV2(input, outputCoder, beamSchema); + // } + + if(bqOptions.getUseReadAPISourceV2() || !bqOptions.getUseReadAPISourceV2()) { + + } + PCollectionView jobIdTokenView; PCollection rows; @@ -1585,6 +1595,195 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } + // private PCollection expandAnonForDirectReadV2( + // PBegin input, Coder outputCoder, Schema beamSchema) { + // Pipeline p = input.getPipeline(); + // PCollectionView jobIdTokenView; + // PCollection rows; + // + // if (!getWithTemplateCompatibility()) { + // // Create a singleton job ID token at pipeline construction time. + // String staticJobUuid = BigQueryHelpers.randomUUIDString(); + // jobIdTokenView = + // p.apply("TriggerIdCreation", Create.of(staticJobUuid)) + // .apply("ViewId", View.asSingleton()); + // // Apply the traditional Source model. + // rows = + // p.apply( + // org.apache.beam.sdk.io.Read.from( + // createStorageQuerySource(staticJobUuid, outputCoder))); + // } else { + // // Create a singleton job ID token at pipeline execution time. + // PCollection jobIdTokenCollection = + // p.apply("TriggerIdCreation", Create.of("ignored")) + // .apply( + // "CreateJobId", + // MapElements.via( + // new SimpleFunction() { + // @Override + // public String apply(String input) { + // return BigQueryHelpers.randomUUIDString(); + // } + // })); + // + // jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); + // + // TupleTag> listReadStreamsTag = new TupleTag<>(); + // TupleTag readSessionTag = new TupleTag<>(); + // TupleTag tableSchemaTag = new TupleTag<>(); + // + // PCollectionTuple tuple = + // jobIdTokenCollection.apply( + // "RunQueryJob", + // ParDo.of( + // new DoFn>() { + // @ProcessElement + // public void processElement(ProcessContext c) throws Exception { + // BigQueryOptions options = + // c.getPipelineOptions().as(BigQueryOptions.class); + // String jobUuid = c.element(); + // // Execute the query and get the destination table holding the results. + // // The getTargetTable call runs a new instance of the query and returns + // // the destination table created to hold the results. + // BigQueryStorageQuerySource querySource = + // createStorageQuerySource(jobUuid, outputCoder); + // Table queryResultTable = querySource.getTargetTable(options); + // + // // Create a read session without specifying a desired stream count and + // // let the BigQuery storage server pick the number of streams. + // CreateReadSessionRequest request = + // CreateReadSessionRequest.newBuilder() + // .setParent( + // BigQueryHelpers.toProjectResourceName( + // options.getBigQueryProject() == null + // ? options.getProject() + // : options.getBigQueryProject())) + // .setReadSession( + // ReadSession.newBuilder() + // .setTable( + // BigQueryHelpers.toTableResourceName( + // queryResultTable.getTableReference())) + // .setDataFormat(DataFormat.AVRO)) + // .setMaxStreamCount(0) + // .build(); + // + // ReadSession readSession; + // try (StorageClient storageClient = + // getBigQueryServices().getStorageClient(options)) { + // readSession = storageClient.createReadSession(request); + // } + // int streamIndex = 0; + // int streamsPerBundle = 10; + // List streamBundle = Lists.newArrayList(); + // for (ReadStream readStream : readSession.getStreamsList()) { + // streamIndex++; + // streamBundle.add(readStream); + // if (streamIndex % streamsPerBundle == 0) { + // c.output(streamBundle); + // streamBundle = Lists.newArrayList(); + // } + // } + // + // c.output(readSessionTag, readSession); + // c.output( + // tableSchemaTag, + // BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + // } + // }) + // .withOutputTags( + // listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + // + // TypeDescriptor> listReadStreamTypeDescriptor = new TypeDescriptor>() {}; + // tuple.get(listReadStreamsTag).setCoder(ProtoCoder.of(listReadStreamTypeDescriptor)); + // tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); + // tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); + // + // PCollectionView readSessionView = + // tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); + // PCollectionView tableSchemaView = + // tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); + // + // rows = + // tuple + // .get(listReadStreamsTag) + // .apply(Reshuffle.viaRandomKey()) + // .apply( + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) throws Exception { + // ReadSession readSession = c.sideInput(readSessionView); + // TableSchema tableSchema = + // BigQueryHelpers.fromJsonString( + // c.sideInput(tableSchemaView), TableSchema.class); + // List streamBundle = c.element(); + // + // BigQueryStorageStreamSourceV2 streamSource = + // BigQueryStorageStreamSourceV2.create( + // readSession, + // streamBundle, + // tableSchema, + // getParseFn(), + // outputCoder, + // getBigQueryServices()); + // + // // Read all of the data from the stream. In the event that this work + // // item fails and is rescheduled, the same rows will be returned in + // // the same order. + // BoundedReader reader = + // streamSource.createReader(c.getPipelineOptions()); + // for (boolean more = reader.start(); more; more = reader.advance()) { + // c.output(reader.getCurrent()); + // } + // } + // }) + // .withSideInputs(readSessionView, tableSchemaView)) + // .setCoder(outputCoder); + // } + // + // CleanupOperation cleanupOperation = + // new CleanupOperation() { + // @Override + // void cleanup(ContextContainer c) throws Exception { + // BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); + // String jobUuid = c.getJobId(); + // + // Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); + // + // TableReference tempTable = + // createTempTableReference( + // options.getBigQueryProject() == null + // ? options.getProject() + // : options.getBigQueryProject(), + // BigQueryResourceNaming.createJobIdPrefix( + // options.getJobName(), jobUuid, JobType.QUERY), + // queryTempDataset); + // + // try (DatasetService datasetService = + // getBigQueryServices().getDatasetService(options)) { + // LOG.info("Deleting temporary table with query results {}", tempTable); + // datasetService.deleteTable(tempTable); + // // Delete dataset only if it was created by Beam + // boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); + // if (datasetCreatedByBeam) { + // LOG.info( + // "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + // datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + // } + // } + // } + // }; + // + // if (beamSchema != null) { + // rows.setSchema( + // beamSchema, + // getTypeDescriptor(), + // getToBeamRowFn().apply(beamSchema), + // getFromBeamRowFn().apply(beamSchema)); + // } + // return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + // } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index f0b3e061597a..cfd0ef984794 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -150,4 +150,10 @@ public interface BigQueryOptions Long getStorageWriteApiMaxRequestSize(); void setStorageWriteApiMaxRequestSize(Long value); + + @Description("If set, BigQueryIO.Read will default to using the V2 implementation of the Read API Source") + @Default.Boolean(false) + Boolean getUseReadAPISourceV2(); + + void setUseReadAPISourceV2(Boolean value); } 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 3c8335ac82b5..8fab8f461d6a 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 @@ -100,7 +100,7 @@ public Coder getOutputCoder() { } @Override - public List> split( + public List> split( long desiredBundleSizeBytes, PipelineOptions options) throws Exception { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); @Nullable Table targetTable = getTargetTable(bqOptions); @@ -133,13 +133,18 @@ public List> split( readSessionBuilder.setDataFormat(format); } + // Setting the requested max stream count to 0, implies that the Read API backend will select + // an appropriate number of streams for the Session to produce reasonable throughput. + // This is required when using the Read API Source V2. int streamCount = 0; - if (desiredBundleSizeBytes > 0) { - long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; - streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); - } + if(!bqOptions.getUseReadAPISourceV2()) { + if (desiredBundleSizeBytes > 0) { + long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; + streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); + } - streamCount = Math.max(streamCount, MIN_SPLIT_COUNT); + streamCount = Math.max(streamCount, MIN_SPLIT_COUNT); + } CreateReadSessionRequest createReadSessionRequest = CreateReadSessionRequest.newBuilder() @@ -166,6 +171,20 @@ public List> split( return ImmutableList.of(); } + int streamsPerBundle = 0; + if (bqOptions.getUseReadAPISourceV2()) { + if (desiredBundleSizeBytes > 0) { + streamsPerBundle = + (int) + Math.ceil((double) + desiredBundleSizeBytes + / (double) (readSession.getEstimatedTotalBytesScanned() + / readSession.getStreamsList().size())); + } else { + streamsPerBundle = (int) Math.ceil((double) streamCount / 10); + } + } + Schema sessionSchema; if (readSession.getDataFormat() == DataFormat.ARROW) { org.apache.arrow.vector.types.pojo.Schema schema = @@ -180,19 +199,34 @@ public List> split( throw new IllegalArgumentException( "data is not in a supported dataFormat: " + readSession.getDataFormat()); } - + int streamIndex = 0; 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(); - for (ReadStream readStream : readSession.getStreamsList()) { - sources.add( - BigQueryStorageStreamSource.create( - readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); + if(!bqOptions.getUseReadAPISourceV2()) { + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + sources.add( + BigQueryStorageStreamSource.create( + readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); + } + return ImmutableList.copyOf(sources); + } else { + List streamBundle = Lists.newArrayList(); + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + sources.add( + BigQueryStorageStreamSourceV2.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); + streamBundle = Lists.newArrayList(); + } + } + return ImmutableList.copyOf(sources); } - - return ImmutableList.copyOf(sources); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java new file mode 100644 index 000000000000..a0ba4e22d1e3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java @@ -0,0 +1,5 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +public class BigQueryStorageSourceBaseV2 { + +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java new file mode 100644 index 000000000000..b4028996a0ef --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +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.checkArgument; + +import com.google.api.gax.rpc.ApiException; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.beam.runners.core.metrics.ServiceCallMetric; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; +import org.apache.beam.sdk.metrics.Metrics; +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.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; + +class BigQueryStorageStreamSourceV2 extends BoundedSource { + + private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSourceV2.class); + + public static BigQueryStorageStreamSourceV2 create( + ReadSession readSession, + List streamBundle, + TableSchema tableSchema, + SerializableFunction parseFn, + Coder outputCoder, + BigQueryServices bqServices) { + return new BigQueryStorageStreamSourceV2<>( + readSession, + streamBundle, + toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), + parseFn, + outputCoder, + bqServices); + } + + /** + * Creates a new source with the same properties as this one, except with a different {@link + * List}. + */ + public BigQueryStorageStreamSourceV2 fromExisting(List newStreamBundle) { + return new BigQueryStorageStreamSourceV2<>( + readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); + } + + private final ReadSession readSession; + private final List streamBundle; + private final String jsonTableSchema; + private final SerializableFunction parseFn; + private final Coder outputCoder; + private final BigQueryServices bqServices; + + private BigQueryStorageStreamSourceV2( + ReadSession readSession, + List streamBundle, + String jsonTableSchema, + SerializableFunction parseFn, + Coder outputCoder, + BigQueryServices bqServices) { + this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); + this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); + 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 + public Coder getOutputCoder() { + return outputCoder; + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .add(DisplayData.item("table", readSession.getTable()).withLabel("Table")) + .add(DisplayData.item("readSession", readSession.getName()).withLabel("Read session")); + for (ReadStream readStream : streamBundle) { + builder.add(DisplayData.item("stream", readStream.getName()).withLabel("Stream")); + } + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) { + // The size of stream source can't be estimated due to server-side liquid sharding. + // TODO: Implement progress reporting. + return 0L; + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) { + // A stream source can't be split without reading from it due to server-side liquid sharding. + // TODO: Implement dynamic work rebalancing. + return ImmutableList.of(this); + } + + @Override + public BigQueryStorageStreamBundleReader createReader(PipelineOptions options) + throws IOException { + return new BigQueryStorageStreamBundleReader<>(this, options.as(BigQueryOptions.class)); + } + + // @Override + // public String toString() { + // return readStream.toString(); + // } + + public static class BigQueryStorageStreamBundleReader extends BoundedSource.BoundedReader { + private final BigQueryStorageReader reader; + private final SerializableFunction parseFn; + private final StorageClient storageClient; + private final TableSchema tableSchema; + + private BigQueryStorageStreamSourceV2 source; + private @Nullable BigQueryServerStream responseStream = null; + private @Nullable Iterator responseIterator = null; + private @Nullable T current = null; + private int currentStreamIndex; + private long currentOffset; + + // Values used for progress reporting. + private double fractionConsumed; + private double progressAtResponseStart; + private double progressAtResponseEnd; + private long rowsConsumedFromCurrentResponse; + private long totalRowsInCurrentResponse; + + private @Nullable TableReference tableReference; + private @Nullable ServiceCallMetric serviceCallMetric; + + private BigQueryStorageStreamBundleReader( + BigQueryStorageStreamSourceV2 source, BigQueryOptions options) throws IOException { + this.source = source; + this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); + this.parseFn = source.parseFn; + this.storageClient = source.bqServices.getStorageClient(options); + this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); + this.currentStreamIndex = 0; + this.fractionConsumed = 0d; + this.progressAtResponseStart = 0d; + this.progressAtResponseEnd = 0d; + this.rowsConsumedFromCurrentResponse = 0L; + this.totalRowsInCurrentResponse = 0L; + } + + @Override + public T getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current; + } + + @Override + public synchronized boolean start() throws IOException { + return readNextStream(); + } + + @Override + public synchronized boolean advance() throws IOException { + Preconditions.checkStateNotNull(responseIterator); + currentOffset++; + return readNextRecord(); + } + + private synchronized boolean readNextStream() throws IOException { + BigQueryStorageStreamSourceV2 source = getCurrentSource(); + if (currentStreamIndex == source.streamBundle.size() - 1) { + fractionConsumed = 1d; + return false; + } + ReadRowsRequest request = + ReadRowsRequest.newBuilder() + .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) + .setOffset(currentOffset) + .build(); + + tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); + serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); + LOG.info( + "Started BigQuery Storage API read from stream {}.", + source.streamBundle.get(0).getName()); + responseStream = storageClient.readRows(request, source.readSession.getTable()); + responseIterator = responseStream.iterator(); + return readNextRecord(); + } + + @RequiresNonNull("responseIterator") + private synchronized boolean readNextRecord() throws IOException { + Iterator responseIterator = this.responseIterator; + while (reader.readyForNextReadResponse()) { + if (!responseIterator.hasNext()) { + currentStreamIndex++; + return readNextStream(); + } + + ReadRowsResponse response; + try { + response = responseIterator.next(); + // Since we don't have a direct hook to the underlying + // API call, record success every time we read a record successfully. + if (serviceCallMetric != null) { + serviceCallMetric.call("ok"); + } + } catch (ApiException e) { + // Occasionally the iterator will fail and raise an exception. + // Capture it here and record the error in the metric. + if (serviceCallMetric != null) { + serviceCallMetric.call(e.getStatusCode().getCode().name()); + } + throw e; + } + + progressAtResponseStart = response.getStats().getProgress().getAtResponseStart(); + progressAtResponseEnd = response.getStats().getProgress().getAtResponseEnd(); + totalRowsInCurrentResponse = response.getRowCount(); + rowsConsumedFromCurrentResponse = 0L; + + checkArgument( + totalRowsInCurrentResponse >= 0, + "Row count from current response (%s) must be non-negative.", + totalRowsInCurrentResponse); + + checkArgument( + 0f <= progressAtResponseStart && progressAtResponseStart <= 1f, + "Progress at response start (%s) is not in the range [0.0, 1.0].", + progressAtResponseStart); + + checkArgument( + 0f <= progressAtResponseEnd && progressAtResponseEnd <= 1f, + "Progress at response end (%s) is not in the range [0.0, 1.0].", + progressAtResponseEnd); + + reader.processReadRowsResponse(response); + } + + SchemaAndRecord schemaAndRecord = new SchemaAndRecord(reader.readSingleRecord(), tableSchema); + + current = parseFn.apply(schemaAndRecord); + + // Updates the fraction consumed value. This value is calculated by interpolating between + // the fraction consumed value from the previous server response (or zero if we're consuming + // the first response) and the fractional value in the current response based on how many of + // the rows in the current response have been consumed. + rowsConsumedFromCurrentResponse++; + + fractionConsumed = + progressAtResponseStart + + (progressAtResponseEnd - progressAtResponseStart) + * rowsConsumedFromCurrentResponse + * 1.0 + / totalRowsInCurrentResponse; + + // Assuming that each stream in the StreamBundle has approximately the same amount of data. + fractionConsumed = fractionConsumed / source.streamBundle.size(); + + return true; + } + + @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(); + } + + @Override + public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { + return source; + } + + @Override + @SuppressWarnings("ReturnValueIgnored") + public @Nullable BoundedSource splitAtFraction(double fraction) { + int streamCountInBundle = source.streamBundle.size(); + double splitIndex = streamCountInBundle * fraction; + if (currentStreamIndex > splitIndex) { + // The reader has moved past the requested split point. + Metrics.counter( + BigQueryStorageStreamBundleReader.class, + "split-at-fraction-calls-failed-due-to-impossible-split-point") + .inc(); + LOG.info( + "BigQuery Storage API Session {} cannot be split at {}.", + source.readSession.getName(), + fraction); + return null; + } + // Splitting the remainder Streams into a new StreamBundle. + List remainderStreamBundle = + new ArrayList<>( + source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); + // Updating the primary StreamBundle. + source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); + Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") + .inc(); + LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); + return source.fromExisting(remainderStreamBundle); + } + + @Override + public synchronized Double getFractionConsumed() { + return fractionConsumed; + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index 518c4a80cdbc..821cdfcb6fb2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -161,6 +161,7 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); + options.as(BigQueryOptions.class).setUseReadAPISourceV2(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } From 0a2fc4f654efac8e6aee50a00baac2c52247068d Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 10 Jan 2023 19:16:54 +0000 Subject: [PATCH 02/32] Renamed the Source V2. Also added tests for the same. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 385 ++- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 7 +- .../bigquery/BigQueryStorageSourceBase.java | 34 +- .../bigquery/BigQueryStorageSourceBaseV2.java | 5 - ...=> BigQueryStorageStreamBundleSource.java} | 49 +- .../bigquery/BigQueryIOStorageReadTest.java | 1 - ...StorageReadWithStreamBundleSourceTest.java | 2119 +++++++++++++++++ 7 files changed, 2363 insertions(+), 237 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{BigQueryStorageStreamSourceV2.java => BigQueryStorageStreamBundleSource.java} (88%) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java 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 37a471aa38c4..34292f446c9a 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 @@ -68,6 +68,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.util.Transport; @@ -1409,12 +1410,8 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - // if (bqOptions.getUseReadAPISourceV2()) { - // return expandAnonForDirectReadV2(input, outputCoder, beamSchema); - // } - - if(bqOptions.getUseReadAPISourceV2() || !bqOptions.getUseReadAPISourceV2()) { - + if (bqOptions.getUseReadAPIStreamBundleSource()) { + return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); } PCollectionView jobIdTokenView; @@ -1595,194 +1592,194 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - // private PCollection expandAnonForDirectReadV2( - // PBegin input, Coder outputCoder, Schema beamSchema) { - // Pipeline p = input.getPipeline(); - // PCollectionView jobIdTokenView; - // PCollection rows; - // - // if (!getWithTemplateCompatibility()) { - // // Create a singleton job ID token at pipeline construction time. - // String staticJobUuid = BigQueryHelpers.randomUUIDString(); - // jobIdTokenView = - // p.apply("TriggerIdCreation", Create.of(staticJobUuid)) - // .apply("ViewId", View.asSingleton()); - // // Apply the traditional Source model. - // rows = - // p.apply( - // org.apache.beam.sdk.io.Read.from( - // createStorageQuerySource(staticJobUuid, outputCoder))); - // } else { - // // Create a singleton job ID token at pipeline execution time. - // PCollection jobIdTokenCollection = - // p.apply("TriggerIdCreation", Create.of("ignored")) - // .apply( - // "CreateJobId", - // MapElements.via( - // new SimpleFunction() { - // @Override - // public String apply(String input) { - // return BigQueryHelpers.randomUUIDString(); - // } - // })); - // - // jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); - // - // TupleTag> listReadStreamsTag = new TupleTag<>(); - // TupleTag readSessionTag = new TupleTag<>(); - // TupleTag tableSchemaTag = new TupleTag<>(); - // - // PCollectionTuple tuple = - // jobIdTokenCollection.apply( - // "RunQueryJob", - // ParDo.of( - // new DoFn>() { - // @ProcessElement - // public void processElement(ProcessContext c) throws Exception { - // BigQueryOptions options = - // c.getPipelineOptions().as(BigQueryOptions.class); - // String jobUuid = c.element(); - // // Execute the query and get the destination table holding the results. - // // The getTargetTable call runs a new instance of the query and returns - // // the destination table created to hold the results. - // BigQueryStorageQuerySource querySource = - // createStorageQuerySource(jobUuid, outputCoder); - // Table queryResultTable = querySource.getTargetTable(options); - // - // // Create a read session without specifying a desired stream count and - // // let the BigQuery storage server pick the number of streams. - // CreateReadSessionRequest request = - // CreateReadSessionRequest.newBuilder() - // .setParent( - // BigQueryHelpers.toProjectResourceName( - // options.getBigQueryProject() == null - // ? options.getProject() - // : options.getBigQueryProject())) - // .setReadSession( - // ReadSession.newBuilder() - // .setTable( - // BigQueryHelpers.toTableResourceName( - // queryResultTable.getTableReference())) - // .setDataFormat(DataFormat.AVRO)) - // .setMaxStreamCount(0) - // .build(); - // - // ReadSession readSession; - // try (StorageClient storageClient = - // getBigQueryServices().getStorageClient(options)) { - // readSession = storageClient.createReadSession(request); - // } - // int streamIndex = 0; - // int streamsPerBundle = 10; - // List streamBundle = Lists.newArrayList(); - // for (ReadStream readStream : readSession.getStreamsList()) { - // streamIndex++; - // streamBundle.add(readStream); - // if (streamIndex % streamsPerBundle == 0) { - // c.output(streamBundle); - // streamBundle = Lists.newArrayList(); - // } - // } - // - // c.output(readSessionTag, readSession); - // c.output( - // tableSchemaTag, - // BigQueryHelpers.toJsonString(queryResultTable.getSchema())); - // } - // }) - // .withOutputTags( - // listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); - // - // TypeDescriptor> listReadStreamTypeDescriptor = new TypeDescriptor>() {}; - // tuple.get(listReadStreamsTag).setCoder(ProtoCoder.of(listReadStreamTypeDescriptor)); - // tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); - // tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - // - // PCollectionView readSessionView = - // tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); - // PCollectionView tableSchemaView = - // tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - // - // rows = - // tuple - // .get(listReadStreamsTag) - // .apply(Reshuffle.viaRandomKey()) - // .apply( - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) throws Exception { - // ReadSession readSession = c.sideInput(readSessionView); - // TableSchema tableSchema = - // BigQueryHelpers.fromJsonString( - // c.sideInput(tableSchemaView), TableSchema.class); - // List streamBundle = c.element(); - // - // BigQueryStorageStreamSourceV2 streamSource = - // BigQueryStorageStreamSourceV2.create( - // readSession, - // streamBundle, - // tableSchema, - // getParseFn(), - // outputCoder, - // getBigQueryServices()); - // - // // Read all of the data from the stream. In the event that this work - // // item fails and is rescheduled, the same rows will be returned in - // // the same order. - // BoundedReader reader = - // streamSource.createReader(c.getPipelineOptions()); - // for (boolean more = reader.start(); more; more = reader.advance()) { - // c.output(reader.getCurrent()); - // } - // } - // }) - // .withSideInputs(readSessionView, tableSchemaView)) - // .setCoder(outputCoder); - // } - // - // CleanupOperation cleanupOperation = - // new CleanupOperation() { - // @Override - // void cleanup(ContextContainer c) throws Exception { - // BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); - // String jobUuid = c.getJobId(); - // - // Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); - // - // TableReference tempTable = - // createTempTableReference( - // options.getBigQueryProject() == null - // ? options.getProject() - // : options.getBigQueryProject(), - // BigQueryResourceNaming.createJobIdPrefix( - // options.getJobName(), jobUuid, JobType.QUERY), - // queryTempDataset); - // - // try (DatasetService datasetService = - // getBigQueryServices().getDatasetService(options)) { - // LOG.info("Deleting temporary table with query results {}", tempTable); - // datasetService.deleteTable(tempTable); - // // Delete dataset only if it was created by Beam - // boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); - // if (datasetCreatedByBeam) { - // LOG.info( - // "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - // datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); - // } - // } - // } - // }; - // - // if (beamSchema != null) { - // rows.setSchema( - // beamSchema, - // getTypeDescriptor(), - // getToBeamRowFn().apply(beamSchema), - // getFromBeamRowFn().apply(beamSchema)); - // } - // return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); - // } + private PCollection expandAnonForDirectReadWithStreamBundle( + PBegin input, Coder outputCoder, Schema beamSchema) { + + Pipeline p = input.getPipeline(); + PCollectionView jobIdTokenView; + PCollection rows; + + if (!getWithTemplateCompatibility()) { + // Create a singleton job ID token at pipeline construction time. + String staticJobUuid = BigQueryHelpers.randomUUIDString(); + jobIdTokenView = + p.apply("TriggerIdCreation", Create.of(staticJobUuid)) + .apply("ViewId", View.asSingleton()); + // Apply the traditional Source model. + rows = + p.apply( + org.apache.beam.sdk.io.Read.from( + createStorageQuerySource(staticJobUuid, outputCoder))); + } else { + // Create a singleton job ID token at pipeline execution time. + PCollection jobIdTokenCollection = + p.apply("TriggerIdCreation", Create.of("ignored")) + .apply( + "CreateJobId", + MapElements.via( + new SimpleFunction() { + @Override + public String apply(String input) { + return BigQueryHelpers.randomUUIDString(); + } + })); + + jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); + + TupleTag> listReadStreamsTag = new TupleTag<>(); + TupleTag readSessionTag = new TupleTag<>(); + TupleTag tableSchemaTag = new TupleTag<>(); + + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); + } + int streamIndex = 0; + int streamsPerBundle = 10; + List streamBundle = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + c.output(streamBundle); + streamBundle = Lists.newArrayList(); + } + } + + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); + tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); + tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); + + PCollectionView readSessionView = + tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); + PCollectionView tableSchemaView = + tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); + + rows = + tuple + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); + } + + CleanupOperation cleanupOperation = + new CleanupOperation() { + @Override + void cleanup(ContextContainer c) throws Exception { + BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.getJobId(); + + Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); + + TableReference tempTable = + createTempTableReference( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject(), + BigQueryResourceNaming.createJobIdPrefix( + options.getJobName(), jobUuid, JobType.QUERY), + queryTempDataset); + + try (DatasetService datasetService = + getBigQueryServices().getDatasetService(options)) { + LOG.info("Deleting temporary table with query results {}", tempTable); + datasetService.deleteTable(tempTable); + // Delete dataset only if it was created by Beam + boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); + if (datasetCreatedByBeam) { + LOG.info( + "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + } + } + } + }; + + if (beamSchema != null) { + rows.setSchema( + beamSchema, + getTypeDescriptor(), + getToBeamRowFn().apply(beamSchema), + getFromBeamRowFn().apply(beamSchema)); + } + return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + } @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index cfd0ef984794..96998b880d23 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -151,9 +151,10 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); - @Description("If set, BigQueryIO.Read will default to using the V2 implementation of the Read API Source") + @Description("If set, BigQueryIO.Read will default to using the StreamBundle based" + + "implementation of the Read API Source") @Default.Boolean(false) - Boolean getUseReadAPISourceV2(); + Boolean getUseReadAPIStreamBundleSource(); - void setUseReadAPISourceV2(Boolean value); + void setUseReadAPIStreamBundleSource(Boolean value); } 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 8fab8f461d6a..e9b8b00f10c5 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 @@ -137,7 +137,7 @@ public List> split( // an appropriate number of streams for the Session to produce reasonable throughput. // This is required when using the Read API Source V2. int streamCount = 0; - if(!bqOptions.getUseReadAPISourceV2()) { + if (!bqOptions.getUseReadAPIStreamBundleSource()) { if (desiredBundleSizeBytes > 0) { long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); @@ -161,9 +161,10 @@ public List> split( try (StorageClient client = bqServices.getStorageClient(bqOptions)) { readSession = client.createReadSession(createReadSessionRequest); LOG.info( - "Sent BigQuery Storage API CreateReadSession request '{}'; received response '{}'.", - createReadSessionRequest, - readSession); + "Sent BigQuery Storage API CreateReadSession request in code '{}'.", + createReadSessionRequest); + LOG.info( + "Received number of streams in response: '{}'.", readSession.getStreamsList().size()); } if (readSession.getStreamsList().isEmpty()) { @@ -171,18 +172,23 @@ public List> split( return ImmutableList.of(); } + streamCount = readSession.getStreamsList().size(); int streamsPerBundle = 0; - if (bqOptions.getUseReadAPISourceV2()) { + double bytesPerStream = 0; + LOG.info( + "readSession.getEstimatedTotalBytesScanned(): '{}'", + readSession.getEstimatedTotalBytesScanned()); + if (bqOptions.getUseReadAPIStreamBundleSource()) { if (desiredBundleSizeBytes > 0) { - streamsPerBundle = - (int) - Math.ceil((double) - desiredBundleSizeBytes - / (double) (readSession.getEstimatedTotalBytesScanned() - / readSession.getStreamsList().size())); + bytesPerStream = + (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); + LOG.info("bytesPerStream: '{}'", bytesPerStream); + streamsPerBundle = (int) Math.ceil(desiredBundleSizeBytes / bytesPerStream); } else { streamsPerBundle = (int) Math.ceil((double) streamCount / 10); } + streamsPerBundle = Math.min(streamCount, streamsPerBundle); + LOG.info("streamsPerBundle: '{}'", streamsPerBundle); } Schema sessionSchema; @@ -204,7 +210,7 @@ public List> split( targetTable); // TODO: this is inconsistent with method above, where it can be null TableSchema trimmedSchema = BigQueryAvroUtils.trimBigQueryTableSchema(targetTable.getSchema(), sessionSchema); - if(!bqOptions.getUseReadAPISourceV2()) { + if (!bqOptions.getUseReadAPIStreamBundleSource()) { List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { sources.add( @@ -214,13 +220,13 @@ public List> split( return ImmutableList.copyOf(sources); } else { List streamBundle = Lists.newArrayList(); - List> sources = Lists.newArrayList(); + List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { streamIndex++; streamBundle.add(readStream); if (streamIndex % streamsPerBundle == 0) { sources.add( - BigQueryStorageStreamSourceV2.create( + BigQueryStorageStreamBundleSource.create( readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); streamBundle = Lists.newArrayList(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java deleted file mode 100644 index a0ba4e22d1e3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.beam.sdk.io.gcp.bigquery; - -public class BigQueryStorageSourceBaseV2 { - -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java similarity index 88% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index b4028996a0ef..fb788eb4920b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -45,22 +45,22 @@ 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; -class BigQueryStorageStreamSourceV2 extends BoundedSource { +class BigQueryStorageStreamBundleSource extends BoundedSource { - private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSourceV2.class); + private static final Logger LOG = + LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); - public static BigQueryStorageStreamSourceV2 create( + public static BigQueryStorageStreamBundleSource create( ReadSession readSession, List streamBundle, TableSchema tableSchema, SerializableFunction parseFn, Coder outputCoder, BigQueryServices bqServices) { - return new BigQueryStorageStreamSourceV2<>( + return new BigQueryStorageStreamBundleSource<>( readSession, streamBundle, toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), @@ -73,8 +73,8 @@ public static BigQueryStorageStreamSourceV2 create( * Creates a new source with the same properties as this one, except with a different {@link * List}. */ - public BigQueryStorageStreamSourceV2 fromExisting(List newStreamBundle) { - return new BigQueryStorageStreamSourceV2<>( + public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { + return new BigQueryStorageStreamBundleSource<>( readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); } @@ -85,7 +85,7 @@ public BigQueryStorageStreamSourceV2 fromExisting(List newStreamB private final Coder outputCoder; private final BigQueryServices bqServices; - private BigQueryStorageStreamSourceV2( + private BigQueryStorageStreamBundleSource( ReadSession readSession, List streamBundle, String jsonTableSchema, @@ -148,7 +148,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private final StorageClient storageClient; private final TableSchema tableSchema; - private BigQueryStorageStreamSourceV2 source; + private BigQueryStorageStreamBundleSource source; private @Nullable BigQueryServerStream responseStream = null; private @Nullable Iterator responseIterator = null; private @Nullable T current = null; @@ -157,6 +157,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B // Values used for progress reporting. private double fractionConsumed; + private double fractionOfStreamsConsumed; private double progressAtResponseStart; private double progressAtResponseEnd; private long rowsConsumedFromCurrentResponse; @@ -166,7 +167,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private @Nullable ServiceCallMetric serviceCallMetric; private BigQueryStorageStreamBundleReader( - BigQueryStorageStreamSourceV2 source, BigQueryOptions options) throws IOException { + BigQueryStorageStreamBundleSource source, BigQueryOptions options) throws IOException { this.source = source; this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); this.parseFn = source.parseFn; @@ -174,6 +175,7 @@ private BigQueryStorageStreamBundleReader( this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; this.fractionConsumed = 0d; + this.fractionOfStreamsConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -195,14 +197,14 @@ public synchronized boolean start() throws IOException { @Override public synchronized boolean advance() throws IOException { - Preconditions.checkStateNotNull(responseIterator); + // Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } private synchronized boolean readNextStream() throws IOException { - BigQueryStorageStreamSourceV2 source = getCurrentSource(); - if (currentStreamIndex == source.streamBundle.size() - 1) { + BigQueryStorageStreamBundleSource source = getCurrentSource(); + if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; return false; } @@ -211,7 +213,6 @@ private synchronized boolean readNextStream() throws IOException { .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) .setOffset(currentOffset) .build(); - tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( @@ -222,11 +223,15 @@ private synchronized boolean readNextStream() throws IOException { return readNextRecord(); } - @RequiresNonNull("responseIterator") + // @RequiresNonNull("responseIterator") private synchronized boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; + if (responseIterator == null) { + return false; + } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { + currentOffset = 0; currentStreamIndex++; return readNextStream(); } @@ -288,9 +293,12 @@ private synchronized boolean readNextRecord() throws IOException { * 1.0 / totalRowsInCurrentResponse; - // Assuming that each stream in the StreamBundle has approximately the same amount of data. + // Assuming that each stream in the StreamBundle has approximately the same amount of data and + // normalizing the value of fractionConsumed. fractionConsumed = fractionConsumed / source.streamBundle.size(); + fractionOfStreamsConsumed = (double) currentStreamIndex / source.streamBundle.size(); + fractionConsumed += fractionOfStreamsConsumed; return true; } @@ -298,14 +306,14 @@ private synchronized boolean readNextRecord() throws IOException { public synchronized void close() { // Because superclass cannot have preconditions around these variables, cannot use // @RequiresNonNull - Preconditions.checkStateNotNull(storageClient); - Preconditions.checkStateNotNull(reader); + // Preconditions.checkStateNotNull(storageClient); + // Preconditions.checkStateNotNull(reader); storageClient.close(); reader.close(); } @Override - public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { + public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } @@ -314,8 +322,9 @@ public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { public @Nullable BoundedSource splitAtFraction(double fraction) { int streamCountInBundle = source.streamBundle.size(); double splitIndex = streamCountInBundle * fraction; - if (currentStreamIndex > splitIndex) { + if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { // The reader has moved past the requested split point. + // NOTE: We do not split below the granularity of a stream. Metrics.counter( BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-failed-due-to-impossible-split-point") diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index 821cdfcb6fb2..518c4a80cdbc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -161,7 +161,6 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); - options.as(BigQueryOptions.class).setUseReadAPISourceV2(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java new file mode 100644 index 000000000000..bd0cb1124f8e --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -0,0 +1,2119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static java.util.Arrays.asList; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.rpc.FailedPreconditionException; +import com.google.api.services.bigquery.model.Streamingbuffer; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.ArrowRecordBatch; +import com.google.cloud.bigquery.storage.v1.ArrowSchema; +import com.google.cloud.bigquery.storage.v1.AvroRows; +import com.google.cloud.bigquery.storage.v1.AvroSchema; +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.DataFormat; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadStream; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; +import com.google.cloud.bigquery.storage.v1.StreamStats; +import com.google.cloud.bigquery.storage.v1.StreamStats.Progress; +import com.google.protobuf.ByteString; +import io.grpc.Status; +import io.grpc.Status.Code; +import io.grpc.StatusRuntimeException; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.math.BigInteger; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.VectorUnloader; +import org.apache.arrow.vector.ipc.WriteChannel; +import org.apache.arrow.vector.ipc.message.MessageSerializer; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.util.Text; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; +import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TableRowParser; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamBundleSource.BigQueryStorageStreamBundleReader; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.ConversionOptions; +import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices; +import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices.FakeBigQueryServerStream; +import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +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; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.junit.runners.model.Statement; + +/** + * Tests for {@link BigQueryIO#readTableRows() using {@link Method#DIRECT_READ}} AND {@link + * BigQueryOptions#setUseReadAPIStreamBundleSource(Boolean)} set to True. + */ +@RunWith(JUnit4.class) +public class BigQueryIOStorageReadWithStreamBundleSourceTest { + + private transient PipelineOptions options; + private final transient TemporaryFolder testFolder = new TemporaryFolder(); + private transient TestPipeline p; + private BufferAllocator allocator; + + @Rule + public final transient TestRule folderThenPipeline = + new TestRule() { + @Override + public Statement apply(Statement base, Description description) { + // We need to set up the temporary folder, and then set up the TestPipeline based on the + // chosen folder. Unfortunately, since rule evaluation order is unspecified and unrelated + // to field order, and is separate from construction, that requires manually creating this + // TestRule. + Statement withPipeline = + new Statement() { + @Override + public void evaluate() throws Throwable { + options = TestPipeline.testingPipelineOptions(); + options.as(BigQueryOptions.class).setProject("project-id"); + if (description.getAnnotations().stream() + .anyMatch(a -> a.annotationType().equals(ProjectOverride.class))) { + options.as(BigQueryOptions.class).setBigQueryProject("bigquery-project-id"); + } + options + .as(BigQueryOptions.class) + .setTempLocation(testFolder.getRoot().getAbsolutePath()); + options.as(BigQueryOptions.class).setUseReadAPIStreamBundleSource(true); + p = TestPipeline.fromOptions(options); + p.apply(base, description).evaluate(); + } + }; + return testFolder.apply(withPipeline, description); + } + }; + + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + private final FakeDatasetService fakeDatasetService = new FakeDatasetService(); + + @Before + public void setUp() throws Exception { + FakeDatasetService.setUp(); + allocator = new RootAllocator(Long.MAX_VALUE); + } + + @After + public void teardown() { + allocator.close(); + } + + @Test + public void testBuildTableBasedSource() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table"); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + assertTrue(typedRead.getValidate()); + } + + @Test + public void testBuildTableBasedSourceWithoutValidation() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .withoutValidation(); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + assertFalse(typedRead.getValidate()); + } + + @Test + public void testBuildTableBasedSourceWithDefaultProject() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("myDataset.myTable"); + checkTypedReadTableObject(typedRead, null, "myDataset", "myTable"); + } + + @Test + public void testBuildTableBasedSourceWithTableReference() { + TableReference tableReference = + new TableReference() + .setProjectId("foo.com:project") + .setDatasetId("dataset") + .setTableId("table"); + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from(tableReference); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + } + + private void checkTypedReadTableObject( + TypedRead typedRead, String project, String dataset, String table) { + assertEquals(project, typedRead.getTable().getProjectId()); + assertEquals(dataset, typedRead.getTable().getDatasetId()); + assertEquals(table, typedRead.getTable().getTableId()); + assertNull(typedRead.getQuery()); + assertEquals(Method.DIRECT_READ, typedRead.getMethod()); + } + + @Test + public void testBuildSourceWithTableAndFlatten() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid BigQueryIO.Read: Specifies a table with a result flattening preference," + + " which only applies to queries"); + p.apply( + "ReadMyTable", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .withoutResultFlattening()); + p.run(); + } + + @Test + public void testBuildSourceWithTableAndSqlDialect() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid BigQueryIO.Read: Specifies a table with a SQL dialect preference," + + " which only applies to queries"); + p.apply( + "ReadMyTable", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .usingStandardSql()); + p.run(); + } + + @Test + public void testDisplayData() { + String tableSpec = "foo.com:project:dataset.table"; + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .withSelectedFields(ImmutableList.of("foo", "bar")) + .withProjectionPushdownApplied() + .from(tableSpec); + DisplayData displayData = DisplayData.from(typedRead); + assertThat(displayData, hasDisplayItem("table", tableSpec)); + assertThat(displayData, hasDisplayItem("selectedFields", "foo, bar")); + assertThat(displayData, hasDisplayItem("projectionPushdownApplied", true)); + } + + @Test + public void testName() { + assertEquals( + "BigQueryIO.TypedRead", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .getName()); + } + + @Test + public void testCoderInference() { + // Lambdas erase too much type information -- use an anonymous class here. + SerializableFunction> parseFn = + new SerializableFunction>() { + @Override + public KV apply(SchemaAndRecord input) { + return null; + } + }; + + assertEquals( + KvCoder.of(ByteStringCoder.of(), ProtoCoder.of(ReadSession.class)), + BigQueryIO.read(parseFn).inferCoder(CoderRegistry.createDefault())); + } + + @Test + public void testTableSourceEstimatedSize() throws Exception { + doTableSourceEstimatedSizeTest(false); + } + + @Test + public void testTableSourceEstimatedSize_IgnoresStreamingBuffer() throws Exception { + doTableSourceEstimatedSizeTest(true); + } + + private void doTableSourceEstimatedSizeTest(boolean useStreamingBuffer) throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + if (useStreamingBuffer) { + table.setStreamingBuffer(new Streamingbuffer().setEstimatedBytes(BigInteger.TEN)); + } + + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + @Test + @ProjectOverride + public void testTableSourceEstimatedSize_WithBigQueryProject() throws Exception { + fakeDatasetService.createDataset("bigquery-project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("bigquery-project-id:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + @Test + public void testTableSourceEstimatedSize_WithDefaultProject() throws Exception { + fakeDatasetService.createDataset("project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + private static final String AVRO_SCHEMA_STRING = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RowRecord\",\n" + + " \"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"number\", \"type\": \"long\"}\n" + + " ]\n" + + "}"; + + private static final Schema AVRO_SCHEMA = new Schema.Parser().parse(AVRO_SCHEMA_STRING); + + private static final String TRIMMED_AVRO_SCHEMA_STRING = + "{\"namespace\": \"example.avro\",\n" + + "\"type\": \"record\",\n" + + "\"name\": \"RowRecord\",\n" + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"}\n" + + " ]\n" + + "}"; + + private static final Schema TRIMMED_AVRO_SCHEMA = + new Schema.Parser().parse(TRIMMED_AVRO_SCHEMA_STRING); + + private static final TableSchema TABLE_SCHEMA = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING").setMode("REQUIRED"), + new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"))); + + private static final org.apache.arrow.vector.types.pojo.Schema ARROW_SCHEMA = + new org.apache.arrow.vector.types.pojo.Schema( + asList( + field("name", new ArrowType.Utf8()), field("number", new ArrowType.Int(64, true)))); + + private void doTableSourceInitialSplitTest(long bundleSize, long tableSize, int streamCount) + throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = + new Table().setTableReference(tableRef).setNumBytes(tableSize).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(tableSize); + for (int i = 0; i < streamCount; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(bundleSize, options); + // Each StreamBundle is expected to contain a single stream. + assertEquals(streamCount, sources.size()); + } + + @Test + public void testTableSourceInitialSplit() throws Exception { + doTableSourceInitialSplitTest(1024L, 1024L * 1024L, 1024); + } + + @Test + public void testTableSourceInitialSplit_MinSplitCount() throws Exception { + doTableSourceInitialSplitTest(1024L, 1024L * 1024L, 10); + } + + @Test + public void testTableSourceInitialSplit_MaxSplitCount() throws Exception { + doTableSourceInitialSplitTest(10L, 1024L * 1024L, 10_000); + } + + @Test + public void testTableSourceInitialSplit_WithSelectedFieldsAndRowRestriction() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = new Table().setTableReference(tableRef).setNumBytes(200L).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder() + .addSelectedFields("name") + .setRowRestriction("number > 5"))) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(100L); + for (int i = 0; i < 10; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + StaticValueProvider.of(Lists.newArrayList("name")), + StaticValueProvider.of("number > 5"), + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(20L, options); + assertEquals(5, sources.size()); + } + + @Test + public void testTableSourceInitialSplit_WithDefaultProject() throws Exception { + fakeDatasetService.createDataset("project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset.table"); + + Table table = + new Table().setTableReference(tableRef).setNumBytes(1024L).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/project-id/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(1024L); + for (int i = 0; i < 50; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(4096L, options); + // A single StreamBundle containing all the Streams. + assertEquals(1, sources.size()); + } + + @Test + public void testTableSourceInitialSplit_EmptyTable() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = + new Table() + .setTableReference(tableRef) + .setNumBytes(1024L * 1024L) + .setSchema(new TableSchema()); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession emptyReadSession = ReadSession.newBuilder().build(); + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(emptyReadSession); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(1024L, options); + assertTrue(sources.isEmpty()); + } + + @Test + public void testTableSourceCreateReader() throws Exception { + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of( + BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage("BigQuery storage source must be split before reading"); + tableSource.createReader(options); + } + + private static GenericRecord createRecord(String name, Schema schema) { + GenericRecord genericRecord = new Record(schema); + genericRecord.put("name", name); + return genericRecord; + } + + private static GenericRecord createRecord(String name, long number, Schema schema) { + GenericRecord genericRecord = new Record(schema); + genericRecord.put("name", name); + genericRecord.put("number", number); + return genericRecord; + } + + private static ByteString serializeArrowSchema( + org.apache.arrow.vector.types.pojo.Schema arrowSchema) { + ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream(); + try { + MessageSerializer.serialize( + new WriteChannel(Channels.newChannel(byteOutputStream)), arrowSchema); + } catch (IOException ex) { + throw new RuntimeException("Failed to serialize arrow schema.", ex); + } + return ByteString.copyFrom(byteOutputStream.toByteArray()); + } + + private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get(); + + private static ReadRowsResponse createResponse( + Schema schema, + Collection genericRecords, + double progressAtResponseStart, + double progressAtResponseEnd) + throws Exception { + GenericDatumWriter writer = new GenericDatumWriter<>(schema); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + Encoder binaryEncoder = ENCODER_FACTORY.binaryEncoder(outputStream, null); + for (GenericRecord genericRecord : genericRecords) { + writer.write(genericRecord, binaryEncoder); + } + + binaryEncoder.flush(); + + return ReadRowsResponse.newBuilder() + .setAvroRows( + AvroRows.newBuilder() + .setSerializedBinaryRows(ByteString.copyFrom(outputStream.toByteArray())) + .setRowCount(genericRecords.size())) + .setRowCount(genericRecords.size()) + .setStats( + StreamStats.newBuilder() + .setProgress( + Progress.newBuilder() + .setAtResponseStart(progressAtResponseStart) + .setAtResponseEnd(progressAtResponseEnd))) + .build(); + } + + private ReadRowsResponse createResponseArrow( + org.apache.arrow.vector.types.pojo.Schema arrowSchema, + List name, + List number, + double progressAtResponseStart, + double progressAtResponseEnd) { + ArrowRecordBatch serializedRecord; + try (VectorSchemaRoot schemaRoot = VectorSchemaRoot.create(arrowSchema, allocator)) { + schemaRoot.allocateNew(); + schemaRoot.setRowCount(name.size()); + VarCharVector strVector = (VarCharVector) schemaRoot.getFieldVectors().get(0); + BigIntVector bigIntVector = (BigIntVector) schemaRoot.getFieldVectors().get(1); + for (int i = 0; i < name.size(); i++) { + bigIntVector.set(i, number.get(i)); + strVector.set(i, new Text(name.get(i))); + } + + VectorUnloader unLoader = new VectorUnloader(schemaRoot); + try (org.apache.arrow.vector.ipc.message.ArrowRecordBatch records = + unLoader.getRecordBatch()) { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + MessageSerializer.serialize(new WriteChannel(Channels.newChannel(os)), records); + serializedRecord = + ArrowRecordBatch.newBuilder() + .setRowCount(records.getLength()) + .setSerializedRecordBatch(ByteString.copyFrom(os.toByteArray())) + .build(); + } catch (IOException e) { + throw new RuntimeException("Error writing to byte array output stream", e); + } + } + } + + return ReadRowsResponse.newBuilder() + .setArrowRecordBatch(serializedRecord) + .setRowCount(name.size()) + .setStats( + StreamStats.newBuilder() + .setProgress( + Progress.newBuilder() + .setAtResponseStart(progressAtResponseStart) + .setAtResponseEnd(progressAtResponseEnd))) + .build(); + } + + @Test + public void testStreamSourceEstimatedSizeBytes() throws Exception { + List streamBundle = Lists.newArrayList(ReadStream.getDefaultInstance()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.getDefaultInstance(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices()); + + assertEquals(0, streamSource.getEstimatedSizeBytes(options)); + } + + @Test + public void testStreamSourceSplit() throws Exception { + List streamBundle = Lists.newArrayList(ReadStream.getDefaultInstance()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.getDefaultInstance(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices()); + + assertThat(streamSource.split(0, options), containsInAnyOrder(streamSource)); + } + + @Test + public void testReadFromStreamSource() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").setOffset(0).build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").setOffset(0).build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)); + + List responsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(2, 3), 0.5, 0.75)); + List responsesTwo = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(3, 5), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(5, 6), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + List rows = new ArrayList<>(); + BigQueryStorageStreamBundleReader reader = streamSource.createReader(options); + for (boolean hasNext = reader.start(); hasNext; hasNext = reader.advance()) { + rows.add(reader.getCurrent()); + } + + System.out.println("Rows: " + rows); + + assertEquals(6, rows.size()); + } + + private static final double DELTA = 1e-6; + + @Test + public void testFractionConsumedWithOneStreamInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List responses = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.25), + // Some responses may contain zero results, so we must ensure that we can are resilient + // to such responses. + createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.25, 0.25), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.3, 0.5), + createResponse(AVRO_SCHEMA, records.subList(4, 7), 0.7, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.4, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.8, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.9, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the stream, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List responsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.5), + // Some responses may contain zero results, so we must ensure that we can are resilient + // to such responses. + createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.5, 0.5), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); + + List responsesTwo = + Lists.newArrayList(createResponse(AVRO_SCHEMA, records.subList(4, 7), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.375, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.6666666666666666, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.8333333333333333, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the streams, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testStreamSourceSplitAtFractionNoOpWithOneStreamInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List parentStreamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); + BigQueryStorageStreamBundleSource streamBundleSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + parentStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent stream and ensure that records are returned in the + // prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the stream. Since we do NOT split below the granularity of a single stream, + // this will be a No-Op and the primary source should be read to completion. + BoundedSource secondary = primary.splitAtFraction(0.5); + assertNull(secondary); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the primary bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the StreamBundle and ensure that the returned source points to a non-null secondary + // StreamBundle + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNotNull(secondarySource); + BoundedReader secondary = secondarySource.createReader(options); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertFalse(secondary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("C", 3, AVRO_SCHEMA), createRecord("D", 4, AVRO_SCHEMA)), + 0.25, + 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("E", 5, AVRO_SCHEMA), createRecord("F", 6, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); + + BoundedSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader primaryReader = primarySource.createReader(options); + assertTrue(primaryReader.start()); + assertEquals("A", primaryReader.getCurrent().get("name")); + + // Should create two sources: the first with 1 stream, the second with 2. + BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + assertNotNull(secondarySource); + assertEquals("A", primaryReader.getCurrent().get("name")); + + assertTrue(primaryReader.advance()); + assertEquals("B", primaryReader.getCurrent().get("name")); + assertFalse(primaryReader.advance()); + + BoundedReader secondaryReader = secondarySource.createReader(options); + assertTrue(secondaryReader.start()); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + // Should create two sources: each with 1 stream. + BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + assertNotNull(tertiarySource); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + assertTrue(secondaryReader.advance()); + assertEquals("D", secondaryReader.getCurrent().get("name")); + assertFalse(secondaryReader.advance()); + + BoundedReader tertiaryReader = tertiarySource.createReader(options); + assertTrue(tertiaryReader.start()); + assertEquals("E", tertiaryReader.getCurrent().get("name")); + + assertTrue(tertiaryReader.advance()); + assertEquals("F", tertiaryReader.getCurrent().get("name")); + assertFalse(tertiaryReader.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List parentStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamBundleSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + parentStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + + // We attempt to split the StreamBundle after starting to read the contents of the second + // stream. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNull(secondarySource); + + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + } + + private static final class ParseKeyValue + implements SerializableFunction> { + + @Override + public KV apply(SchemaAndRecord input) { + return KV.of( + input.getRecord().get("name").toString(), (Long) input.getRecord().get("number")); + } + } + + @Test + public void testReadFromBigQueryIO() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(10L) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection> output = + p.apply( + BigQueryIO.read(new ParseKeyValue()) + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + KV.of("A", 1L), + KV.of("B", 2L), + KV.of("C", 3L), + KV.of("D", 4L), + KV.of("E", 5L), + KV.of("F", 6L), + KV.of("G", 7L))); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOWithTrimmedSchema() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder().addSelectedFields("name")) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", TRIMMED_AVRO_SCHEMA), + createRecord("B", TRIMMED_AVRO_SCHEMA), + createRecord("C", TRIMMED_AVRO_SCHEMA), + createRecord("D", TRIMMED_AVRO_SCHEMA), + createRecord("E", TRIMMED_AVRO_SCHEMA), + createRecord("F", TRIMMED_AVRO_SCHEMA), + createRecord("G", TRIMMED_AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection output = + p.apply( + BigQueryIO.readTableRows() + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withSelectedFields(Lists.newArrayList("name")) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + new TableRow().set("name", "A"), + new TableRow().set("name", "B"), + new TableRow().set("name", "C"), + new TableRow().set("name", "D"), + new TableRow().set("name", "E"), + new TableRow().set("name", "F"), + new TableRow().set("name", "G"))); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOWithBeamSchema() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder().addSelectedFields("name")) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", TRIMMED_AVRO_SCHEMA), + createRecord("B", TRIMMED_AVRO_SCHEMA), + createRecord("C", TRIMMED_AVRO_SCHEMA), + createRecord("D", TRIMMED_AVRO_SCHEMA), + createRecord("E", TRIMMED_AVRO_SCHEMA), + createRecord("F", TRIMMED_AVRO_SCHEMA), + createRecord("G", TRIMMED_AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection output = + p.apply( + BigQueryIO.readTableRowsWithSchema() + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withSelectedFields(Lists.newArrayList("name")) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))) + .apply(Convert.toRows()); + + org.apache.beam.sdk.schemas.Schema beamSchema = + org.apache.beam.sdk.schemas.Schema.of( + org.apache.beam.sdk.schemas.Schema.Field.of( + "name", org.apache.beam.sdk.schemas.Schema.FieldType.STRING)); + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + Row.withSchema(beamSchema).addValue("A").build(), + Row.withSchema(beamSchema).addValue("B").build(), + Row.withSchema(beamSchema).addValue("C").build(), + Row.withSchema(beamSchema).addValue("D").build(), + Row.withSchema(beamSchema).addValue("E").build(), + Row.withSchema(beamSchema).addValue("F").build(), + Row.withSchema(beamSchema).addValue("G").build())); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOArrow() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setDataFormat(DataFormat.ARROW)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List readRowsResponsesOne = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.50), + createResponseArrow( + ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(4, 5), values.subList(4, 5), 0.0, 0.33), + createResponseArrow( + ARROW_SCHEMA, names.subList(5, 6), values.subList(5, 6), 0.33, 0.66), + createResponseArrow( + ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.66, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection> output = + p.apply( + BigQueryIO.read(new ParseKeyValue()) + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withFormat(DataFormat.ARROW) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + KV.of("A", 1L), + KV.of("B", 2L), + KV.of("C", 3L), + KV.of("D", 4L), + KV.of("E", 5L), + KV.of("F", 6L), + KV.of("G", 7L))); + + p.run(); + } + + @Test + public void testReadFromStreamSourceArrow() throws Exception { + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List names = Arrays.asList("A", "B", "C"); + List values = Arrays.asList(1L, 2L, 3L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.50), + createResponseArrow( + ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + List rows = new ArrayList<>(); + BoundedReader reader = streamSource.createReader(options); + for (boolean hasNext = reader.start(); hasNext; hasNext = reader.advance()) { + rows.add(reader.getCurrent()); + } + + System.out.println("Rows: " + rows); + + assertEquals(3, rows.size()); + } + + @Test + public void testFractionConsumedWithArrowAndOneStreamInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), + createResponseArrow( + ARROW_SCHEMA, Lists.newArrayList(), Lists.newArrayList(), 0.25, 0.25), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.3, 0.5), + createResponseArrow(ARROW_SCHEMA, names.subList(4, 7), values.subList(4, 7), 0.7, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.4, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.8, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.9, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the stream, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testFractionConsumedWithArrowAndMultipleStreamsInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responsesOne = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.5), + createResponseArrow(ARROW_SCHEMA, Lists.newArrayList(), Lists.newArrayList(), 0.5, 0.5), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.5, 1.0)); + + List responsesTwo = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(4, 7), values.subList(4, 7), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.375, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.6666666666666666, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.8333333333333333, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the streams, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary + // StreamBundle. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNotNull(secondarySource); + BoundedReader secondary = secondarySource.createReader(options); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertFalse(secondary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsInBundle() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(4, 6), values.subList(4, 6), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); + + BoundedSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader primaryReader = primarySource.createReader(options); + assertTrue(primaryReader.start()); + assertEquals("A", primaryReader.getCurrent().get("name")); + + // Should create two sources: the first with 1 stream, the second with 2. + BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + assertNotNull(secondarySource); + assertEquals("A", primaryReader.getCurrent().get("name")); + + assertTrue(primaryReader.advance()); + assertEquals("B", primaryReader.getCurrent().get("name")); + assertFalse(primaryReader.advance()); + + BoundedReader secondaryReader = secondarySource.createReader(options); + assertTrue(secondaryReader.start()); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + // Should create two sources: each with 1 stream. + BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + assertNotNull(tertiarySource); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + assertTrue(secondaryReader.advance()); + assertEquals("D", secondaryReader.getCurrent().get("name")); + assertFalse(secondaryReader.advance()); + + BoundedReader tertiaryReader = tertiarySource.createReader(options); + assertTrue(tertiaryReader.start()); + assertEquals("E", tertiaryReader.getCurrent().get("name")); + + assertTrue(tertiaryReader.advance()); + assertEquals("F", tertiaryReader.getCurrent().get("name")); + assertFalse(tertiaryReader.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List parentResponses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.25, 0.5), + createResponseArrow( + ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(parentResponses)); + + // Mocks the split call. A response without a primary_stream and remainder_stream means + // that the split is not possible. + // Mocks the split call. + when(fakeStorageClient.splitReadStream( + SplitReadStreamRequest.newBuilder().setName("parentStream").setFraction(0.5f).build())) + .thenReturn( + SplitReadStreamResponse.newBuilder() + .setPrimaryStream(ReadStream.newBuilder().setName("primaryStream")) + .setRemainderStream(ReadStream.newBuilder().setName("remainderStream")) + .build()); + + // Mocks the ReadRows calls expected on the primary and residual streams. + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder() + .setReadStream("primaryStream") + // This test will read rows 0 and 1 from the parent before calling split, + // so we expect the primary read to start at offset 2. + .setOffset(2) + .build(), + "")) + .thenThrow( + new FailedPreconditionException( + "Given row offset is invalid for stream.", + new StatusRuntimeException(Status.FAILED_PRECONDITION), + GrpcStatusCode.of(Code.FAILED_PRECONDITION), + /* retryable = */ false)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent stream and ensure that records are returned in the + // prescribed order. + BoundedReader parent = streamSource.createReader(options); + assertTrue(parent.start()); + assertEquals("A", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("B", parent.getCurrent().get("name")); + + assertNull(parent.splitAtFraction(0.5)); + + // Verify that the parent source still works okay even after an unsuccessful split attempt. + assertTrue(parent.advance()); + assertEquals("C", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("D", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("E", parent.getCurrent().get("name")); + assertFalse(parent.advance()); + } + + @Test + public void testActuateProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertTrue(read.supportsProjectionPushdown()); + PTransform> pushdownT = + read.actuateProjectionPushdown( + ImmutableMap.of(new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo"))); + + TypedRead pushdownRead = (TypedRead) pushdownT; + assertEquals(Method.DIRECT_READ, pushdownRead.getMethod()); + assertThat(pushdownRead.getSelectedFields().get(), Matchers.containsInAnyOrder("foo")); + assertTrue(pushdownRead.getProjectionPushdownApplied()); + } + + @Test + public void testReadFromQueryDoesNotSupportProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .fromQuery("SELECT bar FROM `dataset.table`") + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertFalse(read.supportsProjectionPushdown()); + assertThrows( + IllegalArgumentException.class, + () -> + read.actuateProjectionPushdown( + ImmutableMap.of( + new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo")))); + } + + private static org.apache.arrow.vector.types.pojo.Field field( + String name, + boolean nullable, + ArrowType type, + org.apache.arrow.vector.types.pojo.Field... children) { + return new org.apache.arrow.vector.types.pojo.Field( + name, + new org.apache.arrow.vector.types.pojo.FieldType(nullable, type, null, null), + asList(children)); + } + + static org.apache.arrow.vector.types.pojo.Field field( + String name, ArrowType type, org.apache.arrow.vector.types.pojo.Field... children) { + return field(name, false, type, children); + } +} From fc55a4f61d368fdbfcf658180629a7c9625bc785 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 2 Feb 2023 18:28:22 +0000 Subject: [PATCH 03/32] v2 using OffsetBasedSource and OffsetBasedReader --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 +- .../bigquery/BigQueryStorageSourceBase.java | 2 +- .../BigQueryStorageStreamBundleSource.java | 111 +++++++++++------- ...StorageReadWithStreamBundleSourceTest.java | 52 +++++--- 4 files changed, 106 insertions(+), 62 deletions(-) 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 34292f446c9a..ffe5465c1fe8 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 @@ -1722,7 +1722,8 @@ public void processElement(ProcessContext c) throws Exception { tableSchema, getParseFn(), outputCoder, - getBigQueryServices()); + getBigQueryServices(), + 1L); // Read all of the data from the stream. In the event that this work // item fails and is rescheduled, the same rows will be returned in 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 e9b8b00f10c5..af68277fc100 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 @@ -227,7 +227,7 @@ public List> split( if (streamIndex % streamsPerBundle == 0) { sources.add( BigQueryStorageStreamBundleSource.create( - readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); streamBundle = Lists.newArrayList(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index fb788eb4920b..a6385592e0c8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -35,7 +35,7 @@ import java.util.NoSuchElementException; import org.apache.beam.runners.core.metrics.ServiceCallMetric; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.OffsetBasedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; import org.apache.beam.sdk.metrics.Metrics; @@ -48,7 +48,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class BigQueryStorageStreamBundleSource extends BoundedSource { +class BigQueryStorageStreamBundleSource extends OffsetBasedSource { private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); @@ -59,14 +59,16 @@ public static BigQueryStorageStreamBundleSource create( TableSchema tableSchema, SerializableFunction parseFn, Coder outputCoder, - BigQueryServices bqServices) { + BigQueryServices bqServices, + long minBundleSize) { return new BigQueryStorageStreamBundleSource<>( readSession, streamBundle, toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), parseFn, outputCoder, - bqServices); + bqServices, + minBundleSize); } /** @@ -75,7 +77,7 @@ public static BigQueryStorageStreamBundleSource create( */ public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { return new BigQueryStorageStreamBundleSource<>( - readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); + readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices, getMinBundleSize()); } private final ReadSession readSession; @@ -91,7 +93,9 @@ private BigQueryStorageStreamBundleSource( String jsonTableSchema, SerializableFunction parseFn, Coder outputCoder, - BigQueryServices bqServices) { + BigQueryServices bqServices, + long minBundleSize) { + super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); this.jsonTableSchema = Preconditions.checkArgumentNotNull(jsonTableSchema, "jsonTableSchema"); @@ -124,13 +128,24 @@ public long getEstimatedSizeBytes(PipelineOptions options) { } @Override - public List> split( + public List> split( long desiredBundleSizeBytes, PipelineOptions options) { // A stream source can't be split without reading from it due to server-side liquid sharding. // TODO: Implement dynamic work rebalancing. return ImmutableList.of(this); } + @Override + public long getMaxEndOffset(PipelineOptions options) throws Exception { + return this.streamBundle.size(); + } + + @Override + public OffsetBasedSource createSourceForSubrange(long start, long end) { + List newStreamBundle = streamBundle.subList((int) start, (int) end); + return fromExisting(newStreamBundle); + } + @Override public BigQueryStorageStreamBundleReader createReader(PipelineOptions options) throws IOException { @@ -142,7 +157,7 @@ public BigQueryStorageStreamBundleReader createReader(PipelineOptions options // return readStream.toString(); // } - public static class BigQueryStorageStreamBundleReader extends BoundedSource.BoundedReader { + public static class BigQueryStorageStreamBundleReader extends OffsetBasedReader { private final BigQueryStorageReader reader; private final SerializableFunction parseFn; private final StorageClient storageClient; @@ -168,6 +183,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private BigQueryStorageStreamBundleReader( BigQueryStorageStreamBundleSource source, BigQueryOptions options) throws IOException { + super(source); this.source = source; this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); this.parseFn = source.parseFn; @@ -191,18 +207,31 @@ public T getCurrent() throws NoSuchElementException { } @Override - public synchronized boolean start() throws IOException { + protected long getCurrentOffset() throws NoSuchElementException { + return currentStreamIndex; + } + + @Override + protected boolean isAtSplitPoint() throws NoSuchElementException { + if (currentOffset == 0) { + return true; + } + return false; + } + + @Override + public boolean startImpl() throws IOException { return readNextStream(); } @Override - public synchronized boolean advance() throws IOException { + public boolean advanceImpl() throws IOException { // Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } - private synchronized boolean readNextStream() throws IOException { + private boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; @@ -224,7 +253,7 @@ private synchronized boolean readNextStream() throws IOException { } // @RequiresNonNull("responseIterator") - private synchronized boolean readNextRecord() throws IOException { + private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { return false; @@ -317,35 +346,35 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } - @Override - @SuppressWarnings("ReturnValueIgnored") - public @Nullable BoundedSource splitAtFraction(double fraction) { - int streamCountInBundle = source.streamBundle.size(); - double splitIndex = streamCountInBundle * fraction; - if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { - // The reader has moved past the requested split point. - // NOTE: We do not split below the granularity of a stream. - Metrics.counter( - BigQueryStorageStreamBundleReader.class, - "split-at-fraction-calls-failed-due-to-impossible-split-point") - .inc(); - LOG.info( - "BigQuery Storage API Session {} cannot be split at {}.", - source.readSession.getName(), - fraction); - return null; - } - // Splitting the remainder Streams into a new StreamBundle. - List remainderStreamBundle = - new ArrayList<>( - source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); - // Updating the primary StreamBundle. - source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); - Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") - .inc(); - LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); - return source.fromExisting(remainderStreamBundle); - } + // @Override + // @SuppressWarnings("ReturnValueIgnored") + // public @Nullable OffsetBasedSource splitAtFraction(double fraction) { + // int streamCountInBundle = source.streamBundle.size(); + // double splitIndex = streamCountInBundle * fraction; + // if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { + // // The reader has moved past the requested split point. + // // NOTE: We do not split below the granularity of a stream. + // Metrics.counter( + // BigQueryStorageStreamBundleReader.class, + // "split-at-fraction-calls-failed-due-to-impossible-split-point") + // .inc(); + // LOG.info( + // "BigQuery Storage API Session {} cannot be split at {}.", + // source.readSession.getName(), + // fraction); + // return null; + // } + // // Splitting the remainder Streams into a new StreamBundle. + // List remainderStreamBundle = + // new ArrayList<>( + // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); + // // Updating the primary StreamBundle. + // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); + // Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") + // .inc(); + // LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); + // return source.fromExisting(remainderStreamBundle); + // } @Override public synchronized Double getFractionConsumed() { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index bd0cb1124f8e..87dfe1891175 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -753,7 +753,8 @@ public void testStreamSourceEstimatedSizeBytes() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices()); + new FakeBigQueryServices(), + 1L); assertEquals(0, streamSource.getEstimatedSizeBytes(options)); } @@ -768,7 +769,8 @@ public void testStreamSourceSplit() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices()); + new FakeBigQueryServices(), + 1L); assertThat(streamSource.split(0, options), containsInAnyOrder(streamSource)); } @@ -821,7 +823,8 @@ public void testReadFromStreamSource() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); List rows = new ArrayList<>(); BigQueryStorageStreamBundleReader reader = streamSource.createReader(options); @@ -879,7 +882,8 @@ public void testFractionConsumedWithOneStreamInBundle() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -961,7 +965,8 @@ public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1027,7 +1032,8 @@ public void testStreamSourceSplitAtFractionNoOpWithOneStreamInBundle() throws Ex TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent stream and ensure that records are returned in the // prescribed order. @@ -1093,7 +1099,8 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the primary bundle and ensure the records are returned in // the prescribed order. @@ -1171,14 +1178,15 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader primaryReader = primarySource.createReader(options); assertTrue(primaryReader.start()); assertEquals("A", primaryReader.getCurrent().get("name")); // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); assertNotNull(secondarySource); assertEquals("A", primaryReader.getCurrent().get("name")); @@ -1250,7 +1258,8 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() thr TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent bundle and ensure the records are returned in // the prescribed order. @@ -1664,7 +1673,8 @@ public void testReadFromStreamSourceArrow() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); List rows = new ArrayList<>(); BoundedReader reader = streamSource.createReader(options); @@ -1715,7 +1725,8 @@ public void testFractionConsumedWithArrowAndOneStreamInBundle() throws Exception TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1792,7 +1803,8 @@ public void testFractionConsumedWithArrowAndMultipleStreamsInBundle() throws Exc TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1860,7 +1872,8 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the primary bundle and ensure that records are returned in the // prescribed order. @@ -1871,8 +1884,7 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("B", primary.getCurrent().get("name")); // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary - // StreamBundle. + // secondary StreamBundle. BoundedSource secondarySource = primary.splitAtFraction(0.5); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); @@ -1931,14 +1943,15 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader primaryReader = primarySource.createReader(options); assertTrue(primaryReader.start()); assertEquals("A", primaryReader.getCurrent().get("name")); // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); assertNotNull(secondarySource); assertEquals("A", primaryReader.getCurrent().get("name")); @@ -2028,7 +2041,8 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow( TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent stream and ensure that records are returned in the // prescribed order. From 0b09d5164f286c1c394f4f929cb9867a577cce77 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 3 Feb 2023 00:06:34 +0000 Subject: [PATCH 04/32] Updating tests to have more sensible mock values. --- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 5 +- .../BigQueryStorageStreamBundleSource.java | 63 +--- ...StorageReadWithStreamBundleSourceTest.java | 323 +++++++++--------- 3 files changed, 183 insertions(+), 208 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 96998b880d23..433fb92b3951 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -151,8 +151,9 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); - @Description("If set, BigQueryIO.Read will default to using the StreamBundle based" - + "implementation of the Read API Source") + @Description( + "If set, BigQueryIO.Read will default to using the StreamBundle based" + + "implementation of the Read API Source") @Default.Boolean(false) Boolean getUseReadAPIStreamBundleSource(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index a6385592e0c8..99f4629db301 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -29,7 +29,6 @@ import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.ReadStream; import java.io.IOException; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -38,21 +37,18 @@ import org.apache.beam.sdk.io.OffsetBasedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; -import org.apache.beam.sdk.metrics.Metrics; 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.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; class BigQueryStorageStreamBundleSource extends OffsetBasedSource { - private static final Logger LOG = - LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); - public static BigQueryStorageStreamBundleSource create( ReadSession readSession, List streamBundle, @@ -77,7 +73,13 @@ public static BigQueryStorageStreamBundleSource create( */ public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { return new BigQueryStorageStreamBundleSource<>( - readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices, getMinBundleSize()); + readSession, + newStreamBundle, + jsonTableSchema, + parseFn, + outputCoder, + bqServices, + getMinBundleSize()); } private final ReadSession readSession; @@ -152,12 +154,10 @@ public BigQueryStorageStreamBundleReader createReader(PipelineOptions options return new BigQueryStorageStreamBundleReader<>(this, options.as(BigQueryOptions.class)); } - // @Override - // public String toString() { - // return readStream.toString(); - // } - public static class BigQueryStorageStreamBundleReader extends OffsetBasedReader { + private static final Logger LOG = + LoggerFactory.getLogger(BigQueryStorageStreamBundleReader.class); + private final BigQueryStorageReader reader; private final SerializableFunction parseFn; private final StorageClient storageClient; @@ -226,7 +226,7 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { - // Preconditions.checkStateNotNull(responseIterator); + Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } @@ -246,16 +246,17 @@ private boolean readNextStream() throws IOException { serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( "Started BigQuery Storage API read from stream {}.", - source.streamBundle.get(0).getName()); + source.streamBundle.get(currentStreamIndex).getName()); responseStream = storageClient.readRows(request, source.readSession.getTable()); responseIterator = responseStream.iterator(); return readNextRecord(); } - // @RequiresNonNull("responseIterator") + @RequiresNonNull("responseIterator") private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { + LOG.info("Received null responseIterator for stream {}", currentStreamIndex); return false; } while (reader.readyForNextReadResponse()) { @@ -335,8 +336,8 @@ private boolean readNextRecord() throws IOException { public synchronized void close() { // Because superclass cannot have preconditions around these variables, cannot use // @RequiresNonNull - // Preconditions.checkStateNotNull(storageClient); - // Preconditions.checkStateNotNull(reader); + Preconditions.checkStateNotNull(storageClient); + Preconditions.checkStateNotNull(reader); storageClient.close(); reader.close(); } @@ -346,36 +347,6 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } - // @Override - // @SuppressWarnings("ReturnValueIgnored") - // public @Nullable OffsetBasedSource splitAtFraction(double fraction) { - // int streamCountInBundle = source.streamBundle.size(); - // double splitIndex = streamCountInBundle * fraction; - // if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { - // // The reader has moved past the requested split point. - // // NOTE: We do not split below the granularity of a stream. - // Metrics.counter( - // BigQueryStorageStreamBundleReader.class, - // "split-at-fraction-calls-failed-due-to-impossible-split-point") - // .inc(); - // LOG.info( - // "BigQuery Storage API Session {} cannot be split at {}.", - // source.readSession.getName(), - // fraction); - // return null; - // } - // // Splitting the remainder Streams into a new StreamBundle. - // List remainderStreamBundle = - // new ArrayList<>( - // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); - // // Updating the primary StreamBundle. - // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); - // Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") - // .inc(); - // LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); - // return source.fromExisting(remainderStreamBundle); - // } - @Override public synchronized Double getFractionConsumed() { return fractionConsumed; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index 87dfe1891175..1fba6423b897 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -31,8 +31,6 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; -import com.google.api.gax.grpc.GrpcStatusCode; -import com.google.api.gax.rpc.FailedPreconditionException; import com.google.api.services.bigquery.model.Streamingbuffer; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableFieldSchema; @@ -49,14 +47,9 @@ import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.ReadStream; -import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; -import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; import com.google.cloud.bigquery.storage.v1.StreamStats; import com.google.cloud.bigquery.storage.v1.StreamStats.Progress; import com.google.protobuf.ByteString; -import io.grpc.Status; -import io.grpc.Status.Code; -import io.grpc.StatusRuntimeException; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.math.BigInteger; @@ -1066,15 +1059,19 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.6), createResponse( - AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.6, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( - createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), - 0.50, - 0.75)); + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)), + 0.0, + 1.0), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("G", 7, AVRO_SCHEMA)), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1083,11 +1080,15 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( ReadStream.newBuilder().setName("readStream1").build(), - ReadStream.newBuilder().setName("readStream2").build()); + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( @@ -1102,21 +1103,21 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure the records are returned in - // the prescribed order. + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. BoundedReader primary = primarySource.createReader(options); assertTrue(primary.start()); assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Now split the StreamBundle and ensure that the returned source points to a non-null secondary - // StreamBundle - BoundedSource secondarySource = primary.splitAtFraction(0.5); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primary.advance()); assertEquals("C", primary.getCurrent().get("name")); assertFalse(primary.advance()); @@ -1124,7 +1125,11 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws assertEquals("D", secondary.getCurrent().get("name")); assertTrue(secondary.advance()); assertEquals("E", secondary.getCurrent().get("name")); - assertFalse(secondary.advance()); + assertTrue(secondary.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertTrue((secondary.advance())); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); } @Test @@ -1136,31 +1141,30 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.6), createResponse( - AVRO_SCHEMA, - Lists.newArrayList( - createRecord("C", 3, AVRO_SCHEMA), createRecord("D", 4, AVRO_SCHEMA)), - 0.25, - 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.6, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( - createRecord("E", 5, AVRO_SCHEMA), createRecord("F", 6, AVRO_SCHEMA)), - 0.50, - 0.75)); + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)), + 0.0, + 1.0), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("G", 7, AVRO_SCHEMA)), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); - when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( @@ -1168,7 +1172,7 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() ReadStream.newBuilder().setName("readStream2").build(), ReadStream.newBuilder().setName("readStream3").build()); - BoundedSource primarySource = + BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -1181,39 +1185,44 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - BoundedReader primaryReader = primarySource.createReader(options); - assertTrue(primaryReader.start()); - assertEquals("A", primaryReader.getCurrent().get("name")); + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only + // occur when fraction >= 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); - assertEquals("A", primaryReader.getCurrent().get("name")); + BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primaryReader.advance()); - assertEquals("B", primaryReader.getCurrent().get("name")); - assertFalse(primaryReader.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); - BoundedReader secondaryReader = secondarySource.createReader(options); - assertTrue(secondaryReader.start()); - assertEquals("C", secondaryReader.getCurrent().get("name")); + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); - // Should create two sources: each with 1 stream. - BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle again, and ensure that the returned source points to a non-null + // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only + // occur when fraction >= 0.5. + BoundedSource tertiarySource = secondary.splitAtFraction(0.5); assertNotNull(tertiarySource); - assertEquals("C", secondaryReader.getCurrent().get("name")); - - assertTrue(secondaryReader.advance()); - assertEquals("D", secondaryReader.getCurrent().get("name")); - assertFalse(secondaryReader.advance()); + BoundedReader tertiary = tertiarySource.createReader(options); - BoundedReader tertiaryReader = tertiarySource.createReader(options); - assertTrue(tertiaryReader.start()); - assertEquals("E", tertiaryReader.getCurrent().get("name")); + assertEquals("F", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - assertTrue(tertiaryReader.advance()); - assertEquals("F", tertiaryReader.getCurrent().get("name")); - assertFalse(tertiaryReader.advance()); + assertTrue(tertiary.start()); + assertEquals("G", tertiary.getCurrent().get("name")); + assertFalse((tertiary.advance())); } @Test @@ -1225,15 +1234,15 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() thr Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.66), createResponse( - AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.66, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), - 0.50, - 0.75)); + 0.0, + 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1843,7 +1852,8 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() Lists.newArrayList( createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); + createResponseArrow(ARROW_SCHEMA, names.subList(3, 6), values.subList(3, 6), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1852,11 +1862,15 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( ReadStream.newBuilder().setName("readStream1").build(), - ReadStream.newBuilder().setName("readStream2").build()); + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( @@ -1882,14 +1896,14 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); // Now split the StreamBundle, and ensure that the returned source points to a non-null // secondary StreamBundle. - BoundedSource secondarySource = primary.splitAtFraction(0.5); + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primary.advance()); assertEquals("C", primary.getCurrent().get("name")); assertFalse(primary.advance()); @@ -1897,31 +1911,35 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("D", secondary.getCurrent().get("name")); assertTrue(secondary.advance()); assertEquals("E", secondary.getCurrent().get("name")); - assertFalse(secondary.advance()); + assertTrue(secondary.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertTrue((secondary.advance())); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); } @Test public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsInBundle() throws Exception { - List names = Arrays.asList("A", "B", "C", "D", "E", "F"); - List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); List responses = Lists.newArrayList( - createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.0, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(4, 6), values.subList(4, 6), 0.0, 1.0)); + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 6), values.subList(3, 6), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); - when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( @@ -1929,7 +1947,7 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn ReadStream.newBuilder().setName("readStream2").build(), ReadStream.newBuilder().setName("readStream3").build()); - BoundedSource primarySource = + BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -1946,88 +1964,71 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - BoundedReader primaryReader = primarySource.createReader(options); - assertTrue(primaryReader.start()); - assertEquals("A", primaryReader.getCurrent().get("name")); + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only + // occur when fraction >= 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); - assertEquals("A", primaryReader.getCurrent().get("name")); + BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primaryReader.advance()); - assertEquals("B", primaryReader.getCurrent().get("name")); - assertFalse(primaryReader.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); - BoundedReader secondaryReader = secondarySource.createReader(options); - assertTrue(secondaryReader.start()); - assertEquals("C", secondaryReader.getCurrent().get("name")); + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); - // Should create two sources: each with 1 stream. - BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle again, and ensure that the returned source points to a non-null + // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only + // occur when fraction >= 0.5. + BoundedSource tertiarySource = secondary.splitAtFraction(0.5); assertNotNull(tertiarySource); - assertEquals("C", secondaryReader.getCurrent().get("name")); + BoundedReader tertiary = tertiarySource.createReader(options); - assertTrue(secondaryReader.advance()); - assertEquals("D", secondaryReader.getCurrent().get("name")); - assertFalse(secondaryReader.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - BoundedReader tertiaryReader = tertiarySource.createReader(options); - assertTrue(tertiaryReader.start()); - assertEquals("E", tertiaryReader.getCurrent().get("name")); - - assertTrue(tertiaryReader.advance()); - assertEquals("F", tertiaryReader.getCurrent().get("name")); - assertFalse(tertiaryReader.advance()); + assertTrue(tertiary.start()); + assertEquals("G", tertiary.getCurrent().get("name")); + assertFalse((tertiary.advance())); } @Test public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow() throws Exception { - List names = Arrays.asList("A", "B", "C", "D", "E", "F"); - List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); - List parentResponses = + List names = Arrays.asList("A", "B", "C", "D", "E"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L); + List responses = Lists.newArrayList( - createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), - createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.25, 0.5), - createResponseArrow( - ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.5, 0.75)); + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.66), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.66, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( - ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(parentResponses)); - - // Mocks the split call. A response without a primary_stream and remainder_stream means - // that the split is not possible. - // Mocks the split call. - when(fakeStorageClient.splitReadStream( - SplitReadStreamRequest.newBuilder().setName("parentStream").setFraction(0.5f).build())) - .thenReturn( - SplitReadStreamResponse.newBuilder() - .setPrimaryStream(ReadStream.newBuilder().setName("primaryStream")) - .setRemainderStream(ReadStream.newBuilder().setName("remainderStream")) - .build()); - - // Mocks the ReadRows calls expected on the primary and residual streams. + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( - ReadRowsRequest.newBuilder() - .setReadStream("primaryStream") - // This test will read rows 0 and 1 from the parent before calling split, - // so we expect the primary read to start at offset 2. - .setOffset(2) - .build(), - "")) - .thenThrow( - new FailedPreconditionException( - "Given row offset is invalid for stream.", - new StatusRuntimeException(Status.FAILED_PRECONDITION), - GrpcStatusCode.of(Code.FAILED_PRECONDITION), - /* retryable = */ false)); + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); - List streamBundle = - Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); - BigQueryStorageStreamBundleSource streamSource = + List parentStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamBundleSource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -2037,31 +2038,33 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow( .build()) .setDataFormat(DataFormat.ARROW) .build(), - streamBundle, + parentStreamBundle, TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the parent stream and ensure that records are returned in the - // prescribed order. - BoundedReader parent = streamSource.createReader(options); - assertTrue(parent.start()); - assertEquals("A", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("B", parent.getCurrent().get("name")); - - assertNull(parent.splitAtFraction(0.5)); - - // Verify that the parent source still works okay even after an unsuccessful split attempt. - assertTrue(parent.advance()); - assertEquals("C", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("D", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("E", parent.getCurrent().get("name")); - assertFalse(parent.advance()); + // Read a few records from the parent bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + + // We attempt to split the StreamBundle after starting to read the contents of the second + // stream. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNull(secondarySource); + + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); } @Test From 409afaca573b261b8a3d8677be3e7c4959ebef92 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 3 Feb 2023 01:17:02 +0000 Subject: [PATCH 05/32] Updated BqOption flag. --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java | 6 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java | 6 +++--- .../BigQueryIOStorageReadWithStreamBundleSourceTest.java | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) 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 ffe5465c1fe8..133c42dce664 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 @@ -1410,7 +1410,7 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - if (bqOptions.getUseReadAPIStreamBundleSource()) { + if (bqOptions.getEnableBundling()) { return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 433fb92b3951..5c3f2359c707 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -152,10 +152,10 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); @Description( - "If set, BigQueryIO.Read will default to using the StreamBundle based" + "If set, BigQueryIO.Read will use the StreamBundle based" + "implementation of the Read API Source") @Default.Boolean(false) - Boolean getUseReadAPIStreamBundleSource(); + Boolean getEnableBundling(); - void setUseReadAPIStreamBundleSource(Boolean value); + void setEnableBundling(Boolean value); } 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 af68277fc100..cb72fa34b298 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 @@ -137,7 +137,7 @@ public List> split( // an appropriate number of streams for the Session to produce reasonable throughput. // This is required when using the Read API Source V2. int streamCount = 0; - if (!bqOptions.getUseReadAPIStreamBundleSource()) { + if (!bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); @@ -178,7 +178,7 @@ public List> split( LOG.info( "readSession.getEstimatedTotalBytesScanned(): '{}'", readSession.getEstimatedTotalBytesScanned()); - if (bqOptions.getUseReadAPIStreamBundleSource()) { + if (bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { bytesPerStream = (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); @@ -210,7 +210,7 @@ public List> split( targetTable); // TODO: this is inconsistent with method above, where it can be null TableSchema trimmedSchema = BigQueryAvroUtils.trimBigQueryTableSchema(targetTable.getSchema(), sessionSchema); - if (!bqOptions.getUseReadAPIStreamBundleSource()) { + if (!bqOptions.getEnableBundling()) { List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { sources.add( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index 1fba6423b897..eec0f5104955 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -123,7 +123,7 @@ /** * Tests for {@link BigQueryIO#readTableRows() using {@link Method#DIRECT_READ}} AND {@link - * BigQueryOptions#setUseReadAPIStreamBundleSource(Boolean)} set to True. + * BigQueryOptions#setEnableBundling(Boolean)} (Boolean)} set to True. */ @RunWith(JUnit4.class) public class BigQueryIOStorageReadWithStreamBundleSourceTest { @@ -155,7 +155,7 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); - options.as(BigQueryOptions.class).setUseReadAPIStreamBundleSource(true); + options.as(BigQueryOptions.class).setEnableBundling(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } From ca02997d2b8d22e68500d8271ac422c7682d058e Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 01:43:24 +0000 Subject: [PATCH 06/32] Simplifying `fractionConsumed` calculation. --- .../BigQueryStorageStreamBundleSource.java | 25 ++++++++----------- ...StorageReadWithStreamBundleSourceTest.java | 2 +- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 99f4629db301..c1c8a98f8edb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -172,7 +172,7 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead // Values used for progress reporting. private double fractionConsumed; - private double fractionOfStreamsConsumed; + private double progressAtResponseStart; private double progressAtResponseEnd; private long rowsConsumedFromCurrentResponse; @@ -191,7 +191,6 @@ private BigQueryStorageStreamBundleReader( this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; this.fractionConsumed = 0d; - this.fractionOfStreamsConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -232,7 +231,10 @@ public boolean advanceImpl() throws IOException { } private boolean readNextStream() throws IOException { - BigQueryStorageStreamBundleSource source = getCurrentSource(); + BigQueryStorageStreamBundleSource source; + synchronized (this) { + source = getCurrentSource(); + } if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; return false; @@ -316,19 +318,14 @@ private boolean readNextRecord() throws IOException { // the rows in the current response have been consumed. rowsConsumedFromCurrentResponse++; - fractionConsumed = + double fractionOfCurrentStreamConsumed = progressAtResponseStart - + (progressAtResponseEnd - progressAtResponseStart) - * rowsConsumedFromCurrentResponse - * 1.0 - / totalRowsInCurrentResponse; - + + ((progressAtResponseEnd - progressAtResponseStart) + * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); // Assuming that each stream in the StreamBundle has approximately the same amount of data and - // normalizing the value of fractionConsumed. - fractionConsumed = fractionConsumed / source.streamBundle.size(); - - fractionOfStreamsConsumed = (double) currentStreamIndex / source.streamBundle.size(); - fractionConsumed += fractionOfStreamsConsumed; + // NORMALIZING the value of fractionConsumed. + fractionConsumed = + (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index eec0f5104955..d7ce32326ac5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -932,7 +932,7 @@ public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { List responsesOne = Lists.newArrayList( createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.5), - // Some responses may contain zero results, so we must ensure that we can are resilient + // Some responses may contain zero results, so we must ensure that we are resilient // to such responses. createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.5, 0.5), createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); From a6e2d734cee80427877a70a207bd4b92f5ac9fe8 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 02:22:58 +0000 Subject: [PATCH 07/32] Better variable names. --- .../BigQueryStorageStreamBundleSource.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index c1c8a98f8edb..09e060094584 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -171,7 +171,7 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead private long currentOffset; // Values used for progress reporting. - private double fractionConsumed; + private double fractionOfStreamBundleConsumed; private double progressAtResponseStart; private double progressAtResponseEnd; @@ -190,7 +190,7 @@ private BigQueryStorageStreamBundleReader( this.storageClient = source.bqServices.getStorageClient(options); this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; - this.fractionConsumed = 0d; + this.fractionOfStreamBundleConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -230,13 +230,10 @@ public boolean advanceImpl() throws IOException { return readNextRecord(); } - private boolean readNextStream() throws IOException { - BigQueryStorageStreamBundleSource source; - synchronized (this) { - source = getCurrentSource(); - } + private synchronized boolean readNextStream() throws IOException { + BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamIndex == source.streamBundle.size()) { - fractionConsumed = 1d; + fractionOfStreamBundleConsumed = 1d; return false; } ReadRowsRequest request = @@ -322,9 +319,9 @@ private boolean readNextRecord() throws IOException { progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each stream in the StreamBundle has approximately the same amount of data and - // NORMALIZING the value of fractionConsumed. - fractionConsumed = + // Assuming that each Stream in the StreamBundle has approximately the same amount of data and + // NORMALIZING the value of fractionOfCurrentStreamConsumed. + fractionOfStreamBundleConsumed = (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } @@ -346,7 +343,7 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { @Override public synchronized Double getFractionConsumed() { - return fractionConsumed; + return fractionOfStreamBundleConsumed; } } } From f4677fa2ed26c8e17134b91da562d317ce440e98 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 18:08:06 +0000 Subject: [PATCH 08/32] Minor refactoring. --- .../BigQueryStorageStreamBundleSource.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 09e060094584..09e2b67f946d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -167,8 +167,8 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead private @Nullable BigQueryServerStream responseStream = null; private @Nullable Iterator responseIterator = null; private @Nullable T current = null; - private int currentStreamIndex; - private long currentOffset; + private int currentStreamBundleIndex; + private long currentStreamOffset; // Values used for progress reporting. private double fractionOfStreamBundleConsumed; @@ -189,7 +189,7 @@ private BigQueryStorageStreamBundleReader( this.parseFn = source.parseFn; this.storageClient = source.bqServices.getStorageClient(options); this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); - this.currentStreamIndex = 0; + this.currentStreamBundleIndex = 0; this.fractionOfStreamBundleConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; @@ -207,12 +207,12 @@ public T getCurrent() throws NoSuchElementException { @Override protected long getCurrentOffset() throws NoSuchElementException { - return currentStreamIndex; + return currentStreamBundleIndex; } @Override protected boolean isAtSplitPoint() throws NoSuchElementException { - if (currentOffset == 0) { + if (currentStreamOffset == 0) { return true; } return false; @@ -226,26 +226,26 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { Preconditions.checkStateNotNull(responseIterator); - currentOffset++; + currentStreamOffset++; return readNextRecord(); } private synchronized boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); - if (currentStreamIndex == source.streamBundle.size()) { + if (currentStreamBundleIndex == source.streamBundle.size()) { fractionOfStreamBundleConsumed = 1d; return false; } ReadRowsRequest request = ReadRowsRequest.newBuilder() - .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) - .setOffset(currentOffset) + .setReadStream(source.streamBundle.get(currentStreamBundleIndex).getName()) + .setOffset(currentStreamOffset) .build(); tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( "Started BigQuery Storage API read from stream {}.", - source.streamBundle.get(currentStreamIndex).getName()); + source.streamBundle.get(currentStreamBundleIndex).getName()); responseStream = storageClient.readRows(request, source.readSession.getTable()); responseIterator = responseStream.iterator(); return readNextRecord(); @@ -255,13 +255,13 @@ private synchronized boolean readNextStream() throws IOException { private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { - LOG.info("Received null responseIterator for stream {}", currentStreamIndex); + LOG.info("Received null responseIterator for stream {}", currentStreamBundleIndex); return false; } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { - currentOffset = 0; - currentStreamIndex++; + currentStreamOffset = 0; + currentStreamBundleIndex++; return readNextStream(); } @@ -322,7 +322,7 @@ private boolean readNextRecord() throws IOException { // Assuming that each Stream in the StreamBundle has approximately the same amount of data and // NORMALIZING the value of fractionOfCurrentStreamConsumed. fractionOfStreamBundleConsumed = - (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); + (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } From 139e31768285a294bdf87bf1c43c3125aa059037 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Wed, 8 Feb 2023 20:57:27 +0000 Subject: [PATCH 09/32] Added a synchronized block in readNextRecord(). Also added comments to explain how OffsetBasedSource + RangeTracker is used. --- .../BigQueryStorageStreamBundleSource.java | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 09e2b67f946d..de78b4699716 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -97,6 +97,9 @@ private BigQueryStorageStreamBundleSource( Coder outputCoder, BigQueryServices bqServices, long minBundleSize) { + // The underlying OffsetBasedSource (and RangeTracker) operate only on the StreamBundle and NOT + // the Streams that constitute the StreamBundle. More specifically, the offsets in the + // OffsetBasedSource are indices for the StreamBundle List. super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); @@ -212,6 +215,8 @@ protected long getCurrentOffset() throws NoSuchElementException { @Override protected boolean isAtSplitPoint() throws NoSuchElementException { + // The start of every Stream within a StreamBundle is being defined as a split point. This + // implies that we cannot split below the granularity of a Stream if (currentStreamOffset == 0) { return true; } @@ -226,7 +231,7 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { Preconditions.checkStateNotNull(responseIterator); - currentStreamOffset++; + currentStreamOffset += totalRowsInCurrentResponse; return readNextRecord(); } @@ -239,7 +244,6 @@ private synchronized boolean readNextStream() throws IOException { ReadRowsRequest request = ReadRowsRequest.newBuilder() .setReadStream(source.streamBundle.get(currentStreamBundleIndex).getName()) - .setOffset(currentStreamOffset) .build(); tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); @@ -260,9 +264,11 @@ private boolean readNextRecord() throws IOException { } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { - currentStreamOffset = 0; - currentStreamBundleIndex++; - return readNextStream(); + synchronized (this) { + currentStreamOffset = 0; + currentStreamBundleIndex++; + return readNextStream(); + } } ReadRowsResponse response; @@ -301,7 +307,6 @@ private boolean readNextRecord() throws IOException { 0f <= progressAtResponseEnd && progressAtResponseEnd <= 1f, "Progress at response end (%s) is not in the range [0.0, 1.0].", progressAtResponseEnd); - reader.processReadRowsResponse(response); } @@ -309,18 +314,20 @@ private boolean readNextRecord() throws IOException { current = parseFn.apply(schemaAndRecord); - // Updates the fraction consumed value. This value is calculated by interpolating between - // the fraction consumed value from the previous server response (or zero if we're consuming - // the first response) and the fractional value in the current response based on how many of - // the rows in the current response have been consumed. + // Calculates the fraction of the current stream that has been consumed. This value is + // calculated by interpolating between the fraction consumed value from the previous server + // response (or zero if we're consuming the first response) and the fractional value in the + // current response based on how many of the rows in the current response have been consumed. rowsConsumedFromCurrentResponse++; double fractionOfCurrentStreamConsumed = progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each Stream in the StreamBundle has approximately the same amount of data and - // NORMALIZING the value of fractionOfCurrentStreamConsumed. + // Assuming that each Stream in the StreamBundle has approximately the same amount of data, + // we can use the `fractionOfCurrentStreamConsumed` value to calculate the progress made over + // the + // entire StreamBundle. fractionOfStreamBundleConsumed = (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; From 7e1f974c6266d0eaecf3d096a731e88312b19e23 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 10 Feb 2023 01:27:09 +0000 Subject: [PATCH 10/32] Removed unnecessary synchronized block, added Javadoc and improved unit test coverage. --- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 3 + .../bigquery/BigQueryStorageSourceBase.java | 25 +++--- .../BigQueryStorageStreamBundleSource.java | 46 ++++++++--- ...StorageReadWithStreamBundleSourceTest.java | 76 ++++++++++++------- 4 files changed, 98 insertions(+), 52 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 5c3f2359c707..759aeab52a40 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; @@ -151,6 +153,7 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); + @Experimental(Kind.UNSPECIFIED) @Description( "If set, BigQueryIO.Read will use the StreamBundle based" + "implementation of the Read API Source") 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 cb72fa34b298..636f1fa3f0e5 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 @@ -218,21 +218,20 @@ public List> split( readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); } return ImmutableList.copyOf(sources); - } else { - List streamBundle = Lists.newArrayList(); - List> sources = Lists.newArrayList(); - for (ReadStream readStream : readSession.getStreamsList()) { - streamIndex++; - streamBundle.add(readStream); - if (streamIndex % streamsPerBundle == 0) { - sources.add( - BigQueryStorageStreamBundleSource.create( - readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); - streamBundle = Lists.newArrayList(); - } + } + List streamBundle = Lists.newArrayList(); + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + sources.add( + BigQueryStorageStreamBundleSource.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); + streamBundle = Lists.newArrayList(); } - return ImmutableList.copyOf(sources); } + return ImmutableList.copyOf(sources); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index de78b4699716..85f82e0eafed 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -47,6 +47,34 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * A {@link org.apache.beam.sdk.io.Source} representing a bundle of Streams in a BigQuery ReadAPI + * Session. This Source ONLY supports splitting at the StreamBundle level. + * + *

{@link BigQueryStorageStreamBundleSource} defines a split-point as the starting offset of each + * Stream. As a result, the number of valid split points in the Source is equal to the number of + * Streams in the StreamBundle and this Source does NOT support sub-Stream splitting. + * + *

Additionally, the underlying {@link org.apache.beam.sdk.io.range.OffsetRangeTracker} and + * {@link OffsetBasedSource} operate in the split point space and do NOT directly interact with the + * Streams constituting the StreamBundle. Consequently, fractional values used in + * `splitAtFraction()` are translated into StreamBundleIndices and the underlying RangeTracker + * handles the split operation by checking the validity of the split point. This has the following + * implications for the `splitAtFraction()` operation: + * + * 1. Fraction values that point to the "middle" of a Stream will be translated to the appropriate + * Stream boundary by the RangeTracker. + * + * 2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` calls + * that point to StreamBundleIndices that are greater than the StreamBundleIndex of the current + * Stream

+ * + * @param Type of records represented by the source. + * @see OffsetBasedSource + * @see org.apache.beam.sdk.io.range.OffsetRangeTracker + * @see org.apache.beam.sdk.io.OffsetBasedSource + * (semantically similar to {@link BigQueryStorageStreamBundleSource}) + */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { public static BigQueryStorageStreamBundleSource create( @@ -97,9 +125,6 @@ private BigQueryStorageStreamBundleSource( Coder outputCoder, BigQueryServices bqServices, long minBundleSize) { - // The underlying OffsetBasedSource (and RangeTracker) operate only on the StreamBundle and NOT - // the Streams that constitute the StreamBundle. More specifically, the offsets in the - // OffsetBasedSource are indices for the StreamBundle List. super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); @@ -215,8 +240,6 @@ protected long getCurrentOffset() throws NoSuchElementException { @Override protected boolean isAtSplitPoint() throws NoSuchElementException { - // The start of every Stream within a StreamBundle is being defined as a split point. This - // implies that we cannot split below the granularity of a Stream if (currentStreamOffset == 0) { return true; } @@ -235,7 +258,7 @@ public boolean advanceImpl() throws IOException { return readNextRecord(); } - private synchronized boolean readNextStream() throws IOException { + private boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamBundleIndex == source.streamBundle.size()) { fractionOfStreamBundleConsumed = 1d; @@ -267,8 +290,8 @@ private boolean readNextRecord() throws IOException { synchronized (this) { currentStreamOffset = 0; currentStreamBundleIndex++; - return readNextStream(); } + return readNextStream(); } ReadRowsResponse response; @@ -324,10 +347,11 @@ private boolean readNextRecord() throws IOException { progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each Stream in the StreamBundle has approximately the same amount of data, - // we can use the `fractionOfCurrentStreamConsumed` value to calculate the progress made over - // the - // entire StreamBundle. + + // We now calculate the progress made over the entire StreamBundle by assuming that each + // Stream in the StreamBundle has approximately the same amount of data. Given this, merely + // counting the number of Streams that have been read and linearly interpolating with the + // progress made in the current Stream gives us the overall StreamBundle progress. fractionOfStreamBundleConsumed = (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index d7ce32326ac5..fc1ccd3c8914 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -1103,22 +1103,31 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure that records are returned in the + // Read a few records from the primary Source and ensure that records are returned in the // prescribed order. BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + + // Attempting to split at a sub-Stream level which is NOT supported by the + // `BigQueryStorageStreamBundleSource`. IOTW, since there are exactly 3 Streams in the Source, + // a split will only occur for fraction > 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.05); + assertNull(secondarySource); + assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); - // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary StreamBundle. - BoundedSource secondarySource = primary.splitAtFraction(0.35); + // Now split the primary Source, and ensure that the returned source points to a non-null + // StreamBundle containing Streams 2 & 3. + secondarySource = primary.splitAtFraction(0.5); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertEquals("C", primary.getCurrent().get("name")); + // Since the last two streams were split out the Primary source has been exhausted. assertFalse(primary.advance()); assertTrue(secondary.start()); @@ -1128,6 +1137,12 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws assertTrue(secondary.advance()); assertEquals("F", secondary.getCurrent().get("name")); assertTrue((secondary.advance())); + + // Since we have already started reading from the last Stream in the StreamBundle, splitting + // is now a no-op. + BoundedSource tertiarySource = secondary.splitAtFraction(0.55); + assertNull(tertiarySource); + assertEquals("G", secondary.getCurrent().get("name")); assertFalse((secondary.advance())); } @@ -1185,44 +1200,49 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure that records are returned in the + // Read a few records from the primary Source and ensure that records are returned in the // prescribed order. BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); - // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only - // occur when fraction >= 0.33. - BoundedSource secondarySource = primary.splitAtFraction(0.35); + // Now split the primary Source, and ensure that the returned source points to a non-null + // StreamBundle containing ONLY Stream 3. Since there are exactly 3 Streams in the Source, + // a split will only occur for fraction > 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.7); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - - assertEquals("C", primary.getCurrent().get("name")); - assertFalse(primary.advance()); - assertTrue(secondary.start()); - assertEquals("D", secondary.getCurrent().get("name")); - assertTrue(secondary.advance()); - assertEquals("E", secondary.getCurrent().get("name")); - assertTrue(secondary.advance()); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - // Now split the StreamBundle again, and ensure that the returned source points to a non-null - // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only - // occur when fraction >= 0.5. - BoundedSource tertiarySource = secondary.splitAtFraction(0.5); + // A second splitAtFraction() call on the primary source. The resulting source should + // contain a StreamBundle containing ONLY Stream 2. Since there are 2 Streams in the Source, + // a split will only occur for fraction > 0.50. + BoundedSource tertiarySource = primary.splitAtFraction(0.55); assertNotNull(tertiarySource); BoundedReader tertiary = tertiarySource.createReader(options); - - assertEquals("F", secondary.getCurrent().get("name")); - assertFalse((secondary.advance())); - assertTrue(tertiary.start()); - assertEquals("G", tertiary.getCurrent().get("name")); - assertFalse((tertiary.advance())); + assertEquals("D", tertiary.getCurrent().get("name")); + assertTrue(tertiary.advance()); + assertEquals("E", tertiary.getCurrent().get("name")); + assertTrue(tertiary.advance()); + assertEquals("F", tertiary.getCurrent().get("name")); + assertFalse(tertiary.advance()); + + // A third attempt to split the primary source. This will be ignored since the primary source + // since the Source contains only a single stream now and `BigQueryStorageStreamBundleSource` + // does NOT support sub-stream splitting. + tertiarySource = primary.splitAtFraction(0.9); + assertNull(tertiarySource); + + // All the rows in the primary Source have been read. + assertFalse(primary.advance()); } @Test From cf2cc087ef670f12daa1afcddab690ed6eb8b3f6 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 10 Feb 2023 02:12:28 +0000 Subject: [PATCH 11/32] Consolidated code paths in `BigQueryIO` for Bundled and regular ReadAPI sources. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 499 ++++++++---------- 1 file changed, 224 insertions(+), 275 deletions(-) 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 133c42dce664..d2aac294c3fb 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 @@ -1410,11 +1410,8 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - if (bqOptions.getEnableBundling()) { - return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); - } - PCollectionView jobIdTokenView; + PCollectionTuple tuple; PCollection rows; if (!getWithTemplateCompatibility()) { @@ -1445,108 +1442,34 @@ public String apply(String input) { jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); TupleTag readStreamsTag = new TupleTag<>(); + TupleTag> listReadStreamsTag = new TupleTag<>(); TupleTag readSessionTag = new TupleTag<>(); TupleTag tableSchemaTag = new TupleTag<>(); - PCollectionTuple tuple = - jobIdTokenCollection.apply( - "RunQueryJob", - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - BigQueryOptions options = - c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.element(); - // Execute the query and get the destination table holding the results. - // The getTargetTable call runs a new instance of the query and returns - // the destination table created to hold the results. - BigQueryStorageQuerySource querySource = - createStorageQuerySource(jobUuid, outputCoder); - Table queryResultTable = querySource.getTargetTable(options); - - // Create a read session without specifying a desired stream count and - // let the BigQuery storage server pick the number of streams. - CreateReadSessionRequest request = - CreateReadSessionRequest.newBuilder() - .setParent( - BigQueryHelpers.toProjectResourceName( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject())) - .setReadSession( - ReadSession.newBuilder() - .setTable( - BigQueryHelpers.toTableResourceName( - queryResultTable.getTableReference())) - .setDataFormat(DataFormat.AVRO)) - .setMaxStreamCount(0) - .build(); - - ReadSession readSession; - try (StorageClient storageClient = - getBigQueryServices().getStorageClient(options)) { - readSession = storageClient.createReadSession(request); - } - - for (ReadStream readStream : readSession.getStreamsList()) { - c.output(readStream); - } - - c.output(readSessionTag, readSession); - c.output( - tableSchemaTag, - BigQueryHelpers.toJsonString(queryResultTable.getSchema())); - } - }) - .withOutputTags( - readStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + if (!bqOptions.getEnableBundling()) { + tuple = createTupleForDirectRead(jobIdTokenCollection, outputCoder, readStreamsTag, + readSessionTag, tableSchemaTag); + tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); + } else { + tuple = createTupleForDirectReadWithStreamBundle(jobIdTokenCollection, outputCoder, + listReadStreamsTag, readSessionTag, tableSchemaTag); + tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); + } - tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - PCollectionView readSessionView = tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); PCollectionView tableSchemaView = tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - rows = - tuple - .get(readStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - ReadStream readStream = c.element(); - - BigQueryStorageStreamSource streamSource = - BigQueryStorageStreamSource.create( - readSession, - readStream, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices()); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedSource.BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + if (!bqOptions.getEnableBundling()) { + rows = createPCollectionForDirectRead(tuple, outputCoder, readStreamsTag, readSessionView, + tableSchemaView); + } else { + rows = createPCollectionForDirectReadWithStreamBundle(tuple, outputCoder, + listReadStreamsTag, readSessionView, tableSchemaView); + } } PassThroughThenCleanup.CleanupOperation cleanupOperation = @@ -1575,8 +1498,10 @@ void cleanup(ContextContainer c) throws Exception { boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); if (datasetCreatedByBeam) { LOG.info( - "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + "Deleting temporary dataset with query results {}", + tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), + tempTable.getDatasetId()); } } } @@ -1592,194 +1517,218 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - private PCollection expandAnonForDirectReadWithStreamBundle( - PBegin input, Coder outputCoder, Schema beamSchema) { - - Pipeline p = input.getPipeline(); - PCollectionView jobIdTokenView; - PCollection rows; - - if (!getWithTemplateCompatibility()) { - // Create a singleton job ID token at pipeline construction time. - String staticJobUuid = BigQueryHelpers.randomUUIDString(); - jobIdTokenView = - p.apply("TriggerIdCreation", Create.of(staticJobUuid)) - .apply("ViewId", View.asSingleton()); - // Apply the traditional Source model. - rows = - p.apply( - org.apache.beam.sdk.io.Read.from( - createStorageQuerySource(staticJobUuid, outputCoder))); - } else { - // Create a singleton job ID token at pipeline execution time. - PCollection jobIdTokenCollection = - p.apply("TriggerIdCreation", Create.of("ignored")) - .apply( - "CreateJobId", - MapElements.via( - new SimpleFunction() { - @Override - public String apply(String input) { - return BigQueryHelpers.randomUUIDString(); + private PCollectionTuple createTupleForDirectRead(PCollection jobIdTokenCollection, + Coder outputCoder, TupleTag readStreamsTag, TupleTag + readSessionTag, TupleTag tableSchemaTag) { + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); } - })); - - jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); - TupleTag> listReadStreamsTag = new TupleTag<>(); - TupleTag readSessionTag = new TupleTag<>(); - TupleTag tableSchemaTag = new TupleTag<>(); + for (ReadStream readStream : readSession.getStreamsList()) { + c.output(readStream); + } - PCollectionTuple tuple = - jobIdTokenCollection.apply( - "RunQueryJob", - ParDo.of( - new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - BigQueryOptions options = - c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.element(); - // Execute the query and get the destination table holding the results. - // The getTargetTable call runs a new instance of the query and returns - // the destination table created to hold the results. - BigQueryStorageQuerySource querySource = - createStorageQuerySource(jobUuid, outputCoder); - Table queryResultTable = querySource.getTargetTable(options); - - // Create a read session without specifying a desired stream count and - // let the BigQuery storage server pick the number of streams. - CreateReadSessionRequest request = - CreateReadSessionRequest.newBuilder() - .setParent( - BigQueryHelpers.toProjectResourceName( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject())) - .setReadSession( - ReadSession.newBuilder() - .setTable( - BigQueryHelpers.toTableResourceName( - queryResultTable.getTableReference())) - .setDataFormat(DataFormat.AVRO)) - .setMaxStreamCount(0) - .build(); - - ReadSession readSession; - try (StorageClient storageClient = - getBigQueryServices().getStorageClient(options)) { - readSession = storageClient.createReadSession(request); - } - int streamIndex = 0; - int streamsPerBundle = 10; - List streamBundle = Lists.newArrayList(); - for (ReadStream readStream : readSession.getStreamsList()) { - streamIndex++; - streamBundle.add(readStream); - if (streamIndex % streamsPerBundle == 0) { - c.output(streamBundle); - streamBundle = Lists.newArrayList(); - } + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + readStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + return tuple; + } + + private PCollectionTuple createTupleForDirectReadWithStreamBundle(PCollection + jobIdTokenCollection, + Coder outputCoder, TupleTag> listReadStreamsTag, + TupleTag readSessionTag, TupleTag tableSchemaTag) { + + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); + } + int streamIndex = 0; + int streamsPerBundle = 10; + List streamBundle = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + c.output(streamBundle); + streamBundle = Lists.newArrayList(); } - - c.output(readSessionTag, readSession); - c.output( - tableSchemaTag, - BigQueryHelpers.toJsonString(queryResultTable.getSchema())); } - }) - .withOutputTags( - listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); - - tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); - tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); - tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - - PCollectionView readSessionView = - tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); - PCollectionView tableSchemaView = - tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - rows = - tuple - .get(listReadStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - List streamBundle = c.element(); - - BigQueryStorageStreamBundleSource streamSource = - BigQueryStorageStreamBundleSource.create( - readSession, - streamBundle, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices(), - 1L); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + return tuple; + } + + private PCollection createPCollectionForDirectRead(PCollectionTuple tuple, + Coder outputCoder, TupleTag readStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + PCollection rows = + tuple + .get(readStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + ReadStream readStream = c.element(); + + BigQueryStorageStreamSource streamSource = + BigQueryStorageStreamSource.create( + readSession, + readStream, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedSource.BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); - } - - CleanupOperation cleanupOperation = - new CleanupOperation() { - @Override - void cleanup(ContextContainer c) throws Exception { - BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.getJobId(); + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); - Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); - - TableReference tempTable = - createTempTableReference( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject(), - BigQueryResourceNaming.createJobIdPrefix( - options.getJobName(), jobUuid, JobType.QUERY), - queryTempDataset); + return rows; + } - try (DatasetService datasetService = - getBigQueryServices().getDatasetService(options)) { - LOG.info("Deleting temporary table with query results {}", tempTable); - datasetService.deleteTable(tempTable); - // Delete dataset only if it was created by Beam - boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); - if (datasetCreatedByBeam) { - LOG.info( - "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); - } - } - } - }; + private PCollection createPCollectionForDirectReadWithStreamBundle(PCollectionTuple tuple, + Coder outputCoder, TupleTag> listReadStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + PCollection rows = + tuple + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices(), + 1L); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); - if (beamSchema != null) { - rows.setSchema( - beamSchema, - getTypeDescriptor(), - getToBeamRowFn().apply(beamSchema), - getFromBeamRowFn().apply(beamSchema)); - } - return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + return rows; } @Override From 54e80ae6c7613de3a5b7fadba25946d1f2284d28 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 13 Feb 2023 21:49:27 +0000 Subject: [PATCH 12/32] Lint fixes. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 209 ++++++++++-------- .../BigQueryStorageStreamBundleSource.java | 14 +- 2 files changed, 123 insertions(+), 100 deletions(-) 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 d2aac294c3fb..704324130e89 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 @@ -1447,12 +1447,22 @@ public String apply(String input) { TupleTag tableSchemaTag = new TupleTag<>(); if (!bqOptions.getEnableBundling()) { - tuple = createTupleForDirectRead(jobIdTokenCollection, outputCoder, readStreamsTag, - readSessionTag, tableSchemaTag); + tuple = + createTupleForDirectRead( + jobIdTokenCollection, + outputCoder, + readStreamsTag, + readSessionTag, + tableSchemaTag); tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); } else { - tuple = createTupleForDirectReadWithStreamBundle(jobIdTokenCollection, outputCoder, - listReadStreamsTag, readSessionTag, tableSchemaTag); + tuple = + createTupleForDirectReadWithStreamBundle( + jobIdTokenCollection, + outputCoder, + listReadStreamsTag, + readSessionTag, + tableSchemaTag); tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); } @@ -1464,11 +1474,13 @@ public String apply(String input) { tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); if (!bqOptions.getEnableBundling()) { - rows = createPCollectionForDirectRead(tuple, outputCoder, readStreamsTag, readSessionView, - tableSchemaView); + rows = + createPCollectionForDirectRead( + tuple, outputCoder, readStreamsTag, readSessionView, tableSchemaView); } else { - rows = createPCollectionForDirectReadWithStreamBundle(tuple, outputCoder, - listReadStreamsTag, readSessionView, tableSchemaView); + rows = + createPCollectionForDirectReadWithStreamBundle( + tuple, outputCoder, listReadStreamsTag, readSessionView, tableSchemaView); } } @@ -1498,10 +1510,8 @@ void cleanup(ContextContainer c) throws Exception { boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); if (datasetCreatedByBeam) { LOG.info( - "Deleting temporary dataset with query results {}", - tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), - tempTable.getDatasetId()); + "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); } } } @@ -1517,9 +1527,12 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - private PCollectionTuple createTupleForDirectRead(PCollection jobIdTokenCollection, - Coder outputCoder, TupleTag readStreamsTag, TupleTag - readSessionTag, TupleTag tableSchemaTag) { + private PCollectionTuple createTupleForDirectRead( + PCollection jobIdTokenCollection, + Coder outputCoder, + TupleTag readStreamsTag, + TupleTag readSessionTag, + TupleTag tableSchemaTag) { PCollectionTuple tuple = jobIdTokenCollection.apply( "RunQueryJob", @@ -1577,10 +1590,12 @@ public void processElement(ProcessContext c) throws Exception { return tuple; } - private PCollectionTuple createTupleForDirectReadWithStreamBundle(PCollection - jobIdTokenCollection, - Coder outputCoder, TupleTag> listReadStreamsTag, - TupleTag readSessionTag, TupleTag tableSchemaTag) { + private PCollectionTuple createTupleForDirectReadWithStreamBundle( + PCollection jobIdTokenCollection, + Coder outputCoder, + TupleTag> listReadStreamsTag, + TupleTag readSessionTag, + TupleTag tableSchemaTag) { PCollectionTuple tuple = jobIdTokenCollection.apply( @@ -1646,87 +1661,95 @@ public void processElement(ProcessContext c) throws Exception { return tuple; } - private PCollection createPCollectionForDirectRead(PCollectionTuple tuple, - Coder outputCoder, TupleTag readStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + private PCollection createPCollectionForDirectRead( + PCollectionTuple tuple, + Coder outputCoder, + TupleTag readStreamsTag, + PCollectionView readSessionView, + PCollectionView tableSchemaView) { PCollection rows = - tuple - .get(readStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - ReadStream readStream = c.element(); - - BigQueryStorageStreamSource streamSource = - BigQueryStorageStreamSource.create( - readSession, - readStream, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices()); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedSource.BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + tuple + .get(readStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + ReadStream readStream = c.element(); + + BigQueryStorageStreamSource streamSource = + BigQueryStorageStreamSource.create( + readSession, + readStream, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedSource.BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); return rows; } - private PCollection createPCollectionForDirectReadWithStreamBundle(PCollectionTuple tuple, - Coder outputCoder, TupleTag> listReadStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + private PCollection createPCollectionForDirectReadWithStreamBundle( + PCollectionTuple tuple, + Coder outputCoder, + TupleTag> listReadStreamsTag, + PCollectionView readSessionView, + PCollectionView tableSchemaView) { PCollection rows = tuple - .get(listReadStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - List streamBundle = c.element(); - - BigQueryStorageStreamBundleSource streamSource = - BigQueryStorageStreamBundleSource.create( - readSession, - streamBundle, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices(), - 1L); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices(), + 1L); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); return rows; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 85f82e0eafed..849b3ea395a9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -62,18 +62,18 @@ * handles the split operation by checking the validity of the split point. This has the following * implications for the `splitAtFraction()` operation: * - * 1. Fraction values that point to the "middle" of a Stream will be translated to the appropriate - * Stream boundary by the RangeTracker. + *

1. Fraction values that point to the "middle" of a Stream will be translated to the + * appropriate Stream boundary by the RangeTracker. * - * 2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` calls - * that point to StreamBundleIndices that are greater than the StreamBundleIndex of the current - * Stream

+ *

2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` + * calls that point to StreamBundleIndices that are greater than the StreamBundleIndex of the + * current Stream * * @param Type of records represented by the source. * @see OffsetBasedSource * @see org.apache.beam.sdk.io.range.OffsetRangeTracker - * @see org.apache.beam.sdk.io.OffsetBasedSource - * (semantically similar to {@link BigQueryStorageStreamBundleSource}) + * @see org.apache.beam.sdk.io.OffsetBasedSource (semantically similar to {@link + * BigQueryStorageStreamBundleSource}) */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { From dac8cfe8833183237054972ae2667281f944e05e Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 13 Feb 2023 21:54:50 +0000 Subject: [PATCH 13/32] Minor Javadoc fix. --- .../sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 849b3ea395a9..f1a414d39f4f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -72,7 +72,7 @@ * @param Type of records represented by the source. * @see OffsetBasedSource * @see org.apache.beam.sdk.io.range.OffsetRangeTracker - * @see org.apache.beam.sdk.io.OffsetBasedSource (semantically similar to {@link + * @see org.apache.beam.sdk.io.BlockBasedSource (semantically similar to {@link * BigQueryStorageStreamBundleSource}) */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { From d9023269d17ce5ef335bab8b49fe4a36d8fb28ad Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 22:54:56 +0000 Subject: [PATCH 14/32] Fix StreamBundle creation logic and some minor code comment updates. --- .../apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 +++- .../io/gcp/bigquery/BigQueryStorageSourceBase.java | 11 ++++++++--- .../bigquery/BigQueryStorageStreamBundleSource.java | 6 ++++-- 3 files changed, 15 insertions(+), 6 deletions(-) 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 704324130e89..7f4297da5d1a 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 @@ -1648,7 +1648,9 @@ public void processElement(ProcessContext c) throws Exception { streamBundle = Lists.newArrayList(); } } - + if(streamIndex % streamsPerBundle != 0) { + c.output(streamBundle); + } c.output(readSessionTag, readSession); c.output( tableSchemaTag, 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 636f1fa3f0e5..8f0afe2bf9ed 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 @@ -176,19 +176,19 @@ public List> split( int streamsPerBundle = 0; double bytesPerStream = 0; LOG.info( - "readSession.getEstimatedTotalBytesScanned(): '{}'", + "Estimated bytes this ReadSession will scan when all Streams are consumed: '{}'", readSession.getEstimatedTotalBytesScanned()); if (bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { bytesPerStream = (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); - LOG.info("bytesPerStream: '{}'", bytesPerStream); + LOG.info("Estimated bytes each Stream will consume: '{}'", bytesPerStream); streamsPerBundle = (int) Math.ceil(desiredBundleSizeBytes / bytesPerStream); } else { streamsPerBundle = (int) Math.ceil((double) streamCount / 10); } streamsPerBundle = Math.min(streamCount, streamsPerBundle); - LOG.info("streamsPerBundle: '{}'", streamsPerBundle); + LOG.info("Distributing '{}' Streams per StreamBundle.", streamsPerBundle); } Schema sessionSchema; @@ -231,6 +231,11 @@ public List> split( streamBundle = Lists.newArrayList(); } } + if (streamIndex % streamsPerBundle != 0) { + sources.add( + BigQueryStorageStreamBundleSource.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); + } return ImmutableList.copyOf(sources); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index f1a414d39f4f..00c6c57025e8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.slf4j.Logger; @@ -160,8 +161,9 @@ public long getEstimatedSizeBytes(PipelineOptions options) { @Override public List> split( long desiredBundleSizeBytes, PipelineOptions options) { - // A stream source can't be split without reading from it due to server-side liquid sharding. - // TODO: Implement dynamic work rebalancing. + // This method is only called for initial splits. Since this class will always be a child source + // of BigQueryStorageSourceBase, all splits here will be handled by `splitAtFraction()`. As a + // result, this is a no-op. return ImmutableList.of(this); } From b45bdd7d39c22afca3ae7fc414caa7c5b7ad67ce Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 22:57:44 +0000 Subject: [PATCH 15/32] Updated logging. --- .../sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 8f0afe2bf9ed..64eb1b90d629 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 @@ -161,10 +161,9 @@ public List> split( try (StorageClient client = bqServices.getStorageClient(bqOptions)) { readSession = client.createReadSession(createReadSessionRequest); LOG.info( - "Sent BigQuery Storage API CreateReadSession request in code '{}'.", - createReadSessionRequest); - LOG.info( - "Received number of streams in response: '{}'.", readSession.getStreamsList().size()); + "Sent BigQuery Storage API CreateReadSession request '{}'; received response '{}'.", + createReadSessionRequest, + readSession); } if (readSession.getStreamsList().isEmpty()) { From 323a0809c1be23c194c535ca1659bae8ee55f6a0 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 23:58:50 +0000 Subject: [PATCH 16/32] Lint fixes. --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) 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 7f4297da5d1a..8a396c251c40 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 @@ -1648,7 +1648,7 @@ public void processElement(ProcessContext c) throws Exception { streamBundle = Lists.newArrayList(); } } - if(streamIndex % streamsPerBundle != 0) { + if (streamIndex % streamsPerBundle != 0) { c.output(streamBundle); } c.output(readSessionTag, readSession); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 00c6c57025e8..42e99b6aae38 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.slf4j.Logger; From 73c65284041a55abc1c3fad98653d7126156cbe9 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 5 Dec 2022 18:20:50 +0000 Subject: [PATCH 17/32] ReadAPI Source v2 --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 205 ++++++++++- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 6 + .../bigquery/BigQueryStorageSourceBase.java | 62 +++- .../bigquery/BigQueryStorageSourceBaseV2.java | 5 + .../BigQueryStorageStreamSourceV2.java | 346 ++++++++++++++++++ .../bigquery/BigQueryIOStorageReadTest.java | 1 + 6 files changed, 608 insertions(+), 17 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java 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 04a81821d8e9..9d89687d2de0 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 @@ -75,7 +75,9 @@ import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.fs.MoveOptions; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; @@ -1179,11 +1181,11 @@ public PCollection expand(PBegin input) { // if both toRowFn and fromRowFn values are set, enable Beam schema support Pipeline p = input.getPipeline(); + BigQueryOptions bqOptions = p.getOptions().as(BigQueryOptions.class); final BigQuerySourceDef sourceDef = createSourceDef(); Schema beamSchema = null; if (getTypeDescriptor() != null && getToBeamRowFn() != null && getFromBeamRowFn() != null) { - BigQueryOptions bqOptions = p.getOptions().as(BigQueryOptions.class); beamSchema = sourceDef.getBeamSchema(bqOptions); beamSchema = getFinalSchema(beamSchema, getSelectedFields()); } @@ -1191,7 +1193,7 @@ public PCollection expand(PBegin input) { final Coder coder = inferCoder(p.getCoderRegistry()); if (getMethod() == TypedRead.Method.DIRECT_READ) { - return expandForDirectRead(input, coder, beamSchema); + return expandForDirectRead(input, coder, beamSchema, bqOptions); } checkArgument( @@ -1369,7 +1371,7 @@ private static Schema getFinalSchema( } private PCollection expandForDirectRead( - PBegin input, Coder outputCoder, Schema beamSchema) { + PBegin input, Coder outputCoder, Schema beamSchema, BigQueryOptions bqOptions) { ValueProvider tableProvider = getTableProvider(); Pipeline p = input.getPipeline(); if (tableProvider != null) { @@ -1415,6 +1417,14 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // + // if (bqOptions.getUseReadAPISourceV2()) { + // return expandAnonForDirectReadV2(input, outputCoder, beamSchema); + // } + + if(bqOptions.getUseReadAPISourceV2() || !bqOptions.getUseReadAPISourceV2()) { + + } + PCollectionView jobIdTokenView; PCollection rows; @@ -1593,6 +1603,195 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } + // private PCollection expandAnonForDirectReadV2( + // PBegin input, Coder outputCoder, Schema beamSchema) { + // Pipeline p = input.getPipeline(); + // PCollectionView jobIdTokenView; + // PCollection rows; + // + // if (!getWithTemplateCompatibility()) { + // // Create a singleton job ID token at pipeline construction time. + // String staticJobUuid = BigQueryHelpers.randomUUIDString(); + // jobIdTokenView = + // p.apply("TriggerIdCreation", Create.of(staticJobUuid)) + // .apply("ViewId", View.asSingleton()); + // // Apply the traditional Source model. + // rows = + // p.apply( + // org.apache.beam.sdk.io.Read.from( + // createStorageQuerySource(staticJobUuid, outputCoder))); + // } else { + // // Create a singleton job ID token at pipeline execution time. + // PCollection jobIdTokenCollection = + // p.apply("TriggerIdCreation", Create.of("ignored")) + // .apply( + // "CreateJobId", + // MapElements.via( + // new SimpleFunction() { + // @Override + // public String apply(String input) { + // return BigQueryHelpers.randomUUIDString(); + // } + // })); + // + // jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); + // + // TupleTag> listReadStreamsTag = new TupleTag<>(); + // TupleTag readSessionTag = new TupleTag<>(); + // TupleTag tableSchemaTag = new TupleTag<>(); + // + // PCollectionTuple tuple = + // jobIdTokenCollection.apply( + // "RunQueryJob", + // ParDo.of( + // new DoFn>() { + // @ProcessElement + // public void processElement(ProcessContext c) throws Exception { + // BigQueryOptions options = + // c.getPipelineOptions().as(BigQueryOptions.class); + // String jobUuid = c.element(); + // // Execute the query and get the destination table holding the results. + // // The getTargetTable call runs a new instance of the query and returns + // // the destination table created to hold the results. + // BigQueryStorageQuerySource querySource = + // createStorageQuerySource(jobUuid, outputCoder); + // Table queryResultTable = querySource.getTargetTable(options); + // + // // Create a read session without specifying a desired stream count and + // // let the BigQuery storage server pick the number of streams. + // CreateReadSessionRequest request = + // CreateReadSessionRequest.newBuilder() + // .setParent( + // BigQueryHelpers.toProjectResourceName( + // options.getBigQueryProject() == null + // ? options.getProject() + // : options.getBigQueryProject())) + // .setReadSession( + // ReadSession.newBuilder() + // .setTable( + // BigQueryHelpers.toTableResourceName( + // queryResultTable.getTableReference())) + // .setDataFormat(DataFormat.AVRO)) + // .setMaxStreamCount(0) + // .build(); + // + // ReadSession readSession; + // try (StorageClient storageClient = + // getBigQueryServices().getStorageClient(options)) { + // readSession = storageClient.createReadSession(request); + // } + // int streamIndex = 0; + // int streamsPerBundle = 10; + // List streamBundle = Lists.newArrayList(); + // for (ReadStream readStream : readSession.getStreamsList()) { + // streamIndex++; + // streamBundle.add(readStream); + // if (streamIndex % streamsPerBundle == 0) { + // c.output(streamBundle); + // streamBundle = Lists.newArrayList(); + // } + // } + // + // c.output(readSessionTag, readSession); + // c.output( + // tableSchemaTag, + // BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + // } + // }) + // .withOutputTags( + // listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + // + // TypeDescriptor> listReadStreamTypeDescriptor = new TypeDescriptor>() {}; + // tuple.get(listReadStreamsTag).setCoder(ProtoCoder.of(listReadStreamTypeDescriptor)); + // tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); + // tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); + // + // PCollectionView readSessionView = + // tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); + // PCollectionView tableSchemaView = + // tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); + // + // rows = + // tuple + // .get(listReadStreamsTag) + // .apply(Reshuffle.viaRandomKey()) + // .apply( + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) throws Exception { + // ReadSession readSession = c.sideInput(readSessionView); + // TableSchema tableSchema = + // BigQueryHelpers.fromJsonString( + // c.sideInput(tableSchemaView), TableSchema.class); + // List streamBundle = c.element(); + // + // BigQueryStorageStreamSourceV2 streamSource = + // BigQueryStorageStreamSourceV2.create( + // readSession, + // streamBundle, + // tableSchema, + // getParseFn(), + // outputCoder, + // getBigQueryServices()); + // + // // Read all of the data from the stream. In the event that this work + // // item fails and is rescheduled, the same rows will be returned in + // // the same order. + // BoundedReader reader = + // streamSource.createReader(c.getPipelineOptions()); + // for (boolean more = reader.start(); more; more = reader.advance()) { + // c.output(reader.getCurrent()); + // } + // } + // }) + // .withSideInputs(readSessionView, tableSchemaView)) + // .setCoder(outputCoder); + // } + // + // CleanupOperation cleanupOperation = + // new CleanupOperation() { + // @Override + // void cleanup(ContextContainer c) throws Exception { + // BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); + // String jobUuid = c.getJobId(); + // + // Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); + // + // TableReference tempTable = + // createTempTableReference( + // options.getBigQueryProject() == null + // ? options.getProject() + // : options.getBigQueryProject(), + // BigQueryResourceNaming.createJobIdPrefix( + // options.getJobName(), jobUuid, JobType.QUERY), + // queryTempDataset); + // + // try (DatasetService datasetService = + // getBigQueryServices().getDatasetService(options)) { + // LOG.info("Deleting temporary table with query results {}", tempTable); + // datasetService.deleteTable(tempTable); + // // Delete dataset only if it was created by Beam + // boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); + // if (datasetCreatedByBeam) { + // LOG.info( + // "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + // datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + // } + // } + // } + // }; + // + // if (beamSchema != null) { + // rows.setSchema( + // beamSchema, + // getTypeDescriptor(), + // getToBeamRowFn().apply(beamSchema), + // getFromBeamRowFn().apply(beamSchema)); + // } + // return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + // } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index bf09bf4d9e37..3003666d19be 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -163,4 +163,10 @@ public interface BigQueryOptions Long getStorageWriteApiMaxRequestSize(); void setStorageWriteApiMaxRequestSize(Long value); + + @Description("If set, BigQueryIO.Read will default to using the V2 implementation of the Read API Source") + @Default.Boolean(false) + Boolean getUseReadAPISourceV2(); + + void setUseReadAPISourceV2(Boolean value); } 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 27b88dc39600..69c47ea7c547 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 @@ -100,7 +100,7 @@ public Coder getOutputCoder() { } @Override - public List> split( + public List> split( long desiredBundleSizeBytes, PipelineOptions options) throws Exception { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); @Nullable Table targetTable = getTargetTable(bqOptions); @@ -133,13 +133,18 @@ public List> split( readSessionBuilder.setDataFormat(format); } + // Setting the requested max stream count to 0, implies that the Read API backend will select + // an appropriate number of streams for the Session to produce reasonable throughput. + // This is required when using the Read API Source V2. int streamCount = 0; - if (desiredBundleSizeBytes > 0) { - long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; - streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); - } + if(!bqOptions.getUseReadAPISourceV2()) { + if (desiredBundleSizeBytes > 0) { + long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; + streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); + } - streamCount = Math.max(streamCount, MIN_SPLIT_COUNT); + streamCount = Math.max(streamCount, MIN_SPLIT_COUNT); + } CreateReadSessionRequest createReadSessionRequest = CreateReadSessionRequest.newBuilder() @@ -166,6 +171,20 @@ public List> split( return ImmutableList.of(); } + int streamsPerBundle = 0; + if (bqOptions.getUseReadAPISourceV2()) { + if (desiredBundleSizeBytes > 0) { + streamsPerBundle = + (int) + Math.ceil((double) + desiredBundleSizeBytes + / (double) (readSession.getEstimatedTotalBytesScanned() + / readSession.getStreamsList().size())); + } else { + streamsPerBundle = (int) Math.ceil((double) streamCount / 10); + } + } + Schema sessionSchema; if (readSession.getDataFormat() == DataFormat.ARROW) { org.apache.arrow.vector.types.pojo.Schema schema = @@ -180,19 +199,34 @@ public List> split( throw new IllegalArgumentException( "data is not in a supported dataFormat: " + readSession.getDataFormat()); } - + int streamIndex = 0; 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(); - for (ReadStream readStream : readSession.getStreamsList()) { - sources.add( - BigQueryStorageStreamSource.create( - readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); + if(!bqOptions.getUseReadAPISourceV2()) { + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + sources.add( + BigQueryStorageStreamSource.create( + readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); + } + return ImmutableList.copyOf(sources); + } else { + List streamBundle = Lists.newArrayList(); + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + sources.add( + BigQueryStorageStreamSourceV2.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); + streamBundle = Lists.newArrayList(); + } + } + return ImmutableList.copyOf(sources); } - - return ImmutableList.copyOf(sources); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java new file mode 100644 index 000000000000..a0ba4e22d1e3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java @@ -0,0 +1,5 @@ +package org.apache.beam.sdk.io.gcp.bigquery; + +public class BigQueryStorageSourceBaseV2 { + +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java new file mode 100644 index 000000000000..b4028996a0ef --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +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.checkArgument; + +import com.google.api.gax.rpc.ApiException; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.beam.runners.core.metrics.ServiceCallMetric; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; +import org.apache.beam.sdk.metrics.Metrics; +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.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; + +class BigQueryStorageStreamSourceV2 extends BoundedSource { + + private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSourceV2.class); + + public static BigQueryStorageStreamSourceV2 create( + ReadSession readSession, + List streamBundle, + TableSchema tableSchema, + SerializableFunction parseFn, + Coder outputCoder, + BigQueryServices bqServices) { + return new BigQueryStorageStreamSourceV2<>( + readSession, + streamBundle, + toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), + parseFn, + outputCoder, + bqServices); + } + + /** + * Creates a new source with the same properties as this one, except with a different {@link + * List}. + */ + public BigQueryStorageStreamSourceV2 fromExisting(List newStreamBundle) { + return new BigQueryStorageStreamSourceV2<>( + readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); + } + + private final ReadSession readSession; + private final List streamBundle; + private final String jsonTableSchema; + private final SerializableFunction parseFn; + private final Coder outputCoder; + private final BigQueryServices bqServices; + + private BigQueryStorageStreamSourceV2( + ReadSession readSession, + List streamBundle, + String jsonTableSchema, + SerializableFunction parseFn, + Coder outputCoder, + BigQueryServices bqServices) { + this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); + this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); + 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 + public Coder getOutputCoder() { + return outputCoder; + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .add(DisplayData.item("table", readSession.getTable()).withLabel("Table")) + .add(DisplayData.item("readSession", readSession.getName()).withLabel("Read session")); + for (ReadStream readStream : streamBundle) { + builder.add(DisplayData.item("stream", readStream.getName()).withLabel("Stream")); + } + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) { + // The size of stream source can't be estimated due to server-side liquid sharding. + // TODO: Implement progress reporting. + return 0L; + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) { + // A stream source can't be split without reading from it due to server-side liquid sharding. + // TODO: Implement dynamic work rebalancing. + return ImmutableList.of(this); + } + + @Override + public BigQueryStorageStreamBundleReader createReader(PipelineOptions options) + throws IOException { + return new BigQueryStorageStreamBundleReader<>(this, options.as(BigQueryOptions.class)); + } + + // @Override + // public String toString() { + // return readStream.toString(); + // } + + public static class BigQueryStorageStreamBundleReader extends BoundedSource.BoundedReader { + private final BigQueryStorageReader reader; + private final SerializableFunction parseFn; + private final StorageClient storageClient; + private final TableSchema tableSchema; + + private BigQueryStorageStreamSourceV2 source; + private @Nullable BigQueryServerStream responseStream = null; + private @Nullable Iterator responseIterator = null; + private @Nullable T current = null; + private int currentStreamIndex; + private long currentOffset; + + // Values used for progress reporting. + private double fractionConsumed; + private double progressAtResponseStart; + private double progressAtResponseEnd; + private long rowsConsumedFromCurrentResponse; + private long totalRowsInCurrentResponse; + + private @Nullable TableReference tableReference; + private @Nullable ServiceCallMetric serviceCallMetric; + + private BigQueryStorageStreamBundleReader( + BigQueryStorageStreamSourceV2 source, BigQueryOptions options) throws IOException { + this.source = source; + this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); + this.parseFn = source.parseFn; + this.storageClient = source.bqServices.getStorageClient(options); + this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); + this.currentStreamIndex = 0; + this.fractionConsumed = 0d; + this.progressAtResponseStart = 0d; + this.progressAtResponseEnd = 0d; + this.rowsConsumedFromCurrentResponse = 0L; + this.totalRowsInCurrentResponse = 0L; + } + + @Override + public T getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current; + } + + @Override + public synchronized boolean start() throws IOException { + return readNextStream(); + } + + @Override + public synchronized boolean advance() throws IOException { + Preconditions.checkStateNotNull(responseIterator); + currentOffset++; + return readNextRecord(); + } + + private synchronized boolean readNextStream() throws IOException { + BigQueryStorageStreamSourceV2 source = getCurrentSource(); + if (currentStreamIndex == source.streamBundle.size() - 1) { + fractionConsumed = 1d; + return false; + } + ReadRowsRequest request = + ReadRowsRequest.newBuilder() + .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) + .setOffset(currentOffset) + .build(); + + tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); + serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); + LOG.info( + "Started BigQuery Storage API read from stream {}.", + source.streamBundle.get(0).getName()); + responseStream = storageClient.readRows(request, source.readSession.getTable()); + responseIterator = responseStream.iterator(); + return readNextRecord(); + } + + @RequiresNonNull("responseIterator") + private synchronized boolean readNextRecord() throws IOException { + Iterator responseIterator = this.responseIterator; + while (reader.readyForNextReadResponse()) { + if (!responseIterator.hasNext()) { + currentStreamIndex++; + return readNextStream(); + } + + ReadRowsResponse response; + try { + response = responseIterator.next(); + // Since we don't have a direct hook to the underlying + // API call, record success every time we read a record successfully. + if (serviceCallMetric != null) { + serviceCallMetric.call("ok"); + } + } catch (ApiException e) { + // Occasionally the iterator will fail and raise an exception. + // Capture it here and record the error in the metric. + if (serviceCallMetric != null) { + serviceCallMetric.call(e.getStatusCode().getCode().name()); + } + throw e; + } + + progressAtResponseStart = response.getStats().getProgress().getAtResponseStart(); + progressAtResponseEnd = response.getStats().getProgress().getAtResponseEnd(); + totalRowsInCurrentResponse = response.getRowCount(); + rowsConsumedFromCurrentResponse = 0L; + + checkArgument( + totalRowsInCurrentResponse >= 0, + "Row count from current response (%s) must be non-negative.", + totalRowsInCurrentResponse); + + checkArgument( + 0f <= progressAtResponseStart && progressAtResponseStart <= 1f, + "Progress at response start (%s) is not in the range [0.0, 1.0].", + progressAtResponseStart); + + checkArgument( + 0f <= progressAtResponseEnd && progressAtResponseEnd <= 1f, + "Progress at response end (%s) is not in the range [0.0, 1.0].", + progressAtResponseEnd); + + reader.processReadRowsResponse(response); + } + + SchemaAndRecord schemaAndRecord = new SchemaAndRecord(reader.readSingleRecord(), tableSchema); + + current = parseFn.apply(schemaAndRecord); + + // Updates the fraction consumed value. This value is calculated by interpolating between + // the fraction consumed value from the previous server response (or zero if we're consuming + // the first response) and the fractional value in the current response based on how many of + // the rows in the current response have been consumed. + rowsConsumedFromCurrentResponse++; + + fractionConsumed = + progressAtResponseStart + + (progressAtResponseEnd - progressAtResponseStart) + * rowsConsumedFromCurrentResponse + * 1.0 + / totalRowsInCurrentResponse; + + // Assuming that each stream in the StreamBundle has approximately the same amount of data. + fractionConsumed = fractionConsumed / source.streamBundle.size(); + + return true; + } + + @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(); + } + + @Override + public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { + return source; + } + + @Override + @SuppressWarnings("ReturnValueIgnored") + public @Nullable BoundedSource splitAtFraction(double fraction) { + int streamCountInBundle = source.streamBundle.size(); + double splitIndex = streamCountInBundle * fraction; + if (currentStreamIndex > splitIndex) { + // The reader has moved past the requested split point. + Metrics.counter( + BigQueryStorageStreamBundleReader.class, + "split-at-fraction-calls-failed-due-to-impossible-split-point") + .inc(); + LOG.info( + "BigQuery Storage API Session {} cannot be split at {}.", + source.readSession.getName(), + fraction); + return null; + } + // Splitting the remainder Streams into a new StreamBundle. + List remainderStreamBundle = + new ArrayList<>( + source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); + // Updating the primary StreamBundle. + source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); + Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") + .inc(); + LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); + return source.fromExisting(remainderStreamBundle); + } + + @Override + public synchronized Double getFractionConsumed() { + return fractionConsumed; + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index 491c5ada4f3a..53dd8818e18b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -163,6 +163,7 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); + options.as(BigQueryOptions.class).setUseReadAPISourceV2(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } From fb24fbfdf89e8065575c96edefd2b5a2bbbb45f8 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 10 Jan 2023 19:16:54 +0000 Subject: [PATCH 18/32] Renamed the Source V2. Also added tests for the same. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 385 ++- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 7 +- .../bigquery/BigQueryStorageSourceBase.java | 34 +- .../bigquery/BigQueryStorageSourceBaseV2.java | 5 - ...=> BigQueryStorageStreamBundleSource.java} | 49 +- .../bigquery/BigQueryIOStorageReadTest.java | 1 - ...StorageReadWithStreamBundleSourceTest.java | 2119 +++++++++++++++++ 7 files changed, 2363 insertions(+), 237 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/{BigQueryStorageStreamSourceV2.java => BigQueryStorageStreamBundleSource.java} (88%) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java 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 9d89687d2de0..5290c99ae20b 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 @@ -68,6 +68,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -1417,12 +1418,8 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - // if (bqOptions.getUseReadAPISourceV2()) { - // return expandAnonForDirectReadV2(input, outputCoder, beamSchema); - // } - - if(bqOptions.getUseReadAPISourceV2() || !bqOptions.getUseReadAPISourceV2()) { - + if (bqOptions.getUseReadAPIStreamBundleSource()) { + return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); } PCollectionView jobIdTokenView; @@ -1603,194 +1600,194 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - // private PCollection expandAnonForDirectReadV2( - // PBegin input, Coder outputCoder, Schema beamSchema) { - // Pipeline p = input.getPipeline(); - // PCollectionView jobIdTokenView; - // PCollection rows; - // - // if (!getWithTemplateCompatibility()) { - // // Create a singleton job ID token at pipeline construction time. - // String staticJobUuid = BigQueryHelpers.randomUUIDString(); - // jobIdTokenView = - // p.apply("TriggerIdCreation", Create.of(staticJobUuid)) - // .apply("ViewId", View.asSingleton()); - // // Apply the traditional Source model. - // rows = - // p.apply( - // org.apache.beam.sdk.io.Read.from( - // createStorageQuerySource(staticJobUuid, outputCoder))); - // } else { - // // Create a singleton job ID token at pipeline execution time. - // PCollection jobIdTokenCollection = - // p.apply("TriggerIdCreation", Create.of("ignored")) - // .apply( - // "CreateJobId", - // MapElements.via( - // new SimpleFunction() { - // @Override - // public String apply(String input) { - // return BigQueryHelpers.randomUUIDString(); - // } - // })); - // - // jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); - // - // TupleTag> listReadStreamsTag = new TupleTag<>(); - // TupleTag readSessionTag = new TupleTag<>(); - // TupleTag tableSchemaTag = new TupleTag<>(); - // - // PCollectionTuple tuple = - // jobIdTokenCollection.apply( - // "RunQueryJob", - // ParDo.of( - // new DoFn>() { - // @ProcessElement - // public void processElement(ProcessContext c) throws Exception { - // BigQueryOptions options = - // c.getPipelineOptions().as(BigQueryOptions.class); - // String jobUuid = c.element(); - // // Execute the query and get the destination table holding the results. - // // The getTargetTable call runs a new instance of the query and returns - // // the destination table created to hold the results. - // BigQueryStorageQuerySource querySource = - // createStorageQuerySource(jobUuid, outputCoder); - // Table queryResultTable = querySource.getTargetTable(options); - // - // // Create a read session without specifying a desired stream count and - // // let the BigQuery storage server pick the number of streams. - // CreateReadSessionRequest request = - // CreateReadSessionRequest.newBuilder() - // .setParent( - // BigQueryHelpers.toProjectResourceName( - // options.getBigQueryProject() == null - // ? options.getProject() - // : options.getBigQueryProject())) - // .setReadSession( - // ReadSession.newBuilder() - // .setTable( - // BigQueryHelpers.toTableResourceName( - // queryResultTable.getTableReference())) - // .setDataFormat(DataFormat.AVRO)) - // .setMaxStreamCount(0) - // .build(); - // - // ReadSession readSession; - // try (StorageClient storageClient = - // getBigQueryServices().getStorageClient(options)) { - // readSession = storageClient.createReadSession(request); - // } - // int streamIndex = 0; - // int streamsPerBundle = 10; - // List streamBundle = Lists.newArrayList(); - // for (ReadStream readStream : readSession.getStreamsList()) { - // streamIndex++; - // streamBundle.add(readStream); - // if (streamIndex % streamsPerBundle == 0) { - // c.output(streamBundle); - // streamBundle = Lists.newArrayList(); - // } - // } - // - // c.output(readSessionTag, readSession); - // c.output( - // tableSchemaTag, - // BigQueryHelpers.toJsonString(queryResultTable.getSchema())); - // } - // }) - // .withOutputTags( - // listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); - // - // TypeDescriptor> listReadStreamTypeDescriptor = new TypeDescriptor>() {}; - // tuple.get(listReadStreamsTag).setCoder(ProtoCoder.of(listReadStreamTypeDescriptor)); - // tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); - // tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - // - // PCollectionView readSessionView = - // tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); - // PCollectionView tableSchemaView = - // tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - // - // rows = - // tuple - // .get(listReadStreamsTag) - // .apply(Reshuffle.viaRandomKey()) - // .apply( - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) throws Exception { - // ReadSession readSession = c.sideInput(readSessionView); - // TableSchema tableSchema = - // BigQueryHelpers.fromJsonString( - // c.sideInput(tableSchemaView), TableSchema.class); - // List streamBundle = c.element(); - // - // BigQueryStorageStreamSourceV2 streamSource = - // BigQueryStorageStreamSourceV2.create( - // readSession, - // streamBundle, - // tableSchema, - // getParseFn(), - // outputCoder, - // getBigQueryServices()); - // - // // Read all of the data from the stream. In the event that this work - // // item fails and is rescheduled, the same rows will be returned in - // // the same order. - // BoundedReader reader = - // streamSource.createReader(c.getPipelineOptions()); - // for (boolean more = reader.start(); more; more = reader.advance()) { - // c.output(reader.getCurrent()); - // } - // } - // }) - // .withSideInputs(readSessionView, tableSchemaView)) - // .setCoder(outputCoder); - // } - // - // CleanupOperation cleanupOperation = - // new CleanupOperation() { - // @Override - // void cleanup(ContextContainer c) throws Exception { - // BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); - // String jobUuid = c.getJobId(); - // - // Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); - // - // TableReference tempTable = - // createTempTableReference( - // options.getBigQueryProject() == null - // ? options.getProject() - // : options.getBigQueryProject(), - // BigQueryResourceNaming.createJobIdPrefix( - // options.getJobName(), jobUuid, JobType.QUERY), - // queryTempDataset); - // - // try (DatasetService datasetService = - // getBigQueryServices().getDatasetService(options)) { - // LOG.info("Deleting temporary table with query results {}", tempTable); - // datasetService.deleteTable(tempTable); - // // Delete dataset only if it was created by Beam - // boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); - // if (datasetCreatedByBeam) { - // LOG.info( - // "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - // datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); - // } - // } - // } - // }; - // - // if (beamSchema != null) { - // rows.setSchema( - // beamSchema, - // getTypeDescriptor(), - // getToBeamRowFn().apply(beamSchema), - // getFromBeamRowFn().apply(beamSchema)); - // } - // return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); - // } + private PCollection expandAnonForDirectReadWithStreamBundle( + PBegin input, Coder outputCoder, Schema beamSchema) { + + Pipeline p = input.getPipeline(); + PCollectionView jobIdTokenView; + PCollection rows; + + if (!getWithTemplateCompatibility()) { + // Create a singleton job ID token at pipeline construction time. + String staticJobUuid = BigQueryHelpers.randomUUIDString(); + jobIdTokenView = + p.apply("TriggerIdCreation", Create.of(staticJobUuid)) + .apply("ViewId", View.asSingleton()); + // Apply the traditional Source model. + rows = + p.apply( + org.apache.beam.sdk.io.Read.from( + createStorageQuerySource(staticJobUuid, outputCoder))); + } else { + // Create a singleton job ID token at pipeline execution time. + PCollection jobIdTokenCollection = + p.apply("TriggerIdCreation", Create.of("ignored")) + .apply( + "CreateJobId", + MapElements.via( + new SimpleFunction() { + @Override + public String apply(String input) { + return BigQueryHelpers.randomUUIDString(); + } + })); + + jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); + + TupleTag> listReadStreamsTag = new TupleTag<>(); + TupleTag readSessionTag = new TupleTag<>(); + TupleTag tableSchemaTag = new TupleTag<>(); + + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); + } + int streamIndex = 0; + int streamsPerBundle = 10; + List streamBundle = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + c.output(streamBundle); + streamBundle = Lists.newArrayList(); + } + } + + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); + tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); + tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); + + PCollectionView readSessionView = + tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); + PCollectionView tableSchemaView = + tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); + + rows = + tuple + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); + } + + CleanupOperation cleanupOperation = + new CleanupOperation() { + @Override + void cleanup(ContextContainer c) throws Exception { + BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.getJobId(); + + Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); + + TableReference tempTable = + createTempTableReference( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject(), + BigQueryResourceNaming.createJobIdPrefix( + options.getJobName(), jobUuid, JobType.QUERY), + queryTempDataset); + + try (DatasetService datasetService = + getBigQueryServices().getDatasetService(options)) { + LOG.info("Deleting temporary table with query results {}", tempTable); + datasetService.deleteTable(tempTable); + // Delete dataset only if it was created by Beam + boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); + if (datasetCreatedByBeam) { + LOG.info( + "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + } + } + } + }; + + if (beamSchema != null) { + rows.setSchema( + beamSchema, + getTypeDescriptor(), + getToBeamRowFn().apply(beamSchema), + getFromBeamRowFn().apply(beamSchema)); + } + return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + } @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 3003666d19be..02cb7cd2fba8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -164,9 +164,10 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); - @Description("If set, BigQueryIO.Read will default to using the V2 implementation of the Read API Source") + @Description("If set, BigQueryIO.Read will default to using the StreamBundle based" + + "implementation of the Read API Source") @Default.Boolean(false) - Boolean getUseReadAPISourceV2(); + Boolean getUseReadAPIStreamBundleSource(); - void setUseReadAPISourceV2(Boolean value); + void setUseReadAPIStreamBundleSource(Boolean value); } 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 69c47ea7c547..2d5202aa0443 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 @@ -137,7 +137,7 @@ public List> split( // an appropriate number of streams for the Session to produce reasonable throughput. // This is required when using the Read API Source V2. int streamCount = 0; - if(!bqOptions.getUseReadAPISourceV2()) { + if (!bqOptions.getUseReadAPIStreamBundleSource()) { if (desiredBundleSizeBytes > 0) { long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); @@ -161,9 +161,10 @@ public List> split( try (StorageClient client = bqServices.getStorageClient(bqOptions)) { readSession = client.createReadSession(createReadSessionRequest); LOG.info( - "Sent BigQuery Storage API CreateReadSession request '{}'; received response '{}'.", - createReadSessionRequest, - readSession); + "Sent BigQuery Storage API CreateReadSession request in code '{}'.", + createReadSessionRequest); + LOG.info( + "Received number of streams in response: '{}'.", readSession.getStreamsList().size()); } if (readSession.getStreamsList().isEmpty()) { @@ -171,18 +172,23 @@ public List> split( return ImmutableList.of(); } + streamCount = readSession.getStreamsList().size(); int streamsPerBundle = 0; - if (bqOptions.getUseReadAPISourceV2()) { + double bytesPerStream = 0; + LOG.info( + "readSession.getEstimatedTotalBytesScanned(): '{}'", + readSession.getEstimatedTotalBytesScanned()); + if (bqOptions.getUseReadAPIStreamBundleSource()) { if (desiredBundleSizeBytes > 0) { - streamsPerBundle = - (int) - Math.ceil((double) - desiredBundleSizeBytes - / (double) (readSession.getEstimatedTotalBytesScanned() - / readSession.getStreamsList().size())); + bytesPerStream = + (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); + LOG.info("bytesPerStream: '{}'", bytesPerStream); + streamsPerBundle = (int) Math.ceil(desiredBundleSizeBytes / bytesPerStream); } else { streamsPerBundle = (int) Math.ceil((double) streamCount / 10); } + streamsPerBundle = Math.min(streamCount, streamsPerBundle); + LOG.info("streamsPerBundle: '{}'", streamsPerBundle); } Schema sessionSchema; @@ -204,7 +210,7 @@ public List> split( targetTable); // TODO: this is inconsistent with method above, where it can be null TableSchema trimmedSchema = BigQueryAvroUtils.trimBigQueryTableSchema(targetTable.getSchema(), sessionSchema); - if(!bqOptions.getUseReadAPISourceV2()) { + if (!bqOptions.getUseReadAPIStreamBundleSource()) { List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { sources.add( @@ -214,13 +220,13 @@ public List> split( return ImmutableList.copyOf(sources); } else { List streamBundle = Lists.newArrayList(); - List> sources = Lists.newArrayList(); + List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { streamIndex++; streamBundle.add(readStream); if (streamIndex % streamsPerBundle == 0) { sources.add( - BigQueryStorageStreamSourceV2.create( + BigQueryStorageStreamBundleSource.create( readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); streamBundle = Lists.newArrayList(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java deleted file mode 100644 index a0ba4e22d1e3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBaseV2.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.beam.sdk.io.gcp.bigquery; - -public class BigQueryStorageSourceBaseV2 { - -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java similarity index 88% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index b4028996a0ef..fb788eb4920b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSourceV2.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -45,22 +45,22 @@ 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; -class BigQueryStorageStreamSourceV2 extends BoundedSource { +class BigQueryStorageStreamBundleSource extends BoundedSource { - private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSourceV2.class); + private static final Logger LOG = + LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); - public static BigQueryStorageStreamSourceV2 create( + public static BigQueryStorageStreamBundleSource create( ReadSession readSession, List streamBundle, TableSchema tableSchema, SerializableFunction parseFn, Coder outputCoder, BigQueryServices bqServices) { - return new BigQueryStorageStreamSourceV2<>( + return new BigQueryStorageStreamBundleSource<>( readSession, streamBundle, toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), @@ -73,8 +73,8 @@ public static BigQueryStorageStreamSourceV2 create( * Creates a new source with the same properties as this one, except with a different {@link * List}. */ - public BigQueryStorageStreamSourceV2 fromExisting(List newStreamBundle) { - return new BigQueryStorageStreamSourceV2<>( + public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { + return new BigQueryStorageStreamBundleSource<>( readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); } @@ -85,7 +85,7 @@ public BigQueryStorageStreamSourceV2 fromExisting(List newStreamB private final Coder outputCoder; private final BigQueryServices bqServices; - private BigQueryStorageStreamSourceV2( + private BigQueryStorageStreamBundleSource( ReadSession readSession, List streamBundle, String jsonTableSchema, @@ -148,7 +148,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private final StorageClient storageClient; private final TableSchema tableSchema; - private BigQueryStorageStreamSourceV2 source; + private BigQueryStorageStreamBundleSource source; private @Nullable BigQueryServerStream responseStream = null; private @Nullable Iterator responseIterator = null; private @Nullable T current = null; @@ -157,6 +157,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B // Values used for progress reporting. private double fractionConsumed; + private double fractionOfStreamsConsumed; private double progressAtResponseStart; private double progressAtResponseEnd; private long rowsConsumedFromCurrentResponse; @@ -166,7 +167,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private @Nullable ServiceCallMetric serviceCallMetric; private BigQueryStorageStreamBundleReader( - BigQueryStorageStreamSourceV2 source, BigQueryOptions options) throws IOException { + BigQueryStorageStreamBundleSource source, BigQueryOptions options) throws IOException { this.source = source; this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); this.parseFn = source.parseFn; @@ -174,6 +175,7 @@ private BigQueryStorageStreamBundleReader( this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; this.fractionConsumed = 0d; + this.fractionOfStreamsConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -195,14 +197,14 @@ public synchronized boolean start() throws IOException { @Override public synchronized boolean advance() throws IOException { - Preconditions.checkStateNotNull(responseIterator); + // Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } private synchronized boolean readNextStream() throws IOException { - BigQueryStorageStreamSourceV2 source = getCurrentSource(); - if (currentStreamIndex == source.streamBundle.size() - 1) { + BigQueryStorageStreamBundleSource source = getCurrentSource(); + if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; return false; } @@ -211,7 +213,6 @@ private synchronized boolean readNextStream() throws IOException { .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) .setOffset(currentOffset) .build(); - tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( @@ -222,11 +223,15 @@ private synchronized boolean readNextStream() throws IOException { return readNextRecord(); } - @RequiresNonNull("responseIterator") + // @RequiresNonNull("responseIterator") private synchronized boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; + if (responseIterator == null) { + return false; + } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { + currentOffset = 0; currentStreamIndex++; return readNextStream(); } @@ -288,9 +293,12 @@ private synchronized boolean readNextRecord() throws IOException { * 1.0 / totalRowsInCurrentResponse; - // Assuming that each stream in the StreamBundle has approximately the same amount of data. + // Assuming that each stream in the StreamBundle has approximately the same amount of data and + // normalizing the value of fractionConsumed. fractionConsumed = fractionConsumed / source.streamBundle.size(); + fractionOfStreamsConsumed = (double) currentStreamIndex / source.streamBundle.size(); + fractionConsumed += fractionOfStreamsConsumed; return true; } @@ -298,14 +306,14 @@ private synchronized boolean readNextRecord() throws IOException { public synchronized void close() { // Because superclass cannot have preconditions around these variables, cannot use // @RequiresNonNull - Preconditions.checkStateNotNull(storageClient); - Preconditions.checkStateNotNull(reader); + // Preconditions.checkStateNotNull(storageClient); + // Preconditions.checkStateNotNull(reader); storageClient.close(); reader.close(); } @Override - public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { + public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } @@ -314,8 +322,9 @@ public synchronized BigQueryStorageStreamSourceV2 getCurrentSource() { public @Nullable BoundedSource splitAtFraction(double fraction) { int streamCountInBundle = source.streamBundle.size(); double splitIndex = streamCountInBundle * fraction; - if (currentStreamIndex > splitIndex) { + if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { // The reader has moved past the requested split point. + // NOTE: We do not split below the granularity of a stream. Metrics.counter( BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-failed-due-to-impossible-split-point") diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index 53dd8818e18b..491c5ada4f3a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -163,7 +163,6 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); - options.as(BigQueryOptions.class).setUseReadAPISourceV2(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java new file mode 100644 index 000000000000..bd0cb1124f8e --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -0,0 +1,2119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static java.util.Arrays.asList; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.rpc.FailedPreconditionException; +import com.google.api.services.bigquery.model.Streamingbuffer; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1.ArrowRecordBatch; +import com.google.cloud.bigquery.storage.v1.ArrowSchema; +import com.google.cloud.bigquery.storage.v1.AvroRows; +import com.google.cloud.bigquery.storage.v1.AvroSchema; +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.DataFormat; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadStream; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; +import com.google.cloud.bigquery.storage.v1.StreamStats; +import com.google.cloud.bigquery.storage.v1.StreamStats.Progress; +import com.google.protobuf.ByteString; +import io.grpc.Status; +import io.grpc.Status.Code; +import io.grpc.StatusRuntimeException; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.math.BigInteger; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.VectorUnloader; +import org.apache.arrow.vector.ipc.WriteChannel; +import org.apache.arrow.vector.ipc.message.MessageSerializer; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.util.Text; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; +import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TableRowParser; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamBundleSource.BigQueryStorageStreamBundleReader; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.ConversionOptions; +import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices; +import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices.FakeBigQueryServerStream; +import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +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; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.junit.runners.model.Statement; + +/** + * Tests for {@link BigQueryIO#readTableRows() using {@link Method#DIRECT_READ}} AND {@link + * BigQueryOptions#setUseReadAPIStreamBundleSource(Boolean)} set to True. + */ +@RunWith(JUnit4.class) +public class BigQueryIOStorageReadWithStreamBundleSourceTest { + + private transient PipelineOptions options; + private final transient TemporaryFolder testFolder = new TemporaryFolder(); + private transient TestPipeline p; + private BufferAllocator allocator; + + @Rule + public final transient TestRule folderThenPipeline = + new TestRule() { + @Override + public Statement apply(Statement base, Description description) { + // We need to set up the temporary folder, and then set up the TestPipeline based on the + // chosen folder. Unfortunately, since rule evaluation order is unspecified and unrelated + // to field order, and is separate from construction, that requires manually creating this + // TestRule. + Statement withPipeline = + new Statement() { + @Override + public void evaluate() throws Throwable { + options = TestPipeline.testingPipelineOptions(); + options.as(BigQueryOptions.class).setProject("project-id"); + if (description.getAnnotations().stream() + .anyMatch(a -> a.annotationType().equals(ProjectOverride.class))) { + options.as(BigQueryOptions.class).setBigQueryProject("bigquery-project-id"); + } + options + .as(BigQueryOptions.class) + .setTempLocation(testFolder.getRoot().getAbsolutePath()); + options.as(BigQueryOptions.class).setUseReadAPIStreamBundleSource(true); + p = TestPipeline.fromOptions(options); + p.apply(base, description).evaluate(); + } + }; + return testFolder.apply(withPipeline, description); + } + }; + + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + private final FakeDatasetService fakeDatasetService = new FakeDatasetService(); + + @Before + public void setUp() throws Exception { + FakeDatasetService.setUp(); + allocator = new RootAllocator(Long.MAX_VALUE); + } + + @After + public void teardown() { + allocator.close(); + } + + @Test + public void testBuildTableBasedSource() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table"); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + assertTrue(typedRead.getValidate()); + } + + @Test + public void testBuildTableBasedSourceWithoutValidation() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .withoutValidation(); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + assertFalse(typedRead.getValidate()); + } + + @Test + public void testBuildTableBasedSourceWithDefaultProject() { + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("myDataset.myTable"); + checkTypedReadTableObject(typedRead, null, "myDataset", "myTable"); + } + + @Test + public void testBuildTableBasedSourceWithTableReference() { + TableReference tableReference = + new TableReference() + .setProjectId("foo.com:project") + .setDatasetId("dataset") + .setTableId("table"); + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from(tableReference); + checkTypedReadTableObject(typedRead, "foo.com:project", "dataset", "table"); + } + + private void checkTypedReadTableObject( + TypedRead typedRead, String project, String dataset, String table) { + assertEquals(project, typedRead.getTable().getProjectId()); + assertEquals(dataset, typedRead.getTable().getDatasetId()); + assertEquals(table, typedRead.getTable().getTableId()); + assertNull(typedRead.getQuery()); + assertEquals(Method.DIRECT_READ, typedRead.getMethod()); + } + + @Test + public void testBuildSourceWithTableAndFlatten() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid BigQueryIO.Read: Specifies a table with a result flattening preference," + + " which only applies to queries"); + p.apply( + "ReadMyTable", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .withoutResultFlattening()); + p.run(); + } + + @Test + public void testBuildSourceWithTableAndSqlDialect() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid BigQueryIO.Read: Specifies a table with a SQL dialect preference," + + " which only applies to queries"); + p.apply( + "ReadMyTable", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .usingStandardSql()); + p.run(); + } + + @Test + public void testDisplayData() { + String tableSpec = "foo.com:project:dataset.table"; + BigQueryIO.TypedRead typedRead = + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .withSelectedFields(ImmutableList.of("foo", "bar")) + .withProjectionPushdownApplied() + .from(tableSpec); + DisplayData displayData = DisplayData.from(typedRead); + assertThat(displayData, hasDisplayItem("table", tableSpec)); + assertThat(displayData, hasDisplayItem("selectedFields", "foo, bar")); + assertThat(displayData, hasDisplayItem("projectionPushdownApplied", true)); + } + + @Test + public void testName() { + assertEquals( + "BigQueryIO.TypedRead", + BigQueryIO.read(new TableRowParser()) + .withCoder(TableRowJsonCoder.of()) + .withMethod(Method.DIRECT_READ) + .from("foo.com:project:dataset.table") + .getName()); + } + + @Test + public void testCoderInference() { + // Lambdas erase too much type information -- use an anonymous class here. + SerializableFunction> parseFn = + new SerializableFunction>() { + @Override + public KV apply(SchemaAndRecord input) { + return null; + } + }; + + assertEquals( + KvCoder.of(ByteStringCoder.of(), ProtoCoder.of(ReadSession.class)), + BigQueryIO.read(parseFn).inferCoder(CoderRegistry.createDefault())); + } + + @Test + public void testTableSourceEstimatedSize() throws Exception { + doTableSourceEstimatedSizeTest(false); + } + + @Test + public void testTableSourceEstimatedSize_IgnoresStreamingBuffer() throws Exception { + doTableSourceEstimatedSizeTest(true); + } + + private void doTableSourceEstimatedSizeTest(boolean useStreamingBuffer) throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + if (useStreamingBuffer) { + table.setStreamingBuffer(new Streamingbuffer().setEstimatedBytes(BigInteger.TEN)); + } + + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + @Test + @ProjectOverride + public void testTableSourceEstimatedSize_WithBigQueryProject() throws Exception { + fakeDatasetService.createDataset("bigquery-project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("bigquery-project-id:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + @Test + public void testTableSourceEstimatedSize_WithDefaultProject() throws Exception { + fakeDatasetService.createDataset("project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(100L); + fakeDatasetService.createTable(table); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + assertEquals(100, tableSource.getEstimatedSizeBytes(options)); + } + + private static final String AVRO_SCHEMA_STRING = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RowRecord\",\n" + + " \"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"number\", \"type\": \"long\"}\n" + + " ]\n" + + "}"; + + private static final Schema AVRO_SCHEMA = new Schema.Parser().parse(AVRO_SCHEMA_STRING); + + private static final String TRIMMED_AVRO_SCHEMA_STRING = + "{\"namespace\": \"example.avro\",\n" + + "\"type\": \"record\",\n" + + "\"name\": \"RowRecord\",\n" + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"}\n" + + " ]\n" + + "}"; + + private static final Schema TRIMMED_AVRO_SCHEMA = + new Schema.Parser().parse(TRIMMED_AVRO_SCHEMA_STRING); + + private static final TableSchema TABLE_SCHEMA = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING").setMode("REQUIRED"), + new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"))); + + private static final org.apache.arrow.vector.types.pojo.Schema ARROW_SCHEMA = + new org.apache.arrow.vector.types.pojo.Schema( + asList( + field("name", new ArrowType.Utf8()), field("number", new ArrowType.Int(64, true)))); + + private void doTableSourceInitialSplitTest(long bundleSize, long tableSize, int streamCount) + throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = + new Table().setTableReference(tableRef).setNumBytes(tableSize).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(tableSize); + for (int i = 0; i < streamCount; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(bundleSize, options); + // Each StreamBundle is expected to contain a single stream. + assertEquals(streamCount, sources.size()); + } + + @Test + public void testTableSourceInitialSplit() throws Exception { + doTableSourceInitialSplitTest(1024L, 1024L * 1024L, 1024); + } + + @Test + public void testTableSourceInitialSplit_MinSplitCount() throws Exception { + doTableSourceInitialSplitTest(1024L, 1024L * 1024L, 10); + } + + @Test + public void testTableSourceInitialSplit_MaxSplitCount() throws Exception { + doTableSourceInitialSplitTest(10L, 1024L * 1024L, 10_000); + } + + @Test + public void testTableSourceInitialSplit_WithSelectedFieldsAndRowRestriction() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = new Table().setTableReference(tableRef).setNumBytes(200L).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder() + .addSelectedFields("name") + .setRowRestriction("number > 5"))) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(100L); + for (int i = 0; i < 10; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + StaticValueProvider.of(Lists.newArrayList("name")), + StaticValueProvider.of("number > 5"), + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(20L, options); + assertEquals(5, sources.size()); + } + + @Test + public void testTableSourceInitialSplit_WithDefaultProject() throws Exception { + fakeDatasetService.createDataset("project-id", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset.table"); + + Table table = + new Table().setTableReference(tableRef).setNumBytes(1024L).setSchema(TABLE_SCHEMA); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/project-id/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession.Builder builder = + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(1024L); + for (int i = 0; i < 50; i++) { + builder.addStreams(ReadStream.newBuilder().setName("stream-" + i)); + } + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(builder.build()); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(BigQueryHelpers.parseTableSpec("dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(4096L, options); + // A single StreamBundle containing all the Streams. + assertEquals(1, sources.size()); + } + + @Test + public void testTableSourceInitialSplit_EmptyTable() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + + Table table = + new Table() + .setTableReference(tableRef) + .setNumBytes(1024L * 1024L) + .setSchema(new TableSchema()); + + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table")) + .setMaxStreamCount(0) + .build(); + + ReadSession emptyReadSession = ReadSession.newBuilder().build(); + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.createReadSession(expectedRequest)).thenReturn(emptyReadSession); + + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of(tableRef), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient)); + + List> sources = tableSource.split(1024L, options); + assertTrue(sources.isEmpty()); + } + + @Test + public void testTableSourceCreateReader() throws Exception { + BigQueryStorageTableSource tableSource = + BigQueryStorageTableSource.create( + ValueProvider.StaticValueProvider.of( + BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table")), + null, + null, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withDatasetService(fakeDatasetService)); + + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage("BigQuery storage source must be split before reading"); + tableSource.createReader(options); + } + + private static GenericRecord createRecord(String name, Schema schema) { + GenericRecord genericRecord = new Record(schema); + genericRecord.put("name", name); + return genericRecord; + } + + private static GenericRecord createRecord(String name, long number, Schema schema) { + GenericRecord genericRecord = new Record(schema); + genericRecord.put("name", name); + genericRecord.put("number", number); + return genericRecord; + } + + private static ByteString serializeArrowSchema( + org.apache.arrow.vector.types.pojo.Schema arrowSchema) { + ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream(); + try { + MessageSerializer.serialize( + new WriteChannel(Channels.newChannel(byteOutputStream)), arrowSchema); + } catch (IOException ex) { + throw new RuntimeException("Failed to serialize arrow schema.", ex); + } + return ByteString.copyFrom(byteOutputStream.toByteArray()); + } + + private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get(); + + private static ReadRowsResponse createResponse( + Schema schema, + Collection genericRecords, + double progressAtResponseStart, + double progressAtResponseEnd) + throws Exception { + GenericDatumWriter writer = new GenericDatumWriter<>(schema); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + Encoder binaryEncoder = ENCODER_FACTORY.binaryEncoder(outputStream, null); + for (GenericRecord genericRecord : genericRecords) { + writer.write(genericRecord, binaryEncoder); + } + + binaryEncoder.flush(); + + return ReadRowsResponse.newBuilder() + .setAvroRows( + AvroRows.newBuilder() + .setSerializedBinaryRows(ByteString.copyFrom(outputStream.toByteArray())) + .setRowCount(genericRecords.size())) + .setRowCount(genericRecords.size()) + .setStats( + StreamStats.newBuilder() + .setProgress( + Progress.newBuilder() + .setAtResponseStart(progressAtResponseStart) + .setAtResponseEnd(progressAtResponseEnd))) + .build(); + } + + private ReadRowsResponse createResponseArrow( + org.apache.arrow.vector.types.pojo.Schema arrowSchema, + List name, + List number, + double progressAtResponseStart, + double progressAtResponseEnd) { + ArrowRecordBatch serializedRecord; + try (VectorSchemaRoot schemaRoot = VectorSchemaRoot.create(arrowSchema, allocator)) { + schemaRoot.allocateNew(); + schemaRoot.setRowCount(name.size()); + VarCharVector strVector = (VarCharVector) schemaRoot.getFieldVectors().get(0); + BigIntVector bigIntVector = (BigIntVector) schemaRoot.getFieldVectors().get(1); + for (int i = 0; i < name.size(); i++) { + bigIntVector.set(i, number.get(i)); + strVector.set(i, new Text(name.get(i))); + } + + VectorUnloader unLoader = new VectorUnloader(schemaRoot); + try (org.apache.arrow.vector.ipc.message.ArrowRecordBatch records = + unLoader.getRecordBatch()) { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + MessageSerializer.serialize(new WriteChannel(Channels.newChannel(os)), records); + serializedRecord = + ArrowRecordBatch.newBuilder() + .setRowCount(records.getLength()) + .setSerializedRecordBatch(ByteString.copyFrom(os.toByteArray())) + .build(); + } catch (IOException e) { + throw new RuntimeException("Error writing to byte array output stream", e); + } + } + } + + return ReadRowsResponse.newBuilder() + .setArrowRecordBatch(serializedRecord) + .setRowCount(name.size()) + .setStats( + StreamStats.newBuilder() + .setProgress( + Progress.newBuilder() + .setAtResponseStart(progressAtResponseStart) + .setAtResponseEnd(progressAtResponseEnd))) + .build(); + } + + @Test + public void testStreamSourceEstimatedSizeBytes() throws Exception { + List streamBundle = Lists.newArrayList(ReadStream.getDefaultInstance()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.getDefaultInstance(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices()); + + assertEquals(0, streamSource.getEstimatedSizeBytes(options)); + } + + @Test + public void testStreamSourceSplit() throws Exception { + List streamBundle = Lists.newArrayList(ReadStream.getDefaultInstance()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.getDefaultInstance(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices()); + + assertThat(streamSource.split(0, options), containsInAnyOrder(streamSource)); + } + + @Test + public void testReadFromStreamSource() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").setOffset(0).build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").setOffset(0).build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)); + + List responsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(2, 3), 0.5, 0.75)); + List responsesTwo = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(3, 5), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(5, 6), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + List rows = new ArrayList<>(); + BigQueryStorageStreamBundleReader reader = streamSource.createReader(options); + for (boolean hasNext = reader.start(); hasNext; hasNext = reader.advance()) { + rows.add(reader.getCurrent()); + } + + System.out.println("Rows: " + rows); + + assertEquals(6, rows.size()); + } + + private static final double DELTA = 1e-6; + + @Test + public void testFractionConsumedWithOneStreamInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List responses = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.25), + // Some responses may contain zero results, so we must ensure that we can are resilient + // to such responses. + createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.25, 0.25), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.3, 0.5), + createResponse(AVRO_SCHEMA, records.subList(4, 7), 0.7, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.4, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.8, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.9, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the stream, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List responsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.5), + // Some responses may contain zero results, so we must ensure that we can are resilient + // to such responses. + createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.5, 0.5), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); + + List responsesTwo = + Lists.newArrayList(createResponse(AVRO_SCHEMA, records.subList(4, 7), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.375, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.6666666666666666, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.8333333333333333, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the streams, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testStreamSourceSplitAtFractionNoOpWithOneStreamInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List parentStreamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); + BigQueryStorageStreamBundleSource streamBundleSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + parentStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent stream and ensure that records are returned in the + // prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the stream. Since we do NOT split below the granularity of a single stream, + // this will be a No-Op and the primary source should be read to completion. + BoundedSource secondary = primary.splitAtFraction(0.5); + assertNull(secondary); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the primary bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the StreamBundle and ensure that the returned source points to a non-null secondary + // StreamBundle + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNotNull(secondarySource); + BoundedReader secondary = secondarySource.createReader(options); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertFalse(secondary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("C", 3, AVRO_SCHEMA), createRecord("D", 4, AVRO_SCHEMA)), + 0.25, + 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("E", 5, AVRO_SCHEMA), createRecord("F", 6, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); + + BoundedSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader primaryReader = primarySource.createReader(options); + assertTrue(primaryReader.start()); + assertEquals("A", primaryReader.getCurrent().get("name")); + + // Should create two sources: the first with 1 stream, the second with 2. + BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + assertNotNull(secondarySource); + assertEquals("A", primaryReader.getCurrent().get("name")); + + assertTrue(primaryReader.advance()); + assertEquals("B", primaryReader.getCurrent().get("name")); + assertFalse(primaryReader.advance()); + + BoundedReader secondaryReader = secondarySource.createReader(options); + assertTrue(secondaryReader.start()); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + // Should create two sources: each with 1 stream. + BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + assertNotNull(tertiarySource); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + assertTrue(secondaryReader.advance()); + assertEquals("D", secondaryReader.getCurrent().get("name")); + assertFalse(secondaryReader.advance()); + + BoundedReader tertiaryReader = tertiarySource.createReader(options); + assertTrue(tertiaryReader.start()); + assertEquals("E", tertiaryReader.getCurrent().get("name")); + + assertTrue(tertiaryReader.advance()); + assertEquals("F", tertiaryReader.getCurrent().get("name")); + assertFalse(tertiaryReader.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() throws Exception { + List responses = + Lists.newArrayList( + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), + 0.0, + 0.25), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + createResponse( + AVRO_SCHEMA, + Lists.newArrayList( + createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), + 0.50, + 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List parentStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamBundleSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .build(), + parentStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + + // We attempt to split the StreamBundle after starting to read the contents of the second + // stream. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNull(secondarySource); + + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + } + + private static final class ParseKeyValue + implements SerializableFunction> { + + @Override + public KV apply(SchemaAndRecord input) { + return KV.of( + input.getRecord().get("name").toString(), (Long) input.getRecord().get("number")); + } + } + + @Test + public void testReadFromBigQueryIO() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .setEstimatedTotalBytesScanned(10L) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", 1, AVRO_SCHEMA), + createRecord("B", 2, AVRO_SCHEMA), + createRecord("C", 3, AVRO_SCHEMA), + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA), + createRecord("G", 7, AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection> output = + p.apply( + BigQueryIO.read(new ParseKeyValue()) + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + KV.of("A", 1L), + KV.of("B", 2L), + KV.of("C", 3L), + KV.of("D", 4L), + KV.of("E", 5L), + KV.of("F", 6L), + KV.of("G", 7L))); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOWithTrimmedSchema() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder().addSelectedFields("name")) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", TRIMMED_AVRO_SCHEMA), + createRecord("B", TRIMMED_AVRO_SCHEMA), + createRecord("C", TRIMMED_AVRO_SCHEMA), + createRecord("D", TRIMMED_AVRO_SCHEMA), + createRecord("E", TRIMMED_AVRO_SCHEMA), + createRecord("F", TRIMMED_AVRO_SCHEMA), + createRecord("G", TRIMMED_AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection output = + p.apply( + BigQueryIO.readTableRows() + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withSelectedFields(Lists.newArrayList("name")) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + new TableRow().set("name", "A"), + new TableRow().set("name", "B"), + new TableRow().set("name", "C"), + new TableRow().set("name", "D"), + new TableRow().set("name", "E"), + new TableRow().set("name", "F"), + new TableRow().set("name", "G"))); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOWithBeamSchema() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setReadOptions( + ReadSession.TableReadOptions.newBuilder().addSelectedFields("name")) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setAvroSchema(AvroSchema.newBuilder().setSchema(TRIMMED_AVRO_SCHEMA_STRING)) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.AVRO) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List records = + Lists.newArrayList( + createRecord("A", TRIMMED_AVRO_SCHEMA), + createRecord("B", TRIMMED_AVRO_SCHEMA), + createRecord("C", TRIMMED_AVRO_SCHEMA), + createRecord("D", TRIMMED_AVRO_SCHEMA), + createRecord("E", TRIMMED_AVRO_SCHEMA), + createRecord("F", TRIMMED_AVRO_SCHEMA), + createRecord("G", TRIMMED_AVRO_SCHEMA)); + + List readRowsResponsesOne = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.50), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(4, 5), 0.0, 0.33), + createResponse(TRIMMED_AVRO_SCHEMA, records.subList(5, 7), 0.33, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection output = + p.apply( + BigQueryIO.readTableRowsWithSchema() + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withSelectedFields(Lists.newArrayList("name")) + .withFormat(DataFormat.AVRO) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))) + .apply(Convert.toRows()); + + org.apache.beam.sdk.schemas.Schema beamSchema = + org.apache.beam.sdk.schemas.Schema.of( + org.apache.beam.sdk.schemas.Schema.Field.of( + "name", org.apache.beam.sdk.schemas.Schema.FieldType.STRING)); + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + Row.withSchema(beamSchema).addValue("A").build(), + Row.withSchema(beamSchema).addValue("B").build(), + Row.withSchema(beamSchema).addValue("C").build(), + Row.withSchema(beamSchema).addValue("D").build(), + Row.withSchema(beamSchema).addValue("E").build(), + Row.withSchema(beamSchema).addValue("F").build(), + Row.withSchema(beamSchema).addValue("G").build())); + + p.run(); + } + + @Test + public void testReadFromBigQueryIOArrow() throws Exception { + fakeDatasetService.createDataset("foo.com:project", "dataset", "", "", null); + TableReference tableRef = BigQueryHelpers.parseTableSpec("foo.com:project:dataset.table"); + Table table = new Table().setTableReference(tableRef).setNumBytes(10L).setSchema(TABLE_SCHEMA); + fakeDatasetService.createTable(table); + + CreateReadSessionRequest expectedCreateReadSessionRequest = + CreateReadSessionRequest.newBuilder() + .setParent("projects/project-id") + .setReadSession( + ReadSession.newBuilder() + .setTable("projects/foo.com:project/datasets/dataset/tables/table") + .setDataFormat(DataFormat.ARROW)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSessionName") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .addStreams(ReadStream.newBuilder().setName("streamName1")) + .addStreams(ReadStream.newBuilder().setName("streamName2")) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedReadRowsRequestOne = + ReadRowsRequest.newBuilder().setReadStream("streamName1").build(); + ReadRowsRequest expectedReadRowsRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("streamName2").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List readRowsResponsesOne = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.50), + createResponseArrow( + ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.5, 0.75)); + List readRowsResponsesTwo = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(4, 5), values.subList(4, 5), 0.0, 0.33), + createResponseArrow( + ARROW_SCHEMA, names.subList(5, 6), values.subList(5, 6), 0.33, 0.66), + createResponseArrow( + ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.66, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class, withSettings().serializable()); + when(fakeStorageClient.createReadSession(expectedCreateReadSessionRequest)) + .thenReturn(readSession); + when(fakeStorageClient.readRows(expectedReadRowsRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesOne)); + when(fakeStorageClient.readRows(expectedReadRowsRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(readRowsResponsesTwo)); + + PCollection> output = + p.apply( + BigQueryIO.read(new ParseKeyValue()) + .from("foo.com:project:dataset.table") + .withMethod(Method.DIRECT_READ) + .withFormat(DataFormat.ARROW) + .withTestServices( + new FakeBigQueryServices() + .withDatasetService(fakeDatasetService) + .withStorageClient(fakeStorageClient))); + + PAssert.that(output) + .containsInAnyOrder( + ImmutableList.of( + KV.of("A", 1L), + KV.of("B", 2L), + KV.of("C", 3L), + KV.of("D", 4L), + KV.of("E", 5L), + KV.of("F", 6L), + KV.of("G", 7L))); + + p.run(); + } + + @Test + public void testReadFromStreamSourceArrow() throws Exception { + + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List names = Arrays.asList("A", "B", "C"); + List values = Arrays.asList(1L, 2L, 3L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.50), + createResponseArrow( + ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + List rows = new ArrayList<>(); + BoundedReader reader = streamSource.createReader(options); + for (boolean hasNext = reader.start(); hasNext; hasNext = reader.advance()) { + rows.add(reader.getCurrent()); + } + + System.out.println("Rows: " + rows); + + assertEquals(3, rows.size()); + } + + @Test + public void testFractionConsumedWithArrowAndOneStreamInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequest = + ReadRowsRequest.newBuilder().setReadStream("readStream").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), + createResponseArrow( + ARROW_SCHEMA, Lists.newArrayList(), Lists.newArrayList(), 0.25, 0.25), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.3, 0.5), + createResponseArrow(ARROW_SCHEMA, names.subList(4, 7), values.subList(4, 7), 0.7, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequest, "")) + .thenReturn(new FakeBigQueryServerStream<>(responses)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("readStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.4, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.8, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.9, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the stream, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testFractionConsumedWithArrowAndMultipleStreamsInBundle() throws Exception { + ReadSession readSession = + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(); + + ReadRowsRequest expectedRequestOne = + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(); + ReadRowsRequest expectedRequestTwo = + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(); + + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responsesOne = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.5), + createResponseArrow(ARROW_SCHEMA, Lists.newArrayList(), Lists.newArrayList(), 0.5, 0.5), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.5, 1.0)); + + List responsesTwo = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(4, 7), values.subList(4, 7), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows(expectedRequestOne, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesOne)); + when(fakeStorageClient.readRows(expectedRequestTwo, "")) + .thenReturn(new FakeBigQueryServerStream<>(responsesTwo)); + + List streamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader reader = streamSource.createReader(options); + + // Before call to BoundedReader#start, fraction consumed must be zero. + assertEquals(0.0, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.start()); // Reads A. + assertEquals(0.125, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads B. + assertEquals(0.25, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads C. + assertEquals(0.375, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads D. + assertEquals(0.5, reader.getFractionConsumed(), DELTA); + + assertTrue(reader.advance()); // Reads E. + assertEquals(0.6666666666666666, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads F. + assertEquals(0.8333333333333333, reader.getFractionConsumed(), DELTA); + assertTrue(reader.advance()); // Reads G. + assertEquals(1.0, reader.getFractionConsumed(), DELTA); + + assertFalse(reader.advance()); // Reaches the end. + + // We are done with the streams, so we should report 100% consumption. + assertEquals(Double.valueOf(1.0), reader.getFractionConsumed()); + } + + @Test + public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary + // StreamBundle. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNotNull(secondarySource); + BoundedReader secondary = secondarySource.createReader(options); + + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); + + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertFalse(secondary.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsInBundle() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List responses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(4, 6), values.subList(4, 6), 0.0, 1.0)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + + List primaryStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); + + BoundedSource primarySource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + primaryStreamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + BoundedReader primaryReader = primarySource.createReader(options); + assertTrue(primaryReader.start()); + assertEquals("A", primaryReader.getCurrent().get("name")); + + // Should create two sources: the first with 1 stream, the second with 2. + BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + assertNotNull(secondarySource); + assertEquals("A", primaryReader.getCurrent().get("name")); + + assertTrue(primaryReader.advance()); + assertEquals("B", primaryReader.getCurrent().get("name")); + assertFalse(primaryReader.advance()); + + BoundedReader secondaryReader = secondarySource.createReader(options); + assertTrue(secondaryReader.start()); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + // Should create two sources: each with 1 stream. + BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + assertNotNull(tertiarySource); + assertEquals("C", secondaryReader.getCurrent().get("name")); + + assertTrue(secondaryReader.advance()); + assertEquals("D", secondaryReader.getCurrent().get("name")); + assertFalse(secondaryReader.advance()); + + BoundedReader tertiaryReader = tertiarySource.createReader(options); + assertTrue(tertiaryReader.start()); + assertEquals("E", tertiaryReader.getCurrent().get("name")); + + assertTrue(tertiaryReader.advance()); + assertEquals("F", tertiaryReader.getCurrent().get("name")); + assertFalse(tertiaryReader.advance()); + } + + @Test + public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow() + throws Exception { + List names = Arrays.asList("A", "B", "C", "D", "E", "F"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List parentResponses = + Lists.newArrayList( + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.25, 0.5), + createResponseArrow( + ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.5, 0.75)); + + StorageClient fakeStorageClient = mock(StorageClient.class); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(parentResponses)); + + // Mocks the split call. A response without a primary_stream and remainder_stream means + // that the split is not possible. + // Mocks the split call. + when(fakeStorageClient.splitReadStream( + SplitReadStreamRequest.newBuilder().setName("parentStream").setFraction(0.5f).build())) + .thenReturn( + SplitReadStreamResponse.newBuilder() + .setPrimaryStream(ReadStream.newBuilder().setName("primaryStream")) + .setRemainderStream(ReadStream.newBuilder().setName("remainderStream")) + .build()); + + // Mocks the ReadRows calls expected on the primary and residual streams. + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder() + .setReadStream("primaryStream") + // This test will read rows 0 and 1 from the parent before calling split, + // so we expect the primary read to start at offset 2. + .setOffset(2) + .build(), + "")) + .thenThrow( + new FailedPreconditionException( + "Given row offset is invalid for stream.", + new StatusRuntimeException(Status.FAILED_PRECONDITION), + GrpcStatusCode.of(Code.FAILED_PRECONDITION), + /* retryable = */ false)); + + List streamBundle = + Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + ReadSession.newBuilder() + .setName("readSession") + .setArrowSchema( + ArrowSchema.newBuilder() + .setSerializedSchema(serializeArrowSchema(ARROW_SCHEMA)) + .build()) + .setDataFormat(DataFormat.ARROW) + .build(), + streamBundle, + TABLE_SCHEMA, + new TableRowParser(), + TableRowJsonCoder.of(), + new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + + // Read a few records from the parent stream and ensure that records are returned in the + // prescribed order. + BoundedReader parent = streamSource.createReader(options); + assertTrue(parent.start()); + assertEquals("A", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("B", parent.getCurrent().get("name")); + + assertNull(parent.splitAtFraction(0.5)); + + // Verify that the parent source still works okay even after an unsuccessful split attempt. + assertTrue(parent.advance()); + assertEquals("C", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("D", parent.getCurrent().get("name")); + assertTrue(parent.advance()); + assertEquals("E", parent.getCurrent().get("name")); + assertFalse(parent.advance()); + } + + @Test + public void testActuateProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertTrue(read.supportsProjectionPushdown()); + PTransform> pushdownT = + read.actuateProjectionPushdown( + ImmutableMap.of(new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo"))); + + TypedRead pushdownRead = (TypedRead) pushdownT; + assertEquals(Method.DIRECT_READ, pushdownRead.getMethod()); + assertThat(pushdownRead.getSelectedFields().get(), Matchers.containsInAnyOrder("foo")); + assertTrue(pushdownRead.getProjectionPushdownApplied()); + } + + @Test + public void testReadFromQueryDoesNotSupportProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .fromQuery("SELECT bar FROM `dataset.table`") + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertFalse(read.supportsProjectionPushdown()); + assertThrows( + IllegalArgumentException.class, + () -> + read.actuateProjectionPushdown( + ImmutableMap.of( + new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo")))); + } + + private static org.apache.arrow.vector.types.pojo.Field field( + String name, + boolean nullable, + ArrowType type, + org.apache.arrow.vector.types.pojo.Field... children) { + return new org.apache.arrow.vector.types.pojo.Field( + name, + new org.apache.arrow.vector.types.pojo.FieldType(nullable, type, null, null), + asList(children)); + } + + static org.apache.arrow.vector.types.pojo.Field field( + String name, ArrowType type, org.apache.arrow.vector.types.pojo.Field... children) { + return field(name, false, type, children); + } +} From 65fea1a417da7ddb99ece9bc1f718ae04ef1bb4f Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 2 Feb 2023 18:28:22 +0000 Subject: [PATCH 19/32] v2 using OffsetBasedSource and OffsetBasedReader --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 +- .../bigquery/BigQueryStorageSourceBase.java | 2 +- .../BigQueryStorageStreamBundleSource.java | 111 +++++++++++------- ...StorageReadWithStreamBundleSourceTest.java | 52 +++++--- 4 files changed, 106 insertions(+), 62 deletions(-) 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 5290c99ae20b..762374f67ce5 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 @@ -1730,7 +1730,8 @@ public void processElement(ProcessContext c) throws Exception { tableSchema, getParseFn(), outputCoder, - getBigQueryServices()); + getBigQueryServices(), + 1L); // Read all of the data from the stream. In the event that this work // item fails and is rescheduled, the same rows will be returned in 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 2d5202aa0443..f0abd90682c4 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 @@ -227,7 +227,7 @@ public List> split( if (streamIndex % streamsPerBundle == 0) { sources.add( BigQueryStorageStreamBundleSource.create( - readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices)); + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); streamBundle = Lists.newArrayList(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index fb788eb4920b..a6385592e0c8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -35,7 +35,7 @@ import java.util.NoSuchElementException; import org.apache.beam.runners.core.metrics.ServiceCallMetric; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.OffsetBasedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; import org.apache.beam.sdk.metrics.Metrics; @@ -48,7 +48,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class BigQueryStorageStreamBundleSource extends BoundedSource { +class BigQueryStorageStreamBundleSource extends OffsetBasedSource { private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); @@ -59,14 +59,16 @@ public static BigQueryStorageStreamBundleSource create( TableSchema tableSchema, SerializableFunction parseFn, Coder outputCoder, - BigQueryServices bqServices) { + BigQueryServices bqServices, + long minBundleSize) { return new BigQueryStorageStreamBundleSource<>( readSession, streamBundle, toJsonString(Preconditions.checkArgumentNotNull(tableSchema, "tableSchema")), parseFn, outputCoder, - bqServices); + bqServices, + minBundleSize); } /** @@ -75,7 +77,7 @@ public static BigQueryStorageStreamBundleSource create( */ public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { return new BigQueryStorageStreamBundleSource<>( - readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices); + readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices, getMinBundleSize()); } private final ReadSession readSession; @@ -91,7 +93,9 @@ private BigQueryStorageStreamBundleSource( String jsonTableSchema, SerializableFunction parseFn, Coder outputCoder, - BigQueryServices bqServices) { + BigQueryServices bqServices, + long minBundleSize) { + super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); this.jsonTableSchema = Preconditions.checkArgumentNotNull(jsonTableSchema, "jsonTableSchema"); @@ -124,13 +128,24 @@ public long getEstimatedSizeBytes(PipelineOptions options) { } @Override - public List> split( + public List> split( long desiredBundleSizeBytes, PipelineOptions options) { // A stream source can't be split without reading from it due to server-side liquid sharding. // TODO: Implement dynamic work rebalancing. return ImmutableList.of(this); } + @Override + public long getMaxEndOffset(PipelineOptions options) throws Exception { + return this.streamBundle.size(); + } + + @Override + public OffsetBasedSource createSourceForSubrange(long start, long end) { + List newStreamBundle = streamBundle.subList((int) start, (int) end); + return fromExisting(newStreamBundle); + } + @Override public BigQueryStorageStreamBundleReader createReader(PipelineOptions options) throws IOException { @@ -142,7 +157,7 @@ public BigQueryStorageStreamBundleReader createReader(PipelineOptions options // return readStream.toString(); // } - public static class BigQueryStorageStreamBundleReader extends BoundedSource.BoundedReader { + public static class BigQueryStorageStreamBundleReader extends OffsetBasedReader { private final BigQueryStorageReader reader; private final SerializableFunction parseFn; private final StorageClient storageClient; @@ -168,6 +183,7 @@ public static class BigQueryStorageStreamBundleReader extends BoundedSource.B private BigQueryStorageStreamBundleReader( BigQueryStorageStreamBundleSource source, BigQueryOptions options) throws IOException { + super(source); this.source = source; this.reader = BigQueryStorageReaderFactory.getReader(source.readSession); this.parseFn = source.parseFn; @@ -191,18 +207,31 @@ public T getCurrent() throws NoSuchElementException { } @Override - public synchronized boolean start() throws IOException { + protected long getCurrentOffset() throws NoSuchElementException { + return currentStreamIndex; + } + + @Override + protected boolean isAtSplitPoint() throws NoSuchElementException { + if (currentOffset == 0) { + return true; + } + return false; + } + + @Override + public boolean startImpl() throws IOException { return readNextStream(); } @Override - public synchronized boolean advance() throws IOException { + public boolean advanceImpl() throws IOException { // Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } - private synchronized boolean readNextStream() throws IOException { + private boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; @@ -224,7 +253,7 @@ private synchronized boolean readNextStream() throws IOException { } // @RequiresNonNull("responseIterator") - private synchronized boolean readNextRecord() throws IOException { + private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { return false; @@ -317,35 +346,35 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } - @Override - @SuppressWarnings("ReturnValueIgnored") - public @Nullable BoundedSource splitAtFraction(double fraction) { - int streamCountInBundle = source.streamBundle.size(); - double splitIndex = streamCountInBundle * fraction; - if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { - // The reader has moved past the requested split point. - // NOTE: We do not split below the granularity of a stream. - Metrics.counter( - BigQueryStorageStreamBundleReader.class, - "split-at-fraction-calls-failed-due-to-impossible-split-point") - .inc(); - LOG.info( - "BigQuery Storage API Session {} cannot be split at {}.", - source.readSession.getName(), - fraction); - return null; - } - // Splitting the remainder Streams into a new StreamBundle. - List remainderStreamBundle = - new ArrayList<>( - source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); - // Updating the primary StreamBundle. - source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); - Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") - .inc(); - LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); - return source.fromExisting(remainderStreamBundle); - } + // @Override + // @SuppressWarnings("ReturnValueIgnored") + // public @Nullable OffsetBasedSource splitAtFraction(double fraction) { + // int streamCountInBundle = source.streamBundle.size(); + // double splitIndex = streamCountInBundle * fraction; + // if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { + // // The reader has moved past the requested split point. + // // NOTE: We do not split below the granularity of a stream. + // Metrics.counter( + // BigQueryStorageStreamBundleReader.class, + // "split-at-fraction-calls-failed-due-to-impossible-split-point") + // .inc(); + // LOG.info( + // "BigQuery Storage API Session {} cannot be split at {}.", + // source.readSession.getName(), + // fraction); + // return null; + // } + // // Splitting the remainder Streams into a new StreamBundle. + // List remainderStreamBundle = + // new ArrayList<>( + // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); + // // Updating the primary StreamBundle. + // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); + // Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") + // .inc(); + // LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); + // return source.fromExisting(remainderStreamBundle); + // } @Override public synchronized Double getFractionConsumed() { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index bd0cb1124f8e..87dfe1891175 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -753,7 +753,8 @@ public void testStreamSourceEstimatedSizeBytes() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices()); + new FakeBigQueryServices(), + 1L); assertEquals(0, streamSource.getEstimatedSizeBytes(options)); } @@ -768,7 +769,8 @@ public void testStreamSourceSplit() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices()); + new FakeBigQueryServices(), + 1L); assertThat(streamSource.split(0, options), containsInAnyOrder(streamSource)); } @@ -821,7 +823,8 @@ public void testReadFromStreamSource() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); List rows = new ArrayList<>(); BigQueryStorageStreamBundleReader reader = streamSource.createReader(options); @@ -879,7 +882,8 @@ public void testFractionConsumedWithOneStreamInBundle() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -961,7 +965,8 @@ public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1027,7 +1032,8 @@ public void testStreamSourceSplitAtFractionNoOpWithOneStreamInBundle() throws Ex TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent stream and ensure that records are returned in the // prescribed order. @@ -1093,7 +1099,8 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the primary bundle and ensure the records are returned in // the prescribed order. @@ -1171,14 +1178,15 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader primaryReader = primarySource.createReader(options); assertTrue(primaryReader.start()); assertEquals("A", primaryReader.getCurrent().get("name")); // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); assertNotNull(secondarySource); assertEquals("A", primaryReader.getCurrent().get("name")); @@ -1250,7 +1258,8 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() thr TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent bundle and ensure the records are returned in // the prescribed order. @@ -1664,7 +1673,8 @@ public void testReadFromStreamSourceArrow() throws Exception { TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); List rows = new ArrayList<>(); BoundedReader reader = streamSource.createReader(options); @@ -1715,7 +1725,8 @@ public void testFractionConsumedWithArrowAndOneStreamInBundle() throws Exception TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1792,7 +1803,8 @@ public void testFractionConsumedWithArrowAndMultipleStreamsInBundle() throws Exc TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader reader = streamSource.createReader(options); @@ -1860,7 +1872,8 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the primary bundle and ensure that records are returned in the // prescribed order. @@ -1871,8 +1884,7 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("B", primary.getCurrent().get("name")); // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary - // StreamBundle. + // secondary StreamBundle. BoundedSource secondarySource = primary.splitAtFraction(0.5); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); @@ -1931,14 +1943,15 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); BoundedReader primaryReader = primarySource.createReader(options); assertTrue(primaryReader.start()); assertEquals("A", primaryReader.getCurrent().get("name")); // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.25f); + BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); assertNotNull(secondarySource); assertEquals("A", primaryReader.getCurrent().get("name")); @@ -2028,7 +2041,8 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow( TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), - new FakeBigQueryServices().withStorageClient(fakeStorageClient)); + new FakeBigQueryServices().withStorageClient(fakeStorageClient), + 1L); // Read a few records from the parent stream and ensure that records are returned in the // prescribed order. From d71c3048e2338b79f87af904de815bf65de3c572 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 3 Feb 2023 00:06:34 +0000 Subject: [PATCH 20/32] Updating tests to have more sensible mock values. --- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 5 +- .../BigQueryStorageStreamBundleSource.java | 63 +--- ...StorageReadWithStreamBundleSourceTest.java | 323 +++++++++--------- 3 files changed, 183 insertions(+), 208 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 02cb7cd2fba8..d9d94fda76d9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -164,8 +164,9 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); - @Description("If set, BigQueryIO.Read will default to using the StreamBundle based" - + "implementation of the Read API Source") + @Description( + "If set, BigQueryIO.Read will default to using the StreamBundle based" + + "implementation of the Read API Source") @Default.Boolean(false) Boolean getUseReadAPIStreamBundleSource(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index a6385592e0c8..99f4629db301 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -29,7 +29,6 @@ import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.ReadStream; import java.io.IOException; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -38,21 +37,18 @@ import org.apache.beam.sdk.io.OffsetBasedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.BigQueryServerStream; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; -import org.apache.beam.sdk.metrics.Metrics; 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.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; class BigQueryStorageStreamBundleSource extends OffsetBasedSource { - private static final Logger LOG = - LoggerFactory.getLogger(BigQueryStorageStreamBundleSource.class); - public static BigQueryStorageStreamBundleSource create( ReadSession readSession, List streamBundle, @@ -77,7 +73,13 @@ public static BigQueryStorageStreamBundleSource create( */ public BigQueryStorageStreamBundleSource fromExisting(List newStreamBundle) { return new BigQueryStorageStreamBundleSource<>( - readSession, newStreamBundle, jsonTableSchema, parseFn, outputCoder, bqServices, getMinBundleSize()); + readSession, + newStreamBundle, + jsonTableSchema, + parseFn, + outputCoder, + bqServices, + getMinBundleSize()); } private final ReadSession readSession; @@ -152,12 +154,10 @@ public BigQueryStorageStreamBundleReader createReader(PipelineOptions options return new BigQueryStorageStreamBundleReader<>(this, options.as(BigQueryOptions.class)); } - // @Override - // public String toString() { - // return readStream.toString(); - // } - public static class BigQueryStorageStreamBundleReader extends OffsetBasedReader { + private static final Logger LOG = + LoggerFactory.getLogger(BigQueryStorageStreamBundleReader.class); + private final BigQueryStorageReader reader; private final SerializableFunction parseFn; private final StorageClient storageClient; @@ -226,7 +226,7 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { - // Preconditions.checkStateNotNull(responseIterator); + Preconditions.checkStateNotNull(responseIterator); currentOffset++; return readNextRecord(); } @@ -246,16 +246,17 @@ private boolean readNextStream() throws IOException { serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( "Started BigQuery Storage API read from stream {}.", - source.streamBundle.get(0).getName()); + source.streamBundle.get(currentStreamIndex).getName()); responseStream = storageClient.readRows(request, source.readSession.getTable()); responseIterator = responseStream.iterator(); return readNextRecord(); } - // @RequiresNonNull("responseIterator") + @RequiresNonNull("responseIterator") private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { + LOG.info("Received null responseIterator for stream {}", currentStreamIndex); return false; } while (reader.readyForNextReadResponse()) { @@ -335,8 +336,8 @@ private boolean readNextRecord() throws IOException { public synchronized void close() { // Because superclass cannot have preconditions around these variables, cannot use // @RequiresNonNull - // Preconditions.checkStateNotNull(storageClient); - // Preconditions.checkStateNotNull(reader); + Preconditions.checkStateNotNull(storageClient); + Preconditions.checkStateNotNull(reader); storageClient.close(); reader.close(); } @@ -346,36 +347,6 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { return source; } - // @Override - // @SuppressWarnings("ReturnValueIgnored") - // public @Nullable OffsetBasedSource splitAtFraction(double fraction) { - // int streamCountInBundle = source.streamBundle.size(); - // double splitIndex = streamCountInBundle * fraction; - // if (streamCountInBundle <= 1 || currentStreamIndex >= splitIndex) { - // // The reader has moved past the requested split point. - // // NOTE: We do not split below the granularity of a stream. - // Metrics.counter( - // BigQueryStorageStreamBundleReader.class, - // "split-at-fraction-calls-failed-due-to-impossible-split-point") - // .inc(); - // LOG.info( - // "BigQuery Storage API Session {} cannot be split at {}.", - // source.readSession.getName(), - // fraction); - // return null; - // } - // // Splitting the remainder Streams into a new StreamBundle. - // List remainderStreamBundle = - // new ArrayList<>( - // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle)); - // // Updating the primary StreamBundle. - // source.streamBundle.subList((int) Math.ceil(splitIndex), streamCountInBundle).clear(); - // Metrics.counter(BigQueryStorageStreamBundleReader.class, "split-at-fraction-calls-successful") - // .inc(); - // LOG.info("Successfully split BigQuery Storage API StreamBundle at {}.", fraction); - // return source.fromExisting(remainderStreamBundle); - // } - @Override public synchronized Double getFractionConsumed() { return fractionConsumed; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index 87dfe1891175..1fba6423b897 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -31,8 +31,6 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; -import com.google.api.gax.grpc.GrpcStatusCode; -import com.google.api.gax.rpc.FailedPreconditionException; import com.google.api.services.bigquery.model.Streamingbuffer; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableFieldSchema; @@ -49,14 +47,9 @@ import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.ReadStream; -import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; -import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; import com.google.cloud.bigquery.storage.v1.StreamStats; import com.google.cloud.bigquery.storage.v1.StreamStats.Progress; import com.google.protobuf.ByteString; -import io.grpc.Status; -import io.grpc.Status.Code; -import io.grpc.StatusRuntimeException; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.math.BigInteger; @@ -1066,15 +1059,19 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.6), createResponse( - AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.6, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( - createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), - 0.50, - 0.75)); + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)), + 0.0, + 1.0), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("G", 7, AVRO_SCHEMA)), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1083,11 +1080,15 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( ReadStream.newBuilder().setName("readStream1").build(), - ReadStream.newBuilder().setName("readStream2").build()); + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( @@ -1102,21 +1103,21 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure the records are returned in - // the prescribed order. + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. BoundedReader primary = primarySource.createReader(options); assertTrue(primary.start()); assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Now split the StreamBundle and ensure that the returned source points to a non-null secondary - // StreamBundle - BoundedSource secondarySource = primary.splitAtFraction(0.5); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primary.advance()); assertEquals("C", primary.getCurrent().get("name")); assertFalse(primary.advance()); @@ -1124,7 +1125,11 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws assertEquals("D", secondary.getCurrent().get("name")); assertTrue(secondary.advance()); assertEquals("E", secondary.getCurrent().get("name")); - assertFalse(secondary.advance()); + assertTrue(secondary.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertTrue((secondary.advance())); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); } @Test @@ -1136,31 +1141,30 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.6), createResponse( - AVRO_SCHEMA, - Lists.newArrayList( - createRecord("C", 3, AVRO_SCHEMA), createRecord("D", 4, AVRO_SCHEMA)), - 0.25, - 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.6, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( - createRecord("E", 5, AVRO_SCHEMA), createRecord("F", 6, AVRO_SCHEMA)), - 0.50, - 0.75)); + createRecord("D", 4, AVRO_SCHEMA), + createRecord("E", 5, AVRO_SCHEMA), + createRecord("F", 6, AVRO_SCHEMA)), + 0.0, + 1.0), + createResponse( + AVRO_SCHEMA, Lists.newArrayList(createRecord("G", 7, AVRO_SCHEMA)), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); - when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( @@ -1168,7 +1172,7 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() ReadStream.newBuilder().setName("readStream2").build(), ReadStream.newBuilder().setName("readStream3").build()); - BoundedSource primarySource = + BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -1181,39 +1185,44 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - BoundedReader primaryReader = primarySource.createReader(options); - assertTrue(primaryReader.start()); - assertEquals("A", primaryReader.getCurrent().get("name")); + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only + // occur when fraction >= 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); - assertEquals("A", primaryReader.getCurrent().get("name")); + BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primaryReader.advance()); - assertEquals("B", primaryReader.getCurrent().get("name")); - assertFalse(primaryReader.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); - BoundedReader secondaryReader = secondarySource.createReader(options); - assertTrue(secondaryReader.start()); - assertEquals("C", secondaryReader.getCurrent().get("name")); + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); - // Should create two sources: each with 1 stream. - BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle again, and ensure that the returned source points to a non-null + // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only + // occur when fraction >= 0.5. + BoundedSource tertiarySource = secondary.splitAtFraction(0.5); assertNotNull(tertiarySource); - assertEquals("C", secondaryReader.getCurrent().get("name")); - - assertTrue(secondaryReader.advance()); - assertEquals("D", secondaryReader.getCurrent().get("name")); - assertFalse(secondaryReader.advance()); + BoundedReader tertiary = tertiarySource.createReader(options); - BoundedReader tertiaryReader = tertiarySource.createReader(options); - assertTrue(tertiaryReader.start()); - assertEquals("E", tertiaryReader.getCurrent().get("name")); + assertEquals("F", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - assertTrue(tertiaryReader.advance()); - assertEquals("F", tertiaryReader.getCurrent().get("name")); - assertFalse(tertiaryReader.advance()); + assertTrue(tertiary.start()); + assertEquals("G", tertiary.getCurrent().get("name")); + assertFalse((tertiary.advance())); } @Test @@ -1225,15 +1234,15 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPoint() thr Lists.newArrayList( createRecord("A", 1, AVRO_SCHEMA), createRecord("B", 2, AVRO_SCHEMA)), 0.0, - 0.25), + 0.66), createResponse( - AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.25, 0.50), + AVRO_SCHEMA, Lists.newArrayList(createRecord("C", 3, AVRO_SCHEMA)), 0.66, 1.0), createResponse( AVRO_SCHEMA, Lists.newArrayList( createRecord("D", 4, AVRO_SCHEMA), createRecord("E", 5, AVRO_SCHEMA)), - 0.50, - 0.75)); + 0.0, + 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1843,7 +1852,8 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() Lists.newArrayList( createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); + createResponseArrow(ARROW_SCHEMA, names.subList(3, 6), values.subList(3, 6), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( @@ -1852,11 +1862,15 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + when(fakeStorageClient.readRows( + ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( ReadStream.newBuilder().setName("readStream1").build(), - ReadStream.newBuilder().setName("readStream2").build()); + ReadStream.newBuilder().setName("readStream2").build(), + ReadStream.newBuilder().setName("readStream3").build()); BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( @@ -1882,14 +1896,14 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); // Now split the StreamBundle, and ensure that the returned source points to a non-null // secondary StreamBundle. - BoundedSource secondarySource = primary.splitAtFraction(0.5); + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primary.advance()); assertEquals("C", primary.getCurrent().get("name")); assertFalse(primary.advance()); @@ -1897,31 +1911,35 @@ public void testStreamSourceSplitAtFractionWithArrowAndMultipleStreamsInBundle() assertEquals("D", secondary.getCurrent().get("name")); assertTrue(secondary.advance()); assertEquals("E", secondary.getCurrent().get("name")); - assertFalse(secondary.advance()); + assertTrue(secondary.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertTrue((secondary.advance())); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); } @Test public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsInBundle() throws Exception { - List names = Arrays.asList("A", "B", "C", "D", "E", "F"); - List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); + List names = Arrays.asList("A", "B", "C", "D", "E", "F", "G"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L); List responses = Lists.newArrayList( - createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(2, 4), values.subList(2, 4), 0.0, 1.0), - createResponseArrow(ARROW_SCHEMA, names.subList(4, 6), values.subList(4, 6), 0.0, 1.0)); + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.6), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.6, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 6), values.subList(3, 6), 0.0, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(6, 7), values.subList(6, 7), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); - when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 1))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(1, 2))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); when(fakeStorageClient.readRows( ReadRowsRequest.newBuilder().setReadStream("readStream3").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(3, 4))); List primaryStreamBundle = Lists.newArrayList( @@ -1929,7 +1947,7 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn ReadStream.newBuilder().setName("readStream2").build(), ReadStream.newBuilder().setName("readStream3").build()); - BoundedSource primarySource = + BigQueryStorageStreamBundleSource primarySource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -1946,88 +1964,71 @@ public void testStreamSourceSplitAtFractionRepeatedWithArrowAndMultipleStreamsIn new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - BoundedReader primaryReader = primarySource.createReader(options); - assertTrue(primaryReader.start()); - assertEquals("A", primaryReader.getCurrent().get("name")); + // Read a few records from the primary bundle and ensure that records are returned in the + // prescribed order. + BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); - // Should create two sources: the first with 1 stream, the second with 2. - BoundedSource secondarySource = primaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle, and ensure that the returned source points to a non-null + // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only + // occur when fraction >= 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.35); assertNotNull(secondarySource); - assertEquals("A", primaryReader.getCurrent().get("name")); + BoundedReader secondary = secondarySource.createReader(options); - assertTrue(primaryReader.advance()); - assertEquals("B", primaryReader.getCurrent().get("name")); - assertFalse(primaryReader.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertFalse(primary.advance()); - BoundedReader secondaryReader = secondarySource.createReader(options); - assertTrue(secondaryReader.start()); - assertEquals("C", secondaryReader.getCurrent().get("name")); + assertTrue(secondary.start()); + assertEquals("D", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); + assertEquals("E", secondary.getCurrent().get("name")); + assertTrue(secondary.advance()); - // Should create two sources: each with 1 stream. - BoundedSource tertiarySource = secondaryReader.splitAtFraction(0.5f); + // Now split the StreamBundle again, and ensure that the returned source points to a non-null + // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only + // occur when fraction >= 0.5. + BoundedSource tertiarySource = secondary.splitAtFraction(0.5); assertNotNull(tertiarySource); - assertEquals("C", secondaryReader.getCurrent().get("name")); + BoundedReader tertiary = tertiarySource.createReader(options); - assertTrue(secondaryReader.advance()); - assertEquals("D", secondaryReader.getCurrent().get("name")); - assertFalse(secondaryReader.advance()); + assertEquals("F", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - BoundedReader tertiaryReader = tertiarySource.createReader(options); - assertTrue(tertiaryReader.start()); - assertEquals("E", tertiaryReader.getCurrent().get("name")); - - assertTrue(tertiaryReader.advance()); - assertEquals("F", tertiaryReader.getCurrent().get("name")); - assertFalse(tertiaryReader.advance()); + assertTrue(tertiary.start()); + assertEquals("G", tertiary.getCurrent().get("name")); + assertFalse((tertiary.advance())); } @Test public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow() throws Exception { - List names = Arrays.asList("A", "B", "C", "D", "E", "F"); - List values = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L); - List parentResponses = + List names = Arrays.asList("A", "B", "C", "D", "E"); + List values = Arrays.asList(1L, 2L, 3L, 4L, 5L); + List responses = Lists.newArrayList( - createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.25), - createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.25, 0.5), - createResponseArrow( - ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.5, 0.75)); + createResponseArrow(ARROW_SCHEMA, names.subList(0, 2), values.subList(0, 2), 0.0, 0.66), + createResponseArrow(ARROW_SCHEMA, names.subList(2, 3), values.subList(2, 3), 0.66, 1.0), + createResponseArrow(ARROW_SCHEMA, names.subList(3, 5), values.subList(3, 5), 0.0, 1.0)); StorageClient fakeStorageClient = mock(StorageClient.class); when(fakeStorageClient.readRows( - ReadRowsRequest.newBuilder().setReadStream("parentStream").build(), "")) - .thenReturn(new FakeBigQueryServerStream<>(parentResponses)); - - // Mocks the split call. A response without a primary_stream and remainder_stream means - // that the split is not possible. - // Mocks the split call. - when(fakeStorageClient.splitReadStream( - SplitReadStreamRequest.newBuilder().setName("parentStream").setFraction(0.5f).build())) - .thenReturn( - SplitReadStreamResponse.newBuilder() - .setPrimaryStream(ReadStream.newBuilder().setName("primaryStream")) - .setRemainderStream(ReadStream.newBuilder().setName("remainderStream")) - .build()); - - // Mocks the ReadRows calls expected on the primary and residual streams. + ReadRowsRequest.newBuilder().setReadStream("readStream1").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(0, 2))); when(fakeStorageClient.readRows( - ReadRowsRequest.newBuilder() - .setReadStream("primaryStream") - // This test will read rows 0 and 1 from the parent before calling split, - // so we expect the primary read to start at offset 2. - .setOffset(2) - .build(), - "")) - .thenThrow( - new FailedPreconditionException( - "Given row offset is invalid for stream.", - new StatusRuntimeException(Status.FAILED_PRECONDITION), - GrpcStatusCode.of(Code.FAILED_PRECONDITION), - /* retryable = */ false)); + ReadRowsRequest.newBuilder().setReadStream("readStream2").build(), "")) + .thenReturn(new FakeBigQueryServerStream<>(responses.subList(2, 3))); - List streamBundle = - Lists.newArrayList(ReadStream.newBuilder().setName("parentStream").build()); - BigQueryStorageStreamBundleSource streamSource = + List parentStreamBundle = + Lists.newArrayList( + ReadStream.newBuilder().setName("readStream1").build(), + ReadStream.newBuilder().setName("readStream2").build()); + + BigQueryStorageStreamBundleSource streamBundleSource = BigQueryStorageStreamBundleSource.create( ReadSession.newBuilder() .setName("readSession") @@ -2037,31 +2038,33 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow( .build()) .setDataFormat(DataFormat.ARROW) .build(), - streamBundle, + parentStreamBundle, TABLE_SCHEMA, new TableRowParser(), TableRowJsonCoder.of(), new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the parent stream and ensure that records are returned in the - // prescribed order. - BoundedReader parent = streamSource.createReader(options); - assertTrue(parent.start()); - assertEquals("A", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("B", parent.getCurrent().get("name")); - - assertNull(parent.splitAtFraction(0.5)); - - // Verify that the parent source still works okay even after an unsuccessful split attempt. - assertTrue(parent.advance()); - assertEquals("C", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("D", parent.getCurrent().get("name")); - assertTrue(parent.advance()); - assertEquals("E", parent.getCurrent().get("name")); - assertFalse(parent.advance()); + // Read a few records from the parent bundle and ensure the records are returned in + // the prescribed order. + BoundedReader primary = streamBundleSource.createReader(options); + assertTrue(primary.start()); + assertEquals("A", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("B", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); + assertTrue(primary.advance()); + assertEquals("D", primary.getCurrent().get("name")); + + // We attempt to split the StreamBundle after starting to read the contents of the second + // stream. + BoundedSource secondarySource = primary.splitAtFraction(0.5); + assertNull(secondarySource); + + assertTrue(primary.advance()); + assertEquals("E", primary.getCurrent().get("name")); + assertFalse(primary.advance()); } @Test From a87b56b6dca85f5f87f0a34d32962a8a7a6e3a21 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 3 Feb 2023 01:17:02 +0000 Subject: [PATCH 21/32] Updated BqOption flag. --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java | 6 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java | 6 +++--- .../BigQueryIOStorageReadWithStreamBundleSourceTest.java | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) 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 762374f67ce5..353d21c702f2 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 @@ -1418,7 +1418,7 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - if (bqOptions.getUseReadAPIStreamBundleSource()) { + if (bqOptions.getEnableBundling()) { return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index d9d94fda76d9..45e6ca9f134d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -165,10 +165,10 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); @Description( - "If set, BigQueryIO.Read will default to using the StreamBundle based" + "If set, BigQueryIO.Read will use the StreamBundle based" + "implementation of the Read API Source") @Default.Boolean(false) - Boolean getUseReadAPIStreamBundleSource(); + Boolean getEnableBundling(); - void setUseReadAPIStreamBundleSource(Boolean value); + void setEnableBundling(Boolean value); } 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 f0abd90682c4..5573fbdfdde0 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 @@ -137,7 +137,7 @@ public List> split( // an appropriate number of streams for the Session to produce reasonable throughput. // This is required when using the Read API Source V2. int streamCount = 0; - if (!bqOptions.getUseReadAPIStreamBundleSource()) { + if (!bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { long tableSizeBytes = (targetTable != null) ? targetTable.getNumBytes() : 0; streamCount = (int) Math.min(tableSizeBytes / desiredBundleSizeBytes, MAX_SPLIT_COUNT); @@ -178,7 +178,7 @@ public List> split( LOG.info( "readSession.getEstimatedTotalBytesScanned(): '{}'", readSession.getEstimatedTotalBytesScanned()); - if (bqOptions.getUseReadAPIStreamBundleSource()) { + if (bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { bytesPerStream = (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); @@ -210,7 +210,7 @@ public List> split( targetTable); // TODO: this is inconsistent with method above, where it can be null TableSchema trimmedSchema = BigQueryAvroUtils.trimBigQueryTableSchema(targetTable.getSchema(), sessionSchema); - if (!bqOptions.getUseReadAPIStreamBundleSource()) { + if (!bqOptions.getEnableBundling()) { List> sources = Lists.newArrayList(); for (ReadStream readStream : readSession.getStreamsList()) { sources.add( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index 1fba6423b897..eec0f5104955 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -123,7 +123,7 @@ /** * Tests for {@link BigQueryIO#readTableRows() using {@link Method#DIRECT_READ}} AND {@link - * BigQueryOptions#setUseReadAPIStreamBundleSource(Boolean)} set to True. + * BigQueryOptions#setEnableBundling(Boolean)} (Boolean)} set to True. */ @RunWith(JUnit4.class) public class BigQueryIOStorageReadWithStreamBundleSourceTest { @@ -155,7 +155,7 @@ public void evaluate() throws Throwable { options .as(BigQueryOptions.class) .setTempLocation(testFolder.getRoot().getAbsolutePath()); - options.as(BigQueryOptions.class).setUseReadAPIStreamBundleSource(true); + options.as(BigQueryOptions.class).setEnableBundling(true); p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } From 572467cf4184afa724f4cf877f8198d94eb751b7 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 01:43:24 +0000 Subject: [PATCH 22/32] Simplifying `fractionConsumed` calculation. --- .../BigQueryStorageStreamBundleSource.java | 25 ++++++++----------- ...StorageReadWithStreamBundleSourceTest.java | 2 +- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 99f4629db301..c1c8a98f8edb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -172,7 +172,7 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead // Values used for progress reporting. private double fractionConsumed; - private double fractionOfStreamsConsumed; + private double progressAtResponseStart; private double progressAtResponseEnd; private long rowsConsumedFromCurrentResponse; @@ -191,7 +191,6 @@ private BigQueryStorageStreamBundleReader( this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; this.fractionConsumed = 0d; - this.fractionOfStreamsConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -232,7 +231,10 @@ public boolean advanceImpl() throws IOException { } private boolean readNextStream() throws IOException { - BigQueryStorageStreamBundleSource source = getCurrentSource(); + BigQueryStorageStreamBundleSource source; + synchronized (this) { + source = getCurrentSource(); + } if (currentStreamIndex == source.streamBundle.size()) { fractionConsumed = 1d; return false; @@ -316,19 +318,14 @@ private boolean readNextRecord() throws IOException { // the rows in the current response have been consumed. rowsConsumedFromCurrentResponse++; - fractionConsumed = + double fractionOfCurrentStreamConsumed = progressAtResponseStart - + (progressAtResponseEnd - progressAtResponseStart) - * rowsConsumedFromCurrentResponse - * 1.0 - / totalRowsInCurrentResponse; - + + ((progressAtResponseEnd - progressAtResponseStart) + * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); // Assuming that each stream in the StreamBundle has approximately the same amount of data and - // normalizing the value of fractionConsumed. - fractionConsumed = fractionConsumed / source.streamBundle.size(); - - fractionOfStreamsConsumed = (double) currentStreamIndex / source.streamBundle.size(); - fractionConsumed += fractionOfStreamsConsumed; + // NORMALIZING the value of fractionConsumed. + fractionConsumed = + (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index eec0f5104955..d7ce32326ac5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -932,7 +932,7 @@ public void testFractionConsumedWithMultipleStreamsInBundle() throws Exception { List responsesOne = Lists.newArrayList( createResponse(AVRO_SCHEMA, records.subList(0, 2), 0.0, 0.5), - // Some responses may contain zero results, so we must ensure that we can are resilient + // Some responses may contain zero results, so we must ensure that we are resilient // to such responses. createResponse(AVRO_SCHEMA, Lists.newArrayList(), 0.5, 0.5), createResponse(AVRO_SCHEMA, records.subList(2, 4), 0.5, 1.0)); From 9778b354945eb944c2501730cd8491e8e6002511 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 02:22:58 +0000 Subject: [PATCH 23/32] Better variable names. --- .../BigQueryStorageStreamBundleSource.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index c1c8a98f8edb..09e060094584 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -171,7 +171,7 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead private long currentOffset; // Values used for progress reporting. - private double fractionConsumed; + private double fractionOfStreamBundleConsumed; private double progressAtResponseStart; private double progressAtResponseEnd; @@ -190,7 +190,7 @@ private BigQueryStorageStreamBundleReader( this.storageClient = source.bqServices.getStorageClient(options); this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); this.currentStreamIndex = 0; - this.fractionConsumed = 0d; + this.fractionOfStreamBundleConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; this.rowsConsumedFromCurrentResponse = 0L; @@ -230,13 +230,10 @@ public boolean advanceImpl() throws IOException { return readNextRecord(); } - private boolean readNextStream() throws IOException { - BigQueryStorageStreamBundleSource source; - synchronized (this) { - source = getCurrentSource(); - } + private synchronized boolean readNextStream() throws IOException { + BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamIndex == source.streamBundle.size()) { - fractionConsumed = 1d; + fractionOfStreamBundleConsumed = 1d; return false; } ReadRowsRequest request = @@ -322,9 +319,9 @@ private boolean readNextRecord() throws IOException { progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each stream in the StreamBundle has approximately the same amount of data and - // NORMALIZING the value of fractionConsumed. - fractionConsumed = + // Assuming that each Stream in the StreamBundle has approximately the same amount of data and + // NORMALIZING the value of fractionOfCurrentStreamConsumed. + fractionOfStreamBundleConsumed = (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } @@ -346,7 +343,7 @@ public synchronized BigQueryStorageStreamBundleSource getCurrentSource() { @Override public synchronized Double getFractionConsumed() { - return fractionConsumed; + return fractionOfStreamBundleConsumed; } } } From e780266f16d2b000e51d592c51f1f95e51fecac5 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Tue, 7 Feb 2023 18:08:06 +0000 Subject: [PATCH 24/32] Minor refactoring. --- .../BigQueryStorageStreamBundleSource.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 09e060094584..09e2b67f946d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -167,8 +167,8 @@ public static class BigQueryStorageStreamBundleReader extends OffsetBasedRead private @Nullable BigQueryServerStream responseStream = null; private @Nullable Iterator responseIterator = null; private @Nullable T current = null; - private int currentStreamIndex; - private long currentOffset; + private int currentStreamBundleIndex; + private long currentStreamOffset; // Values used for progress reporting. private double fractionOfStreamBundleConsumed; @@ -189,7 +189,7 @@ private BigQueryStorageStreamBundleReader( this.parseFn = source.parseFn; this.storageClient = source.bqServices.getStorageClient(options); this.tableSchema = fromJsonString(source.jsonTableSchema, TableSchema.class); - this.currentStreamIndex = 0; + this.currentStreamBundleIndex = 0; this.fractionOfStreamBundleConsumed = 0d; this.progressAtResponseStart = 0d; this.progressAtResponseEnd = 0d; @@ -207,12 +207,12 @@ public T getCurrent() throws NoSuchElementException { @Override protected long getCurrentOffset() throws NoSuchElementException { - return currentStreamIndex; + return currentStreamBundleIndex; } @Override protected boolean isAtSplitPoint() throws NoSuchElementException { - if (currentOffset == 0) { + if (currentStreamOffset == 0) { return true; } return false; @@ -226,26 +226,26 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { Preconditions.checkStateNotNull(responseIterator); - currentOffset++; + currentStreamOffset++; return readNextRecord(); } private synchronized boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); - if (currentStreamIndex == source.streamBundle.size()) { + if (currentStreamBundleIndex == source.streamBundle.size()) { fractionOfStreamBundleConsumed = 1d; return false; } ReadRowsRequest request = ReadRowsRequest.newBuilder() - .setReadStream(source.streamBundle.get(currentStreamIndex).getName()) - .setOffset(currentOffset) + .setReadStream(source.streamBundle.get(currentStreamBundleIndex).getName()) + .setOffset(currentStreamOffset) .build(); tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); LOG.info( "Started BigQuery Storage API read from stream {}.", - source.streamBundle.get(currentStreamIndex).getName()); + source.streamBundle.get(currentStreamBundleIndex).getName()); responseStream = storageClient.readRows(request, source.readSession.getTable()); responseIterator = responseStream.iterator(); return readNextRecord(); @@ -255,13 +255,13 @@ private synchronized boolean readNextStream() throws IOException { private boolean readNextRecord() throws IOException { Iterator responseIterator = this.responseIterator; if (responseIterator == null) { - LOG.info("Received null responseIterator for stream {}", currentStreamIndex); + LOG.info("Received null responseIterator for stream {}", currentStreamBundleIndex); return false; } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { - currentOffset = 0; - currentStreamIndex++; + currentStreamOffset = 0; + currentStreamBundleIndex++; return readNextStream(); } @@ -322,7 +322,7 @@ private boolean readNextRecord() throws IOException { // Assuming that each Stream in the StreamBundle has approximately the same amount of data and // NORMALIZING the value of fractionOfCurrentStreamConsumed. fractionOfStreamBundleConsumed = - (currentStreamIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); + (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; } From 03ca8d619572416f98f2d4112b551f601102edc3 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Wed, 8 Feb 2023 20:57:27 +0000 Subject: [PATCH 25/32] Added a synchronized block in readNextRecord(). Also added comments to explain how OffsetBasedSource + RangeTracker is used. --- .../BigQueryStorageStreamBundleSource.java | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 09e2b67f946d..de78b4699716 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -97,6 +97,9 @@ private BigQueryStorageStreamBundleSource( Coder outputCoder, BigQueryServices bqServices, long minBundleSize) { + // The underlying OffsetBasedSource (and RangeTracker) operate only on the StreamBundle and NOT + // the Streams that constitute the StreamBundle. More specifically, the offsets in the + // OffsetBasedSource are indices for the StreamBundle List. super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); @@ -212,6 +215,8 @@ protected long getCurrentOffset() throws NoSuchElementException { @Override protected boolean isAtSplitPoint() throws NoSuchElementException { + // The start of every Stream within a StreamBundle is being defined as a split point. This + // implies that we cannot split below the granularity of a Stream if (currentStreamOffset == 0) { return true; } @@ -226,7 +231,7 @@ public boolean startImpl() throws IOException { @Override public boolean advanceImpl() throws IOException { Preconditions.checkStateNotNull(responseIterator); - currentStreamOffset++; + currentStreamOffset += totalRowsInCurrentResponse; return readNextRecord(); } @@ -239,7 +244,6 @@ private synchronized boolean readNextStream() throws IOException { ReadRowsRequest request = ReadRowsRequest.newBuilder() .setReadStream(source.streamBundle.get(currentStreamBundleIndex).getName()) - .setOffset(currentStreamOffset) .build(); tableReference = BigQueryUtils.toTableReference(source.readSession.getTable()); serviceCallMetric = BigQueryUtils.readCallMetric(tableReference); @@ -260,9 +264,11 @@ private boolean readNextRecord() throws IOException { } while (reader.readyForNextReadResponse()) { if (!responseIterator.hasNext()) { - currentStreamOffset = 0; - currentStreamBundleIndex++; - return readNextStream(); + synchronized (this) { + currentStreamOffset = 0; + currentStreamBundleIndex++; + return readNextStream(); + } } ReadRowsResponse response; @@ -301,7 +307,6 @@ private boolean readNextRecord() throws IOException { 0f <= progressAtResponseEnd && progressAtResponseEnd <= 1f, "Progress at response end (%s) is not in the range [0.0, 1.0].", progressAtResponseEnd); - reader.processReadRowsResponse(response); } @@ -309,18 +314,20 @@ private boolean readNextRecord() throws IOException { current = parseFn.apply(schemaAndRecord); - // Updates the fraction consumed value. This value is calculated by interpolating between - // the fraction consumed value from the previous server response (or zero if we're consuming - // the first response) and the fractional value in the current response based on how many of - // the rows in the current response have been consumed. + // Calculates the fraction of the current stream that has been consumed. This value is + // calculated by interpolating between the fraction consumed value from the previous server + // response (or zero if we're consuming the first response) and the fractional value in the + // current response based on how many of the rows in the current response have been consumed. rowsConsumedFromCurrentResponse++; double fractionOfCurrentStreamConsumed = progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each Stream in the StreamBundle has approximately the same amount of data and - // NORMALIZING the value of fractionOfCurrentStreamConsumed. + // Assuming that each Stream in the StreamBundle has approximately the same amount of data, + // we can use the `fractionOfCurrentStreamConsumed` value to calculate the progress made over + // the + // entire StreamBundle. fractionOfStreamBundleConsumed = (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; From b337c57c8343f6a0ea6d62bec2a634303c0a4f59 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 10 Feb 2023 01:27:09 +0000 Subject: [PATCH 26/32] Removed unnecessary synchronized block, added Javadoc and improved unit test coverage. --- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 3 + .../bigquery/BigQueryStorageSourceBase.java | 25 +++--- .../BigQueryStorageStreamBundleSource.java | 46 ++++++++--- ...StorageReadWithStreamBundleSourceTest.java | 76 ++++++++++++------- 4 files changed, 98 insertions(+), 52 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index 45e6ca9f134d..938d131a0da5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; @@ -164,6 +166,7 @@ public interface BigQueryOptions void setStorageWriteApiMaxRequestSize(Long value); + @Experimental(Kind.UNSPECIFIED) @Description( "If set, BigQueryIO.Read will use the StreamBundle based" + "implementation of the Read API Source") 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 5573fbdfdde0..f757cd5c65b0 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 @@ -218,21 +218,20 @@ public List> split( readSession, readStream, trimmedSchema, parseFn, outputCoder, bqServices)); } return ImmutableList.copyOf(sources); - } else { - List streamBundle = Lists.newArrayList(); - List> sources = Lists.newArrayList(); - for (ReadStream readStream : readSession.getStreamsList()) { - streamIndex++; - streamBundle.add(readStream); - if (streamIndex % streamsPerBundle == 0) { - sources.add( - BigQueryStorageStreamBundleSource.create( - readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); - streamBundle = Lists.newArrayList(); - } + } + List streamBundle = Lists.newArrayList(); + List> sources = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + sources.add( + BigQueryStorageStreamBundleSource.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); + streamBundle = Lists.newArrayList(); } - return ImmutableList.copyOf(sources); } + return ImmutableList.copyOf(sources); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index de78b4699716..85f82e0eafed 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -47,6 +47,34 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * A {@link org.apache.beam.sdk.io.Source} representing a bundle of Streams in a BigQuery ReadAPI + * Session. This Source ONLY supports splitting at the StreamBundle level. + * + *

{@link BigQueryStorageStreamBundleSource} defines a split-point as the starting offset of each + * Stream. As a result, the number of valid split points in the Source is equal to the number of + * Streams in the StreamBundle and this Source does NOT support sub-Stream splitting. + * + *

Additionally, the underlying {@link org.apache.beam.sdk.io.range.OffsetRangeTracker} and + * {@link OffsetBasedSource} operate in the split point space and do NOT directly interact with the + * Streams constituting the StreamBundle. Consequently, fractional values used in + * `splitAtFraction()` are translated into StreamBundleIndices and the underlying RangeTracker + * handles the split operation by checking the validity of the split point. This has the following + * implications for the `splitAtFraction()` operation: + * + * 1. Fraction values that point to the "middle" of a Stream will be translated to the appropriate + * Stream boundary by the RangeTracker. + * + * 2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` calls + * that point to StreamBundleIndices that are greater than the StreamBundleIndex of the current + * Stream

+ * + * @param Type of records represented by the source. + * @see OffsetBasedSource + * @see org.apache.beam.sdk.io.range.OffsetRangeTracker + * @see org.apache.beam.sdk.io.OffsetBasedSource + * (semantically similar to {@link BigQueryStorageStreamBundleSource}) + */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { public static BigQueryStorageStreamBundleSource create( @@ -97,9 +125,6 @@ private BigQueryStorageStreamBundleSource( Coder outputCoder, BigQueryServices bqServices, long minBundleSize) { - // The underlying OffsetBasedSource (and RangeTracker) operate only on the StreamBundle and NOT - // the Streams that constitute the StreamBundle. More specifically, the offsets in the - // OffsetBasedSource are indices for the StreamBundle List. super(0, streamBundle.size(), minBundleSize); this.readSession = Preconditions.checkArgumentNotNull(readSession, "readSession"); this.streamBundle = Preconditions.checkArgumentNotNull(streamBundle, "streams"); @@ -215,8 +240,6 @@ protected long getCurrentOffset() throws NoSuchElementException { @Override protected boolean isAtSplitPoint() throws NoSuchElementException { - // The start of every Stream within a StreamBundle is being defined as a split point. This - // implies that we cannot split below the granularity of a Stream if (currentStreamOffset == 0) { return true; } @@ -235,7 +258,7 @@ public boolean advanceImpl() throws IOException { return readNextRecord(); } - private synchronized boolean readNextStream() throws IOException { + private boolean readNextStream() throws IOException { BigQueryStorageStreamBundleSource source = getCurrentSource(); if (currentStreamBundleIndex == source.streamBundle.size()) { fractionOfStreamBundleConsumed = 1d; @@ -267,8 +290,8 @@ private boolean readNextRecord() throws IOException { synchronized (this) { currentStreamOffset = 0; currentStreamBundleIndex++; - return readNextStream(); } + return readNextStream(); } ReadRowsResponse response; @@ -324,10 +347,11 @@ private boolean readNextRecord() throws IOException { progressAtResponseStart + ((progressAtResponseEnd - progressAtResponseStart) * (rowsConsumedFromCurrentResponse * 1.0 / totalRowsInCurrentResponse)); - // Assuming that each Stream in the StreamBundle has approximately the same amount of data, - // we can use the `fractionOfCurrentStreamConsumed` value to calculate the progress made over - // the - // entire StreamBundle. + + // We now calculate the progress made over the entire StreamBundle by assuming that each + // Stream in the StreamBundle has approximately the same amount of data. Given this, merely + // counting the number of Streams that have been read and linearly interpolating with the + // progress made in the current Stream gives us the overall StreamBundle progress. fractionOfStreamBundleConsumed = (currentStreamBundleIndex + fractionOfCurrentStreamConsumed) / source.streamBundle.size(); return true; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java index d7ce32326ac5..fc1ccd3c8914 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadWithStreamBundleSourceTest.java @@ -1103,22 +1103,31 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure that records are returned in the + // Read a few records from the primary Source and ensure that records are returned in the // prescribed order. BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); + + // Attempting to split at a sub-Stream level which is NOT supported by the + // `BigQueryStorageStreamBundleSource`. IOTW, since there are exactly 3 Streams in the Source, + // a split will only occur for fraction > 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.05); + assertNull(secondarySource); + assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); - // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary StreamBundle. - BoundedSource secondarySource = primary.splitAtFraction(0.35); + // Now split the primary Source, and ensure that the returned source points to a non-null + // StreamBundle containing Streams 2 & 3. + secondarySource = primary.splitAtFraction(0.5); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - assertEquals("C", primary.getCurrent().get("name")); + // Since the last two streams were split out the Primary source has been exhausted. assertFalse(primary.advance()); assertTrue(secondary.start()); @@ -1128,6 +1137,12 @@ public void testStreamSourceSplitAtFractionWithMultipleStreamsInBundle() throws assertTrue(secondary.advance()); assertEquals("F", secondary.getCurrent().get("name")); assertTrue((secondary.advance())); + + // Since we have already started reading from the last Stream in the StreamBundle, splitting + // is now a no-op. + BoundedSource tertiarySource = secondary.splitAtFraction(0.55); + assertNull(tertiarySource); + assertEquals("G", secondary.getCurrent().get("name")); assertFalse((secondary.advance())); } @@ -1185,44 +1200,49 @@ public void testStreamSourceSplitAtFractionRepeatedWithMultipleStreamInBundle() new FakeBigQueryServices().withStorageClient(fakeStorageClient), 1L); - // Read a few records from the primary bundle and ensure that records are returned in the + // Read a few records from the primary Source and ensure that records are returned in the // prescribed order. BoundedReader primary = primarySource.createReader(options); + assertTrue(primary.start()); assertEquals("A", primary.getCurrent().get("name")); assertTrue(primary.advance()); assertEquals("B", primary.getCurrent().get("name")); assertTrue(primary.advance()); + assertEquals("C", primary.getCurrent().get("name")); - // Now split the StreamBundle, and ensure that the returned source points to a non-null - // secondary StreamBundle. Since there are 3 streams in this Bundle, splitting will only - // occur when fraction >= 0.33. - BoundedSource secondarySource = primary.splitAtFraction(0.35); + // Now split the primary Source, and ensure that the returned source points to a non-null + // StreamBundle containing ONLY Stream 3. Since there are exactly 3 Streams in the Source, + // a split will only occur for fraction > 0.33. + BoundedSource secondarySource = primary.splitAtFraction(0.7); assertNotNull(secondarySource); BoundedReader secondary = secondarySource.createReader(options); - - assertEquals("C", primary.getCurrent().get("name")); - assertFalse(primary.advance()); - assertTrue(secondary.start()); - assertEquals("D", secondary.getCurrent().get("name")); - assertTrue(secondary.advance()); - assertEquals("E", secondary.getCurrent().get("name")); - assertTrue(secondary.advance()); + assertEquals("G", secondary.getCurrent().get("name")); + assertFalse((secondary.advance())); - // Now split the StreamBundle again, and ensure that the returned source points to a non-null - // tertiary StreamBundle. Since there are 2 streams in this Bundle, splitting will only - // occur when fraction >= 0.5. - BoundedSource tertiarySource = secondary.splitAtFraction(0.5); + // A second splitAtFraction() call on the primary source. The resulting source should + // contain a StreamBundle containing ONLY Stream 2. Since there are 2 Streams in the Source, + // a split will only occur for fraction > 0.50. + BoundedSource tertiarySource = primary.splitAtFraction(0.55); assertNotNull(tertiarySource); BoundedReader tertiary = tertiarySource.createReader(options); - - assertEquals("F", secondary.getCurrent().get("name")); - assertFalse((secondary.advance())); - assertTrue(tertiary.start()); - assertEquals("G", tertiary.getCurrent().get("name")); - assertFalse((tertiary.advance())); + assertEquals("D", tertiary.getCurrent().get("name")); + assertTrue(tertiary.advance()); + assertEquals("E", tertiary.getCurrent().get("name")); + assertTrue(tertiary.advance()); + assertEquals("F", tertiary.getCurrent().get("name")); + assertFalse(tertiary.advance()); + + // A third attempt to split the primary source. This will be ignored since the primary source + // since the Source contains only a single stream now and `BigQueryStorageStreamBundleSource` + // does NOT support sub-stream splitting. + tertiarySource = primary.splitAtFraction(0.9); + assertNull(tertiarySource); + + // All the rows in the primary Source have been read. + assertFalse(primary.advance()); } @Test From e02665f52052626e6c676b665a8f28404e03d074 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Fri, 10 Feb 2023 02:12:28 +0000 Subject: [PATCH 27/32] Consolidated code paths in `BigQueryIO` for Bundled and regular ReadAPI sources. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 499 ++++++++---------- 1 file changed, 224 insertions(+), 275 deletions(-) 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 353d21c702f2..bfbd743d2ccd 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 @@ -1418,11 +1418,8 @@ private PCollection expandForDirectRead( // cleanup can be removed. [https://github.com/apache/beam/issues/19375] // - if (bqOptions.getEnableBundling()) { - return expandAnonForDirectReadWithStreamBundle(input, outputCoder, beamSchema); - } - PCollectionView jobIdTokenView; + PCollectionTuple tuple; PCollection rows; if (!getWithTemplateCompatibility()) { @@ -1453,108 +1450,34 @@ public String apply(String input) { jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); TupleTag readStreamsTag = new TupleTag<>(); + TupleTag> listReadStreamsTag = new TupleTag<>(); TupleTag readSessionTag = new TupleTag<>(); TupleTag tableSchemaTag = new TupleTag<>(); - PCollectionTuple tuple = - jobIdTokenCollection.apply( - "RunQueryJob", - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - BigQueryOptions options = - c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.element(); - // Execute the query and get the destination table holding the results. - // The getTargetTable call runs a new instance of the query and returns - // the destination table created to hold the results. - BigQueryStorageQuerySource querySource = - createStorageQuerySource(jobUuid, outputCoder); - Table queryResultTable = querySource.getTargetTable(options); - - // Create a read session without specifying a desired stream count and - // let the BigQuery storage server pick the number of streams. - CreateReadSessionRequest request = - CreateReadSessionRequest.newBuilder() - .setParent( - BigQueryHelpers.toProjectResourceName( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject())) - .setReadSession( - ReadSession.newBuilder() - .setTable( - BigQueryHelpers.toTableResourceName( - queryResultTable.getTableReference())) - .setDataFormat(DataFormat.AVRO)) - .setMaxStreamCount(0) - .build(); - - ReadSession readSession; - try (StorageClient storageClient = - getBigQueryServices().getStorageClient(options)) { - readSession = storageClient.createReadSession(request); - } - - for (ReadStream readStream : readSession.getStreamsList()) { - c.output(readStream); - } - - c.output(readSessionTag, readSession); - c.output( - tableSchemaTag, - BigQueryHelpers.toJsonString(queryResultTable.getSchema())); - } - }) - .withOutputTags( - readStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + if (!bqOptions.getEnableBundling()) { + tuple = createTupleForDirectRead(jobIdTokenCollection, outputCoder, readStreamsTag, + readSessionTag, tableSchemaTag); + tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); + } else { + tuple = createTupleForDirectReadWithStreamBundle(jobIdTokenCollection, outputCoder, + listReadStreamsTag, readSessionTag, tableSchemaTag); + tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); + } - tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - PCollectionView readSessionView = tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); PCollectionView tableSchemaView = tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - rows = - tuple - .get(readStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - ReadStream readStream = c.element(); - - BigQueryStorageStreamSource streamSource = - BigQueryStorageStreamSource.create( - readSession, - readStream, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices()); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedSource.BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + if (!bqOptions.getEnableBundling()) { + rows = createPCollectionForDirectRead(tuple, outputCoder, readStreamsTag, readSessionView, + tableSchemaView); + } else { + rows = createPCollectionForDirectReadWithStreamBundle(tuple, outputCoder, + listReadStreamsTag, readSessionView, tableSchemaView); + } } PassThroughThenCleanup.CleanupOperation cleanupOperation = @@ -1583,8 +1506,10 @@ void cleanup(ContextContainer c) throws Exception { boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); if (datasetCreatedByBeam) { LOG.info( - "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); + "Deleting temporary dataset with query results {}", + tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), + tempTable.getDatasetId()); } } } @@ -1600,194 +1525,218 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - private PCollection expandAnonForDirectReadWithStreamBundle( - PBegin input, Coder outputCoder, Schema beamSchema) { - - Pipeline p = input.getPipeline(); - PCollectionView jobIdTokenView; - PCollection rows; - - if (!getWithTemplateCompatibility()) { - // Create a singleton job ID token at pipeline construction time. - String staticJobUuid = BigQueryHelpers.randomUUIDString(); - jobIdTokenView = - p.apply("TriggerIdCreation", Create.of(staticJobUuid)) - .apply("ViewId", View.asSingleton()); - // Apply the traditional Source model. - rows = - p.apply( - org.apache.beam.sdk.io.Read.from( - createStorageQuerySource(staticJobUuid, outputCoder))); - } else { - // Create a singleton job ID token at pipeline execution time. - PCollection jobIdTokenCollection = - p.apply("TriggerIdCreation", Create.of("ignored")) - .apply( - "CreateJobId", - MapElements.via( - new SimpleFunction() { - @Override - public String apply(String input) { - return BigQueryHelpers.randomUUIDString(); + private PCollectionTuple createTupleForDirectRead(PCollection jobIdTokenCollection, + Coder outputCoder, TupleTag readStreamsTag, TupleTag + readSessionTag, TupleTag tableSchemaTag) { + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); } - })); - - jobIdTokenView = jobIdTokenCollection.apply("ViewId", View.asSingleton()); - TupleTag> listReadStreamsTag = new TupleTag<>(); - TupleTag readSessionTag = new TupleTag<>(); - TupleTag tableSchemaTag = new TupleTag<>(); + for (ReadStream readStream : readSession.getStreamsList()) { + c.output(readStream); + } - PCollectionTuple tuple = - jobIdTokenCollection.apply( - "RunQueryJob", - ParDo.of( - new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - BigQueryOptions options = - c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.element(); - // Execute the query and get the destination table holding the results. - // The getTargetTable call runs a new instance of the query and returns - // the destination table created to hold the results. - BigQueryStorageQuerySource querySource = - createStorageQuerySource(jobUuid, outputCoder); - Table queryResultTable = querySource.getTargetTable(options); - - // Create a read session without specifying a desired stream count and - // let the BigQuery storage server pick the number of streams. - CreateReadSessionRequest request = - CreateReadSessionRequest.newBuilder() - .setParent( - BigQueryHelpers.toProjectResourceName( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject())) - .setReadSession( - ReadSession.newBuilder() - .setTable( - BigQueryHelpers.toTableResourceName( - queryResultTable.getTableReference())) - .setDataFormat(DataFormat.AVRO)) - .setMaxStreamCount(0) - .build(); - - ReadSession readSession; - try (StorageClient storageClient = - getBigQueryServices().getStorageClient(options)) { - readSession = storageClient.createReadSession(request); - } - int streamIndex = 0; - int streamsPerBundle = 10; - List streamBundle = Lists.newArrayList(); - for (ReadStream readStream : readSession.getStreamsList()) { - streamIndex++; - streamBundle.add(readStream); - if (streamIndex % streamsPerBundle == 0) { - c.output(streamBundle); - streamBundle = Lists.newArrayList(); - } + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + readStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + return tuple; + } + + private PCollectionTuple createTupleForDirectReadWithStreamBundle(PCollection + jobIdTokenCollection, + Coder outputCoder, TupleTag> listReadStreamsTag, + TupleTag readSessionTag, TupleTag tableSchemaTag) { + + PCollectionTuple tuple = + jobIdTokenCollection.apply( + "RunQueryJob", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + BigQueryOptions options = + c.getPipelineOptions().as(BigQueryOptions.class); + String jobUuid = c.element(); + // Execute the query and get the destination table holding the results. + // The getTargetTable call runs a new instance of the query and returns + // the destination table created to hold the results. + BigQueryStorageQuerySource querySource = + createStorageQuerySource(jobUuid, outputCoder); + Table queryResultTable = querySource.getTargetTable(options); + + // Create a read session without specifying a desired stream count and + // let the BigQuery storage server pick the number of streams. + CreateReadSessionRequest request = + CreateReadSessionRequest.newBuilder() + .setParent( + BigQueryHelpers.toProjectResourceName( + options.getBigQueryProject() == null + ? options.getProject() + : options.getBigQueryProject())) + .setReadSession( + ReadSession.newBuilder() + .setTable( + BigQueryHelpers.toTableResourceName( + queryResultTable.getTableReference())) + .setDataFormat(DataFormat.AVRO)) + .setMaxStreamCount(0) + .build(); + + ReadSession readSession; + try (StorageClient storageClient = + getBigQueryServices().getStorageClient(options)) { + readSession = storageClient.createReadSession(request); + } + int streamIndex = 0; + int streamsPerBundle = 10; + List streamBundle = Lists.newArrayList(); + for (ReadStream readStream : readSession.getStreamsList()) { + streamIndex++; + streamBundle.add(readStream); + if (streamIndex % streamsPerBundle == 0) { + c.output(streamBundle); + streamBundle = Lists.newArrayList(); } - - c.output(readSessionTag, readSession); - c.output( - tableSchemaTag, - BigQueryHelpers.toJsonString(queryResultTable.getSchema())); } - }) - .withOutputTags( - listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); - - tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); - tuple.get(readSessionTag).setCoder(ProtoCoder.of(ReadSession.class)); - tuple.get(tableSchemaTag).setCoder(StringUtf8Coder.of()); - - PCollectionView readSessionView = - tuple.get(readSessionTag).apply("ReadSessionView", View.asSingleton()); - PCollectionView tableSchemaView = - tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); - rows = - tuple - .get(listReadStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - List streamBundle = c.element(); - - BigQueryStorageStreamBundleSource streamSource = - BigQueryStorageStreamBundleSource.create( - readSession, - streamBundle, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices(), - 1L); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); + c.output(readSessionTag, readSession); + c.output( + tableSchemaTag, + BigQueryHelpers.toJsonString(queryResultTable.getSchema())); + } + }) + .withOutputTags( + listReadStreamsTag, TupleTagList.of(readSessionTag).and(tableSchemaTag))); + + return tuple; + } + + private PCollection createPCollectionForDirectRead(PCollectionTuple tuple, + Coder outputCoder, TupleTag readStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + PCollection rows = + tuple + .get(readStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + ReadStream readStream = c.element(); + + BigQueryStorageStreamSource streamSource = + BigQueryStorageStreamSource.create( + readSession, + readStream, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedSource.BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); - } - - CleanupOperation cleanupOperation = - new CleanupOperation() { - @Override - void cleanup(ContextContainer c) throws Exception { - BigQueryOptions options = c.getPipelineOptions().as(BigQueryOptions.class); - String jobUuid = c.getJobId(); + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); - Optional queryTempDataset = Optional.ofNullable(getQueryTempDataset()); - - TableReference tempTable = - createTempTableReference( - options.getBigQueryProject() == null - ? options.getProject() - : options.getBigQueryProject(), - BigQueryResourceNaming.createJobIdPrefix( - options.getJobName(), jobUuid, JobType.QUERY), - queryTempDataset); + return rows; + } - try (DatasetService datasetService = - getBigQueryServices().getDatasetService(options)) { - LOG.info("Deleting temporary table with query results {}", tempTable); - datasetService.deleteTable(tempTable); - // Delete dataset only if it was created by Beam - boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); - if (datasetCreatedByBeam) { - LOG.info( - "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); - } - } - } - }; + private PCollection createPCollectionForDirectReadWithStreamBundle(PCollectionTuple tuple, + Coder outputCoder, TupleTag> listReadStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + PCollection rows = + tuple + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices(), + 1L); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); - if (beamSchema != null) { - rows.setSchema( - beamSchema, - getTypeDescriptor(), - getToBeamRowFn().apply(beamSchema), - getFromBeamRowFn().apply(beamSchema)); - } - return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); + return rows; } @Override From d248d665e610eefd87b11b9f3df5d7b8b19b382f Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 13 Feb 2023 21:49:27 +0000 Subject: [PATCH 28/32] Lint fixes. --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 209 ++++++++++-------- .../BigQueryStorageStreamBundleSource.java | 14 +- 2 files changed, 123 insertions(+), 100 deletions(-) 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 bfbd743d2ccd..e2d8c31f3f0a 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 @@ -1455,12 +1455,22 @@ public String apply(String input) { TupleTag tableSchemaTag = new TupleTag<>(); if (!bqOptions.getEnableBundling()) { - tuple = createTupleForDirectRead(jobIdTokenCollection, outputCoder, readStreamsTag, - readSessionTag, tableSchemaTag); + tuple = + createTupleForDirectRead( + jobIdTokenCollection, + outputCoder, + readStreamsTag, + readSessionTag, + tableSchemaTag); tuple.get(readStreamsTag).setCoder(ProtoCoder.of(ReadStream.class)); } else { - tuple = createTupleForDirectReadWithStreamBundle(jobIdTokenCollection, outputCoder, - listReadStreamsTag, readSessionTag, tableSchemaTag); + tuple = + createTupleForDirectReadWithStreamBundle( + jobIdTokenCollection, + outputCoder, + listReadStreamsTag, + readSessionTag, + tableSchemaTag); tuple.get(listReadStreamsTag).setCoder(ListCoder.of(ProtoCoder.of(ReadStream.class))); } @@ -1472,11 +1482,13 @@ public String apply(String input) { tuple.get(tableSchemaTag).apply("TableSchemaView", View.asSingleton()); if (!bqOptions.getEnableBundling()) { - rows = createPCollectionForDirectRead(tuple, outputCoder, readStreamsTag, readSessionView, - tableSchemaView); + rows = + createPCollectionForDirectRead( + tuple, outputCoder, readStreamsTag, readSessionView, tableSchemaView); } else { - rows = createPCollectionForDirectReadWithStreamBundle(tuple, outputCoder, - listReadStreamsTag, readSessionView, tableSchemaView); + rows = + createPCollectionForDirectReadWithStreamBundle( + tuple, outputCoder, listReadStreamsTag, readSessionView, tableSchemaView); } } @@ -1506,10 +1518,8 @@ void cleanup(ContextContainer c) throws Exception { boolean datasetCreatedByBeam = !queryTempDataset.isPresent(); if (datasetCreatedByBeam) { LOG.info( - "Deleting temporary dataset with query results {}", - tempTable.getDatasetId()); - datasetService.deleteDataset(tempTable.getProjectId(), - tempTable.getDatasetId()); + "Deleting temporary dataset with query results {}", tempTable.getDatasetId()); + datasetService.deleteDataset(tempTable.getProjectId(), tempTable.getDatasetId()); } } } @@ -1525,9 +1535,12 @@ void cleanup(ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView)); } - private PCollectionTuple createTupleForDirectRead(PCollection jobIdTokenCollection, - Coder outputCoder, TupleTag readStreamsTag, TupleTag - readSessionTag, TupleTag tableSchemaTag) { + private PCollectionTuple createTupleForDirectRead( + PCollection jobIdTokenCollection, + Coder outputCoder, + TupleTag readStreamsTag, + TupleTag readSessionTag, + TupleTag tableSchemaTag) { PCollectionTuple tuple = jobIdTokenCollection.apply( "RunQueryJob", @@ -1585,10 +1598,12 @@ public void processElement(ProcessContext c) throws Exception { return tuple; } - private PCollectionTuple createTupleForDirectReadWithStreamBundle(PCollection - jobIdTokenCollection, - Coder outputCoder, TupleTag> listReadStreamsTag, - TupleTag readSessionTag, TupleTag tableSchemaTag) { + private PCollectionTuple createTupleForDirectReadWithStreamBundle( + PCollection jobIdTokenCollection, + Coder outputCoder, + TupleTag> listReadStreamsTag, + TupleTag readSessionTag, + TupleTag tableSchemaTag) { PCollectionTuple tuple = jobIdTokenCollection.apply( @@ -1654,87 +1669,95 @@ public void processElement(ProcessContext c) throws Exception { return tuple; } - private PCollection createPCollectionForDirectRead(PCollectionTuple tuple, - Coder outputCoder, TupleTag readStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + private PCollection createPCollectionForDirectRead( + PCollectionTuple tuple, + Coder outputCoder, + TupleTag readStreamsTag, + PCollectionView readSessionView, + PCollectionView tableSchemaView) { PCollection rows = - tuple - .get(readStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - ReadStream readStream = c.element(); - - BigQueryStorageStreamSource streamSource = - BigQueryStorageStreamSource.create( - readSession, - readStream, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices()); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedSource.BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + tuple + .get(readStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + ReadStream readStream = c.element(); + + BigQueryStorageStreamSource streamSource = + BigQueryStorageStreamSource.create( + readSession, + readStream, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices()); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedSource.BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); return rows; } - private PCollection createPCollectionForDirectReadWithStreamBundle(PCollectionTuple tuple, - Coder outputCoder, TupleTag> listReadStreamsTag, PCollectionView readSessionView, PCollectionView tableSchemaView) { + private PCollection createPCollectionForDirectReadWithStreamBundle( + PCollectionTuple tuple, + Coder outputCoder, + TupleTag> listReadStreamsTag, + PCollectionView readSessionView, + PCollectionView tableSchemaView) { PCollection rows = tuple - .get(listReadStreamsTag) - .apply(Reshuffle.viaRandomKey()) - .apply( - ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - ReadSession readSession = c.sideInput(readSessionView); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(tableSchemaView), TableSchema.class); - List streamBundle = c.element(); - - BigQueryStorageStreamBundleSource streamSource = - BigQueryStorageStreamBundleSource.create( - readSession, - streamBundle, - tableSchema, - getParseFn(), - outputCoder, - getBigQueryServices(), - 1L); - - // Read all of the data from the stream. In the event that this work - // item fails and is rescheduled, the same rows will be returned in - // the same order. - BoundedReader reader = - streamSource.createReader(c.getPipelineOptions()); - for (boolean more = reader.start(); more; more = reader.advance()) { - c.output(reader.getCurrent()); - } - } - }) - .withSideInputs(readSessionView, tableSchemaView)) - .setCoder(outputCoder); + .get(listReadStreamsTag) + .apply(Reshuffle.viaRandomKey()) + .apply( + ParDo.of( + new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + ReadSession readSession = c.sideInput(readSessionView); + TableSchema tableSchema = + BigQueryHelpers.fromJsonString( + c.sideInput(tableSchemaView), TableSchema.class); + List streamBundle = c.element(); + + BigQueryStorageStreamBundleSource streamSource = + BigQueryStorageStreamBundleSource.create( + readSession, + streamBundle, + tableSchema, + getParseFn(), + outputCoder, + getBigQueryServices(), + 1L); + + // Read all of the data from the stream. In the event that this work + // item fails and is rescheduled, the same rows will be returned in + // the same order. + BoundedReader reader = + streamSource.createReader(c.getPipelineOptions()); + for (boolean more = reader.start(); more; more = reader.advance()) { + c.output(reader.getCurrent()); + } + } + }) + .withSideInputs(readSessionView, tableSchemaView)) + .setCoder(outputCoder); return rows; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 85f82e0eafed..849b3ea395a9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -62,18 +62,18 @@ * handles the split operation by checking the validity of the split point. This has the following * implications for the `splitAtFraction()` operation: * - * 1. Fraction values that point to the "middle" of a Stream will be translated to the appropriate - * Stream boundary by the RangeTracker. + *

1. Fraction values that point to the "middle" of a Stream will be translated to the + * appropriate Stream boundary by the RangeTracker. * - * 2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` calls - * that point to StreamBundleIndices that are greater than the StreamBundleIndex of the current - * Stream

+ *

2. Once a Stream is being read from, the RangeTracker will only accept `splitAtFraction()` + * calls that point to StreamBundleIndices that are greater than the StreamBundleIndex of the + * current Stream * * @param Type of records represented by the source. * @see OffsetBasedSource * @see org.apache.beam.sdk.io.range.OffsetRangeTracker - * @see org.apache.beam.sdk.io.OffsetBasedSource - * (semantically similar to {@link BigQueryStorageStreamBundleSource}) + * @see org.apache.beam.sdk.io.OffsetBasedSource (semantically similar to {@link + * BigQueryStorageStreamBundleSource}) */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { From 1432cbc5385169e444756a855ffb3db9b2c2419d Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Mon, 13 Feb 2023 21:54:50 +0000 Subject: [PATCH 29/32] Minor Javadoc fix. --- .../sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 849b3ea395a9..f1a414d39f4f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -72,7 +72,7 @@ * @param Type of records represented by the source. * @see OffsetBasedSource * @see org.apache.beam.sdk.io.range.OffsetRangeTracker - * @see org.apache.beam.sdk.io.OffsetBasedSource (semantically similar to {@link + * @see org.apache.beam.sdk.io.BlockBasedSource (semantically similar to {@link * BigQueryStorageStreamBundleSource}) */ class BigQueryStorageStreamBundleSource extends OffsetBasedSource { From 3adc8156a00e047bb2294e264ecf1fe23348f2f8 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 22:54:56 +0000 Subject: [PATCH 30/32] Fix StreamBundle creation logic and some minor code comment updates. --- .../apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 +++- .../io/gcp/bigquery/BigQueryStorageSourceBase.java | 11 ++++++++--- .../bigquery/BigQueryStorageStreamBundleSource.java | 6 ++++-- 3 files changed, 15 insertions(+), 6 deletions(-) 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 e2d8c31f3f0a..261f2f04783f 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 @@ -1656,7 +1656,9 @@ public void processElement(ProcessContext c) throws Exception { streamBundle = Lists.newArrayList(); } } - + if(streamIndex % streamsPerBundle != 0) { + c.output(streamBundle); + } c.output(readSessionTag, readSession); c.output( tableSchemaTag, 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 f757cd5c65b0..42f0396e3e8e 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 @@ -176,19 +176,19 @@ public List> split( int streamsPerBundle = 0; double bytesPerStream = 0; LOG.info( - "readSession.getEstimatedTotalBytesScanned(): '{}'", + "Estimated bytes this ReadSession will scan when all Streams are consumed: '{}'", readSession.getEstimatedTotalBytesScanned()); if (bqOptions.getEnableBundling()) { if (desiredBundleSizeBytes > 0) { bytesPerStream = (double) readSession.getEstimatedTotalBytesScanned() / readSession.getStreamsCount(); - LOG.info("bytesPerStream: '{}'", bytesPerStream); + LOG.info("Estimated bytes each Stream will consume: '{}'", bytesPerStream); streamsPerBundle = (int) Math.ceil(desiredBundleSizeBytes / bytesPerStream); } else { streamsPerBundle = (int) Math.ceil((double) streamCount / 10); } streamsPerBundle = Math.min(streamCount, streamsPerBundle); - LOG.info("streamsPerBundle: '{}'", streamsPerBundle); + LOG.info("Distributing '{}' Streams per StreamBundle.", streamsPerBundle); } Schema sessionSchema; @@ -231,6 +231,11 @@ public List> split( streamBundle = Lists.newArrayList(); } } + if (streamIndex % streamsPerBundle != 0) { + sources.add( + BigQueryStorageStreamBundleSource.create( + readSession, streamBundle, trimmedSchema, parseFn, outputCoder, bqServices, 1L)); + } return ImmutableList.copyOf(sources); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index f1a414d39f4f..00c6c57025e8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.slf4j.Logger; @@ -160,8 +161,9 @@ public long getEstimatedSizeBytes(PipelineOptions options) { @Override public List> split( long desiredBundleSizeBytes, PipelineOptions options) { - // A stream source can't be split without reading from it due to server-side liquid sharding. - // TODO: Implement dynamic work rebalancing. + // This method is only called for initial splits. Since this class will always be a child source + // of BigQueryStorageSourceBase, all splits here will be handled by `splitAtFraction()`. As a + // result, this is a no-op. return ImmutableList.of(this); } From 5eb1aa4c6f12ec222793205c2de7ec7aee1c8909 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 22:57:44 +0000 Subject: [PATCH 31/32] Updated logging. --- .../sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 42f0396e3e8e..834409062ccd 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 @@ -161,10 +161,9 @@ public List> split( try (StorageClient client = bqServices.getStorageClient(bqOptions)) { readSession = client.createReadSession(createReadSessionRequest); LOG.info( - "Sent BigQuery Storage API CreateReadSession request in code '{}'.", - createReadSessionRequest); - LOG.info( - "Received number of streams in response: '{}'.", readSession.getStreamsList().size()); + "Sent BigQuery Storage API CreateReadSession request '{}'; received response '{}'.", + createReadSessionRequest, + readSession); } if (readSession.getStreamsList().isEmpty()) { From d1a6d43a9356c936e2837b24124a40e881899836 Mon Sep 17 00:00:00 2001 From: Vachan Shetty Date: Thu, 16 Feb 2023 23:58:50 +0000 Subject: [PATCH 32/32] Lint fixes. --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) 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 261f2f04783f..6745f7aceea0 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 @@ -1656,7 +1656,7 @@ public void processElement(ProcessContext c) throws Exception { streamBundle = Lists.newArrayList(); } } - if(streamIndex % streamsPerBundle != 0) { + if (streamIndex % streamsPerBundle != 0) { c.output(streamBundle); } c.output(readSessionTag, readSession); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java index 00c6c57025e8..42e99b6aae38 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamBundleSource.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.slf4j.Logger;