ChaoFang created FLINK-38491:
--------------------------------
Summary: flink-cdc connector-iceberg compaction parallelism error
Key: FLINK-38491
URL: https://issues.apache.org/jira/browse/FLINK-38491
Project: Flink
Issue Type: Bug
Components: Flink CDC
Affects Versions: 3.0.0
Reporter: ChaoFang
Fix For: 3.0.0
# **sink.compaction.commit.parallelism** not expose to user.
# compaction partitionCustom Logic error,( bucket % numPartitions ) maybe
negative
```
java.lang.ArrayIndexOutOfBoundsException: Array index out of range: -1
at
org.apache.flink.runtime.io.network.partition.BufferWritingResultPartition.appendUnicastDataForNewRecord(BufferWritingResultPartition.java:290)
at
org.apache.flink.runtime.io.network.partition.BufferWritingResultPartition.emitRecord(BufferWritingResultPartition.java:156)
at
org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:108)
at
org.apache.flink.runtime.io.network.api.writer.ChannelSelectorRecordWriter.emit(ChannelSelectorRecordWriter.java:55)
at
org.apache.flink.streaming.runtime.io.RecordWriterOutput.pushToRecordWriter(RecordWriterOutput.java:140)
at
org.apache.flink.streaming.runtime.io.RecordWriterOutput.collectAndCheckIfChained(RecordWriterOutput.java:120)
at
org.apache.flink.streaming.runtime.io.RecordWriterOutput.collect(RecordWriterOutput.java:101)
at
org.apache.flink.streaming.runtime.io.RecordWriterOutput.collect(RecordWriterOutput.java:53)
at
org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:60)
at
org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:32)
at
org.apache.flink.streaming.runtime.operators.sink.CommitterOperator.emit(CommitterOperator.java:199)
```
In java -5 % 4 = -5 - (4 * (-1)) = -5 + 4 = -1
--
This message was sent by Atlassian Jira
(v8.20.10#820010)