[ https://issues.apache.org/jira/browse/FLINK-20951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17272893#comment-17272893 ]
Jude Zhu commented on FLINK-20951: ---------------------------------- I' alread set *tableEnv.getConfig().getConfiguration().setString("table.exec.hive.fallback-mapred-reader", "true");* *when i only append one partition condition。all of two* *partition*** the exception is: 21/01/27 20:53:26 ERROR base.source.reader.fetcher.SplitFetcherManager: Received uncaught exception.21/01/27 20:53:26 ERROR base.source.reader.fetcher.SplitFetcherManager: Received uncaught exception.java.lang.RuntimeException: SplitFetcher thread 79 received unexpected exception while polling the records at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:146) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:101) [TalosEngine-Prod-1.12-2.1.11.jar:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_121] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_121] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_121] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_121] at java.lang.Thread.run(Thread.java:745) [?:1.8.0_121]Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:858) ~[hadoop-hdfs-2.6.0-cdh5.11.0.jar:?] at org.apache.hadoop.hdfs.DFSInputStream.close(DFSInputStream.java:707) ~[hadoop-hdfs-2.6.0-cdh5.11.0.jar:?] at java.io.FilterInputStream.close(FilterInputStream.java:181) ~[?:1.8.0_121] at parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:432) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:385) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:371) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.getSplit(ParquetRecordReaderWrapper.java:252) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:99) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:85) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:72) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveMapredSplitReader.<init>(HiveMapredSplitReader.java:113) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter$HiveReader.<init>(HiveBulkFormatAdapter.java:309) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter$HiveReader.<init>(HiveBulkFormatAdapter.java:288) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter$HiveMapRedBulkFormat.createReader(HiveBulkFormatAdapter.java:265) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter$HiveMapRedBulkFormat.createReader(HiveBulkFormatAdapter.java:258) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter.createReader(HiveBulkFormatAdapter.java:108) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter.createReader(HiveBulkFormatAdapter.java:63) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connector.file.src.impl.FileSourceSplitReader.checkSplitOrStartNext(FileSourceSplitReader.java:112) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connector.file.src.impl.FileSourceSplitReader.fetch(FileSourceSplitReader.java:65) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:56) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:138) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] ... 6 more and 21/01/27 20:53:26 ERROR runtime.io.network.partition.ResultPartition: Error during release of result subpartition: /data1/yarn/local/usercache/venus/appcache/application_1605868815011_5695885/flink-netty-shuffle-a2999438-b901-472e-a19a-99d025c13c20/0ae8a74acd295d92c30242edc432e358.channel21/01/27 20:53:26 ERROR runtime.io.network.partition.ResultPartition: Error during release of result subpartition: /data1/yarn/local/usercache/venus/appcache/application_1605868815011_5695885/flink-netty-shuffle-a2999438-b901-472e-a19a-99d025c13c20/0ae8a74acd295d92c30242edc432e358.channeljava.nio.file.NoSuchFileException: /data1/yarn/local/usercache/venus/appcache/application_1605868815011_5695885/flink-netty-shuffle-a2999438-b901-472e-a19a-99d025c13c20/0ae8a74acd295d92c30242edc432e358.channel at sun.nio.fs.UnixException.translateToIOException(UnixException.java:86) ~[?:1.8.0_121] at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) ~[?:1.8.0_121] at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) ~[?:1.8.0_121] at sun.nio.fs.UnixFileSystemProvider.implDelete(UnixFileSystemProvider.java:244) ~[?:1.8.0_121] at sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103) ~[?:1.8.0_121] at java.nio.file.Files.delete(Files.java:1126) ~[?:1.8.0_121] at org.apache.flink.runtime.io.network.partition.FileChannelBoundedData.close(FileChannelBoundedData.java:97) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.BoundedBlockingSubpartition.checkReaderReferencesAndDispose(BoundedBlockingSubpartition.java:253) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.BoundedBlockingSubpartition.release(BoundedBlockingSubpartition.java:205) ~[TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.BufferWritingResultPartition.releaseInternal(BufferWritingResultPartition.java:231) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.ResultPartition.release(ResultPartition.java:226) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.ResultPartitionManager.releasePartition(ResultPartitionManager.java:86) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.io.network.partition.ResultPartition.fail(ResultPartition.java:242) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.taskmanager.Task.releaseResources(Task.java:941) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:873) [TalosEngine-Prod-1.12-2.1.11.jar:?] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:570) [TalosEngine-Prod-1.12-2.1.11.jar:?] at java.lang.Thread.run(Thread.java:745) [?:1.8.0_121] > IllegalArgumentException when reading Hive parquet table if condition not > contain all partitioned fields > -------------------------------------------------------------------------------------------------------- > > Key: FLINK-20951 > URL: https://issues.apache.org/jira/browse/FLINK-20951 > Project: Flink > Issue Type: Bug > Components: Connectors / Hive, Table SQL / Runtime > Affects Versions: 1.12.0 > Environment: flink 1.12.0 release-12 > sql-cli > Reporter: YUJIANBO > Priority: Major > > The production hive table is partitioned by two fields:datekey and event > I have do this test by Flink-sql-cli:(Spark Sql All is OK) > (1)First: > SELECT vid From table_A WHERE datekey = '20210112' AND event = 'XXX' AND vid > = 'aaaaaa'; (OK) > SELECT vid From table_A WHERE datekey = '20210112' AND vid = 'aaaaaa'; > (Error) > (2)Second: > SELECT vid From table_B WHERE datekey = '20210112' AND event = 'YYY' AND vid > = 'bbbbbb'; (OK) > SELECT vid From table_B WHERE datekey = '20210112' AND vid = 'bbbbbb'; > (Error) > The exception is: > {code} > java.lang.RuntimeException: One or more fetchers have encountered exception > at > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:199) > at > org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:154) > at > org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:116) > at > org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:273) > at > org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:67) > at > org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:395) > at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:191) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:609) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:573) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:755) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:570) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.RuntimeException: SplitFetcher thread 19 received > unexpected exception while polling the records > at > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:146) > at > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:101) > 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:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > ... 1 more > Caused by: java.lang.IllegalArgumentException > at java.nio.Buffer.position(Buffer.java:244) > at > org.apache.flink.hive.shaded.parquet.io.api.Binary$ByteBufferBackedBinary.getBytes(Binary.java:424) > at > org.apache.flink.hive.shaded.formats.parquet.vector.reader.BytesColumnReader.readBatchFromDictionaryIds(BytesColumnReader.java:79) > at > org.apache.flink.hive.shaded.formats.parquet.vector.reader.BytesColumnReader.readBatchFromDictionaryIds(BytesColumnReader.java:33) > at > org.apache.flink.hive.shaded.formats.parquet.vector.reader.AbstractColumnReader.readToVector(AbstractColumnReader.java:199) > at > org.apache.flink.hive.shaded.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.nextBatch(ParquetVectorizedInputFormat.java:359) > at > org.apache.flink.hive.shaded.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.readBatch(ParquetVectorizedInputFormat.java:328) > 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:56) > at > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:138) > ... 6 more > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)