-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Spark: Structured Streaming read limit support follow-up #12260
base: main
Are you sure you want to change the base?
Conversation
Use the ReadLimit passed in to SparkMicroBatchStream::latestOffset. In addition, fix a bug.
readLimits[1] = ReadLimit.maxRows(maxFilesPerMicroBatch); | ||
readLimits[1] = ReadLimit.maxRows(maxRecordsPerMicroBatch); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Bug!
There was a problem hiding this comment.
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!
public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfFiles_1() | ||
throws Exception { | ||
public void testReadStreamWithMaxFiles1() throws Exception { |
There was a problem hiding this comment.
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.
assertThat( | ||
microBatchCount( | ||
ImmutableMap.of( | ||
SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1", | ||
SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"))) | ||
.isEqualTo(6); |
There was a problem hiding this comment.
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 ReadLimit
s is a ReadMaxRows(1)
.
@singhpk234 @jackye1995 @RussellSpitzer this is a small fix; can you please review? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Mostly LGTM with a minor suggestion, Thanks @wypoon !
for (int i = 0; i < limits.length; i++) { | ||
ReadLimit limit = limits[i]; | ||
if (limit instanceof ReadMaxFiles) { | ||
return ((ReadMaxFiles) limit).maxFiles(); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
[minor] can we use this ?
for (int i = 0; i < limits.length; i++) { | |
ReadLimit limit = limits[i]; | |
if (limit instanceof ReadMaxFiles) { | |
return ((ReadMaxFiles) limit).maxFiles(); | |
} | |
} | |
for (ReadLimit limit: limits) { | |
if (limit instanceof ReadMaxFiles) { | |
return ((ReadMaxFiles) limit).maxFiles(); | |
} | |
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Adopted.
readLimits[1] = ReadLimit.maxRows(maxFilesPerMicroBatch); | ||
readLimits[1] = ReadLimit.maxRows(maxRecordsPerMicroBatch); |
There was a problem hiding this comment.
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!
Thanks @singhpk234. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM, Thanks @wypoon !
This fixes the TODO in #4479.
Use the ReadLimit passed in to
SparkMicroBatchStream::latestOffset(Offset, ReadLimit)
. In testing this, a bug was found inSparkMicroBatchStream::getDefaultReadLimit()
and fixed.