[
https://issues.apache.org/jira/browse/FLINK-25482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17477522#comment-17477522
]
luoyuxia commented on FLINK-25482:
----------------------------------
[~miaojianlong]Thanks for reporting it, feel free to open pr to us.
> Hive Lookup Join with decimal type failed
> -----------------------------------------
>
> Key: FLINK-25482
> URL: https://issues.apache.org/jira/browse/FLINK-25482
> Project: Flink
> Issue Type: Bug
> Components: Connectors / Hive
> Affects Versions: 1.12.3, 1.13.5, 1.14.2
> Reporter: miaojianlong
> Priority: Critical
>
> Hive Lookup Join with decimal type failed.
> {code:java}
> java.lang.NumberFormatException: Zero length BigInteger
> at java.math.BigInteger.<init>(BigInteger.java:302) ~[?:1.8.0_231]
> at
> org.apache.flink.table.data.DecimalData.fromUnscaledBytes(DecimalData.java:211)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.hive.shaded.formats.parquet.vector.ParquetDecimalVector.getDecimal(ParquetDecimalVector.java:50)
> ~[flink-connector-hive_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.data.vector.VectorizedColumnBatch.getDecimal(VectorizedColumnBatch.java:115)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.data.ColumnarRowData.getDecimal(ColumnarRowData.java:119)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.data.RowData.lambda$createFieldGetter$89bd9445$1(RowData.java:233)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.data.RowData.lambda$createFieldGetter$25774257$1(RowData.java:296)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.runtime.typeutils.RowDataSerializer.copyRowData(RowDataSerializer.java:170)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.runtime.typeutils.RowDataSerializer.copy(RowDataSerializer.java:131)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.runtime.typeutils.RowDataSerializer.copy(RowDataSerializer.java:48)
> ~[flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.filesystem.FileSystemLookupFunction.checkCacheReload(FileSystemLookupFunction.java:138)
> [flink-table-blink_2.11-1.12.2.jar:1.12.2]
> at
> org.apache.flink.table.filesystem.FileSystemLookupFunction.eval(FileSystemLookupFunction.java:105)
> [flink-table-blink_2.11-1.12.2.jar:1.12.2] {code}
> Error exception place :
> {code:java}
> public BigInteger(byte[] val) {
> if (val.length == 0)
> throw new NumberFormatException("Zero length BigInteger");{code}
> Because of this
> code:org.apache.flink.connectors.hive.read.HiveInputFormatPartitionReader#hasNext
> {code:java}
> private boolean hasNext() throws IOException {
> if (inputSplits.length > 0) {
> if (hiveTableInputFormat.reachedEnd() && readingSplitId ==
> inputSplits.length - 1) {
> return false;
> } else if (hiveTableInputFormat.reachedEnd()) {
> readingSplitId++;
> hiveTableInputFormat.open(inputSplits[readingSplitId]);
> }
> return true;
> }
> return false;
> }{code}
> when we have more than one file in hive table and when we reachedEnd of the
> first inputSplits.Then at this time it will open the next inputSplit,but it
> did not trigger the batch reading of the file.Then after calling readRecord
> later, the data read is empty.For decimal type, it will trigger The above
> exception.In other scenarios, it will read more empty row.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)