[
https://issues.apache.org/jira/browse/FLINK-32566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17751546#comment-17751546
]
Yun Tang commented on FLINK-32566:
----------------------------------
The root cause, which is analysed by [~catyee], is that the size of the batch
to read is hard coded by
[VectorizedColumnBatch#DEFAULT_SIZE|https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatch.java#L46],
and once we read some huge part, the memory would increase to OOM.
> HeapBytesVector meets java.lang.NegativeArraySizeException
> ----------------------------------------------------------
>
> Key: FLINK-32566
> URL: https://issues.apache.org/jira/browse/FLINK-32566
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Runtime
> Affects Versions: 1.17.1
> Reporter: Yuan Kui
> Priority: Major
>
> When reading the parquet format files, if the data of some fields is too
> large(to support large fields, we modified the default maxMessageSize value
> from 100MB to 300MB of thrift in parquet shaded class :
> shaded.parquet.org.apache.thrift.TConfiguration), HeapBytesVector will exceed
> the int maximum value when expending the capacity, and then it will meet the
> java.lang.NegativeArraySizeException.
> {code:java}
> // code placeholder
> switched from RUNNING to FAILED with failure cause:
> java.lang.RuntimeException: One or more fetchers have encountered exception
> at
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:261)
> at
> org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:169)
> at
> org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:131)
> at
> org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:422)
> at
> org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
> at
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:550)
> at
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:839)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:788)
> at
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:961)
> at
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:939)
> at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:749)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:564)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.RuntimeException: SplitFetcher thread 0 received
> unexpected exception while polling the records
> at
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:165)
> at
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:114)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> ... 1 more
> Caused by: java.lang.NegativeArraySizeException
> at
> org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector.reserve(HeapBytesVector.java:102)
> at
> org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector.appendBytes(HeapBytesVector.java:79)
> at
> org.apache.flink.formats.parquet.vector.reader.BytesColumnReader.readBinary(BytesColumnReader.java:88)
> at
> org.apache.flink.formats.parquet.vector.reader.BytesColumnReader.readBatch(BytesColumnReader.java:50)
> at
> org.apache.flink.formats.parquet.vector.reader.BytesColumnReader.readBatch(BytesColumnReader.java:31)
> at
> org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader.readToVector(AbstractColumnReader.java:189)
> at
> org.apache.flink.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.nextBatch(ParquetVectorizedInputFormat.java:401)
> at
> org.apache.flink.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.readBatch(ParquetVectorizedInputFormat.java:369)
> at
> org.apache.flink.connector.file.src.impl.FileSourceSplitReader.fetch(FileSourceSplitReader.java:67)
> at
> org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:58)
> at
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:162)
> ... 6 more {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)