ahmedabu98 commented on code in PR #31125:
URL: https://github.com/apache/beam/pull/31125#discussion_r1591097140


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java:
##########
@@ -1384,6 +1386,87 @@ public KV<String, Long> apply(SchemaAndRecord input) {
     }
   }
 
+  static class StuckResponse extends ArrayList<ReadRowsResponse> {
+
+    private CountDownLatch latch;
+
+    public StuckResponse(CountDownLatch latch) {
+      this.latch = latch;
+    }
+
+    @Override
+    public Iterator<ReadRowsResponse> iterator() {
+      return new StuckIterator();
+    }
+
+    private class StuckIterator implements Iterator<ReadRowsResponse> {
+
+      @Override
+      public boolean hasNext() {
+        latch.countDown();
+        try {
+          Thread.sleep(Long.MAX_VALUE);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+        return false;
+      }
+
+      @Override
+      public ReadRowsResponse next() {
+        return null;
+      }
+    }
+  }
+
+  @Test
+  public void testStreamSourceSplitAtFractionFailsWhenReaderRunning() throws 
Exception {
+    ReadSession readSession =
+        ReadSession.newBuilder()
+            .setName("readSession")
+            
.setAvroSchema(AvroSchema.newBuilder().setSchema(AVRO_SCHEMA_STRING))
+            .build();
+
+    ReadRowsRequest expectedRequest =
+        ReadRowsRequest.newBuilder().setReadStream("readStream").build();
+
+    CountDownLatch latch = new CountDownLatch(1);
+    StorageClient fakeStorageClient = mock(StorageClient.class);
+    when(fakeStorageClient.readRows(expectedRequest, ""))
+        .thenReturn(new FakeBigQueryServerStream<>(new StuckResponse(latch)));
+
+    // new StuckIteratorList<Void>()

Review Comment:
   cleanup



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java:
##########
@@ -238,9 +245,12 @@ public synchronized boolean advance() throws IOException {
     private synchronized boolean readNextRecord() throws IOException {
       Iterator<ReadRowsResponse> responseIterator = this.responseIterator;
       while (reader.readyForNextReadResponse()) {
+        boolean previous = splitAllowed;
+        splitAllowed = false;
         // hasNext call has internal retry. Record throttling metrics after 
called
         boolean hasNext = responseIterator.hasNext();
         storageClient.reportPendingMetrics();
+        splitAllowed = previous;

Review Comment:
   Why the need to set `splitAllowed` to false while reporting metrics?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to