RussellSpitzer commented on code in PR #12988: URL: https://github.com/apache/iceberg/pull/12988#discussion_r2153053702
########## docs/docs/spark-configuration.md: ########## @@ -225,8 +225,7 @@ spark.read | streaming-max-rows-per-micro-batch | INT_MAX | Maximum number of rows per microbatch | !!! warning - streaming-max-rows-per-micro-batch should always be greater than the number of records in any data file in the table. - The smallest unit that will be streamed is a single file, so if a data file contains more records than this limit, the stream will get stuck at this file. + streaming-max-rows-per-micro-batch option sets a “soft max”, meaning that a batch processes approximately this amount of records and may process more than the limit in order to make the streaming query move forward in cases when the smallest input unit is larger than this limit. Review Comment: This is a really complicated sentence, suggestion on a rewording? ```suggestion streaming-max-rows-per-micro-batch option sets a “soft max”, a batch will always include all the rows in the next unprocessed data file but additional files will not be included if doing so would exceed the the soft-max. ``` ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ########## @@ -387,7 +387,7 @@ public Offset latestOffset(Offset startOffset, ReadLimit limit) { boolean shouldContinueReading = true; int curFilesAdded = 0; - int curRecordCount = 0; + long curRecordCount = 0; Review Comment: Interesting ... I think think we may actually have a spark issue with we exceed int-max rows in a microbatch .... Not sure. I'm mostly thinking about the limits on array length in the inmemory versions of rows getting shipped to shufflers ... not important though. I'm just rambling -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org