Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -58,9 +58,12 @@
import org.apache.spark.broadcast.Broadcast;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReaderFactory;
import org.apache.spark.sql.connector.read.streaming.CompositeReadLimit;
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
import org.apache.spark.sql.connector.read.streaming.Offset;
import org.apache.spark.sql.connector.read.streaming.ReadLimit;
import org.apache.spark.sql.connector.read.streaming.ReadMaxFiles;
import org.apache.spark.sql.connector.read.streaming.ReadMaxRows;
import org.apache.spark.sql.connector.read.streaming.SupportsAdmissionControl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -309,6 +312,47 @@ private static StreamingOffset determineStartingOffset(Table table, Long fromTim
}
}

private static int getMaxFiles(ReadLimit readLimit) {
if (readLimit instanceof ReadMaxFiles) {
return ((ReadMaxFiles) readLimit).maxFiles();
}

if (readLimit instanceof CompositeReadLimit) {
// We do not expect a CompositeReadLimit to contain a nested CompositeReadLimit.
// In fact, it should only be a composite of two or more of ReadMinRows, ReadMaxRows and
// ReadMaxFiles, with no more than one of each.
ReadLimit[] limits = ((CompositeReadLimit) readLimit).getReadLimits();
for (ReadLimit limit : limits) {
if (limit instanceof ReadMaxFiles) {
return ((ReadMaxFiles) limit).maxFiles();
}
}
}

// there is no ReadMaxFiles, so return the default
return Integer.MAX_VALUE;
}

private static int getMaxRows(ReadLimit readLimit) {
if (readLimit instanceof ReadMaxRows) {
long maxRows = ((ReadMaxRows) readLimit).maxRows();
return Math.toIntExact(maxRows);
}

if (readLimit instanceof CompositeReadLimit) {
ReadLimit[] limits = ((CompositeReadLimit) readLimit).getReadLimits();
for (ReadLimit limit : limits) {
if (limit instanceof ReadMaxRows) {
long maxRows = ((ReadMaxRows) limit).maxRows();
return Math.toIntExact(maxRows);
}
}
}

// there is no ReadMaxRows, so return the default
return Integer.MAX_VALUE;
}

@Override
@SuppressWarnings("checkstyle:CyclomaticComplexity")
public Offset latestOffset(Offset startOffset, ReadLimit limit) {
Expand Down Expand Up @@ -368,10 +412,8 @@ public Offset latestOffset(Offset startOffset, ReadLimit limit) {
while (taskIter.hasNext()) {
FileScanTask task = taskIter.next();
if (curPos >= startPosOfSnapOffset) {
// TODO : use readLimit provided in function param, the readLimits are derived from
// these 2 properties.
if ((curFilesAdded + 1) > maxFilesPerMicroBatch
|| (curRecordCount + task.file().recordCount()) > maxRecordsPerMicroBatch) {
if ((curFilesAdded + 1) > getMaxFiles(limit)
|| (curRecordCount + task.file().recordCount()) > getMaxRows(limit)) {
shouldContinueReading = false;
break;
}
Expand Down Expand Up @@ -458,7 +500,7 @@ public ReadLimit getDefaultReadLimit() {
&& maxRecordsPerMicroBatch != Integer.MAX_VALUE) {
ReadLimit[] readLimits = new ReadLimit[2];
readLimits[0] = ReadLimit.maxFiles(maxFilesPerMicroBatch);
readLimits[1] = ReadLimit.maxRows(maxFilesPerMicroBatch);
readLimits[1] = ReadLimit.maxRows(maxRecordsPerMicroBatch);
Comment on lines -461 to +503
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for catching ! This got missed, as we don't take the Readlimit we get from latestOffset API but rather from the configs which are set in constructor earlier!

return ReadLimit.compositeLimit(readLimits);
} else if (maxFilesPerMicroBatch != Integer.MAX_VALUE) {
return ReadLimit.maxFiles(maxFilesPerMicroBatch);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,8 +154,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception
}

@TestTemplate
public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_1()
throws Exception {
public void testReadStreamWithMaxFiles1() throws Exception {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I renamed a few tests to be more concise. The old names were unwieldy and also not conforming to Java style.

appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

assertThat(
Expand All @@ -165,8 +164,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_
}

@TestTemplate
public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_2()
throws Exception {
public void testReadStreamWithMaxFiles2() throws Exception {
appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

assertThat(
Expand All @@ -176,8 +174,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_
}

@TestTemplate
public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1()
throws Exception {
public void testReadStreamWithMaxRows1() throws Exception {
appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

// only 1 micro-batch will be formed and we will read data partially
Expand All @@ -186,7 +183,8 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1
ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1")))
.isEqualTo(1);

StreamingQuery query = startStream(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1");
StreamingQuery query =
startStream(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"));

// check answer correctness only 1 record read the micro-batch will be stuck
List<SimpleRecord> actual = rowsAvailable(query);
Expand All @@ -196,8 +194,24 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1
}

@TestTemplate
public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_4()
throws Exception {
public void testReadStreamWithMaxRows2() throws Exception {
appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

assertThat(
microBatchCount(
ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2")))
.isEqualTo(4);

StreamingQuery query =
startStream(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"));

List<SimpleRecord> actual = rowsAvailable(query);
assertThat(actual)
.containsExactlyInAnyOrderElementsOf(Iterables.concat(TEST_DATA_MULTIPLE_SNAPSHOTS));
}

@TestTemplate
public void testReadStreamWithMaxRows4() throws Exception {
appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

assertThat(
Expand All @@ -206,6 +220,18 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_4
.isEqualTo(2);
}

@TestTemplate
public void testReadStreamWithCompositeReadLimit() throws Exception {
appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS);

assertThat(
microBatchCount(
ImmutableMap.of(
SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1",
SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2")))
.isEqualTo(6);
Comment on lines +227 to +232
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This fails without the fix to SparkMicroBatchStream::getDefaultReadLimit(), as Spark then calls SparkMicroBatchStream::latestOffset(Offset, ReadLimit) with a CompositeReadLimit where one of the ReadLimits is a ReadMaxRows(1).

}

@TestTemplate
public void testReadStreamOnIcebergThenAddData() throws Exception {
List<List<SimpleRecord>> expected = TEST_DATA_MULTIPLE_SNAPSHOTS;
Expand Down