binary-signal opened a new pull request, #2165: URL: https://github.com/apache/fluss/pull/2165
## Purpose <!-- Linking this pull request to the issue --> Linked issue: close #2164 <!-- What is the purpose of the change --> Fix `IndexOutOfBoundsException` when writing rows with array columns where the total number of array elements exceeds `INITIAL_CAPACITY` (1024) while the row count stays below it. ### Brief change log In `ArrowWriter.writeRow()`, the `handleSafe` flag is determined by comparing row count against `INITIAL_CAPACITY`: ```java boolean handleSafe = recordsCount >= INITIAL_CAPACITY; ``` When `handleSafe = false`, Arrow writers use `vector.set()` which doesn't auto-grow the buffer. The bug is in `ArrowArrayWriter.doWrite()` which passes the parent's `handleSafe` flag to the element writer. However, array element indices grow based on cumulative element count, not row count. Example: 250 rows with 10-element arrays → row count (250) < 1024 so `handleSafe = false`, but total elements (2500) exceeds the vector's initial capacity, causing `IndexOutOfBoundsException`. **Fix:** Always use safe writes (`handleSafe = true`) for array element writers in `ArrowArrayWriter.doWrite()`, since element indices can exceed `INITIAL_CAPACITY` independently of row count. ```java // Before elementWriter.write(fieldIndex, array, arrIndex, handleSafe); // After elementWriter.write(fieldIndex, array, arrIndex, true); ``` ### Tests - Added `ArrowReaderWriterTest#testArrayWriterWithManyElements`: writes 200 rows with 10-element arrays (2000 total elements), verifying serialization succeeds and data can be read back correctly. ### API and Format No API or storage format changes. ### Documentation No documentation changes needed. This is a bug fix. ``` -- 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]
