Abacn commented on code in PR #31125:
URL: https://github.com/apache/beam/pull/31125#discussion_r1581651936
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java:
##########
@@ -1384,6 +1386,91 @@ public KV<String, Long> apply(SchemaAndRecord input) {
}
}
+ static class StuckResponse extends ArrayList<ReadRowsResponse> {
+
+ private CountDownLatch latch;
+
+ public StuckResponse(CountDownLatch latch) {
+ this.latch = latch;
+ }
+
+ static final Object lock = new Object();
+
+ static boolean hasNextCalled = false;
+
+ @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>()
+
+ BigQueryStorageStreamSource<TableRow> streamSource =
+ BigQueryStorageStreamSource.create(
+ readSession,
+ ReadStream.newBuilder().setName("readStream").build(),
+ TABLE_SCHEMA,
+ new TableRowParser(),
+ TableRowJsonCoder.of(),
+ new FakeBigQueryServices().withStorageClient(fakeStorageClient));
+
+ BigQueryStorageStreamReader<TableRow> reader =
streamSource.createReader(options);
+
+ Thread t =
+ new Thread(
+ () -> {
+ try {
+ reader.start();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ t.start();
+
+ // wait until thread proceed
+ latch.await();
+
+ // Beam does not split storage read api v2 stream
+ assertNull(reader.splitAtFraction(0.5));
Review Comment:
without the changes in main scope code, the test fails with
```
java.lang.NullPointerException
at
org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamSource$BigQueryStorageStreamReader.splitAtFraction(BigQueryStorageStreamSource.java:372)
```
because it passed "splitAllowed" check
--
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]