|
99 | 99 | import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; |
100 | 100 | import org.apache.beam.sdk.io.BoundedSource; |
101 | 101 | import org.apache.beam.sdk.io.BoundedSource.BoundedReader; |
| 102 | +import org.apache.beam.sdk.io.UnboundedSource; |
102 | 103 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TableRowParser; |
103 | 104 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; |
104 | 105 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; |
|
120 | 121 | import org.apache.beam.sdk.transforms.PTransform; |
121 | 122 | import org.apache.beam.sdk.transforms.SerializableFunction; |
122 | 123 | import org.apache.beam.sdk.transforms.display.DisplayData; |
| 124 | +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; |
123 | 125 | import org.apache.beam.sdk.values.KV; |
124 | 126 | import org.apache.beam.sdk.values.PBegin; |
125 | 127 | import org.apache.beam.sdk.values.PCollection; |
@@ -705,6 +707,49 @@ public void testTableSourceCreateReader() throws Exception { |
705 | 707 | tableSource.createReader(options); |
706 | 708 | } |
707 | 709 |
|
| 710 | + @Test |
| 711 | + public void testUnboundedReadFromBoundedSourceWithEmptyTable() throws Exception { |
| 712 | + fakeDatasetService.createDataset("project-id", "dataset", "", "", null); |
| 713 | + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset.table"); |
| 714 | + |
| 715 | + Table table = |
| 716 | + new Table().setTableReference(tableRef).setNumBytes(0L).setSchema(new TableSchema()); |
| 717 | + |
| 718 | + fakeDatasetService.createTable(table); |
| 719 | + |
| 720 | + ReadSession emptyReadSession = ReadSession.newBuilder().build(); |
| 721 | + StorageClient fakeStorageClient = mock(StorageClient.class); |
| 722 | + when(fakeStorageClient.createReadSession(any())).thenReturn(emptyReadSession); |
| 723 | + |
| 724 | + BigQueryStorageTableSource<TableRow> tableSource = |
| 725 | + BigQueryStorageTableSource.create( |
| 726 | + ValueProvider.StaticValueProvider.of(tableRef), |
| 727 | + null, |
| 728 | + null, |
| 729 | + new TableRowParser(), |
| 730 | + TableRowJsonCoder.of(), |
| 731 | + new FakeBigQueryServices() |
| 732 | + .withDatasetService(fakeDatasetService) |
| 733 | + .withStorageClient(fakeStorageClient)); |
| 734 | + |
| 735 | + // This simulates what happens in a streaming pipeline when BoundedSource is used |
| 736 | + UnboundedSource<TableRow, ?> unboundedSource = |
| 737 | + new UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter<>(tableSource); |
| 738 | + |
| 739 | + // Initial split |
| 740 | + List<? extends UnboundedSource<TableRow, ?>> splits = unboundedSource.split(1, options); |
| 741 | + // Because tableSource.split returns empty list, BoundedToUnboundedSourceAdapter falls back to |
| 742 | + // returning itself |
| 743 | + assertEquals(1, splits.size()); |
| 744 | + UnboundedSource<TableRow, ?> splitSource = splits.get(0); |
| 745 | + |
| 746 | + // Create reader |
| 747 | + UnboundedSource.UnboundedReader<TableRow> reader = splitSource.createReader(options, null); |
| 748 | + |
| 749 | + // This should NOT throw UnsupportedOperationException |
| 750 | + assertFalse(reader.start()); |
| 751 | + } |
| 752 | + |
708 | 753 | private static GenericRecord createRecord(String name, Schema schema) { |
709 | 754 | GenericRecord genericRecord = new Record(schema); |
710 | 755 | genericRecord.put("name", name); |
|
0 commit comments