Hi, 不好意思,不会cherry-pick到1.12了,因为这是个feature,在1.14及其之后的版本支持
Best, Jingsong On Fri, Nov 12, 2021 at 3:06 PM 陈卓宇 <[email protected]> wrote: > > 社区您好 我通过代码debug已经定位到问题: > > > 在flink1.12.5版本下flink-orc_2.11模块下的org/apache/flink/orc/vector/AbstractOrcColumnVector.java文件 > 下createFlinkVector中是没有对ListColumnVector进行实现的,我到flink的master上看在2021/5/12由wangwei1025提交的pr进行了实现,现在想请问社区有没有打算对1.12.5版本的此次问题根据wangwei1025的提交进行补丁的修复 > > > 表字段: > > > > > string_tag string > > > > > number_tag number > > > > > boolean_tag boolean > > > > > datetime_tag datetime > > > > > arr_tag array<string> > > 字段这里我进行了转换,生成这个SQL ,我发现具有array<string>类型的表读取就是失败的 > SQL:CREATE TABLE smarttag_base_table_5 ( > distinct_id BIGINT, > xwho VARCHAR, > string_tag string, > number_tag decimal, > boolean_tag integer, > datetime_tag bigint, > arr_tag ARRAY<STRING>, > ds INTEGER > ) WITH ( > 'connector' = > 'filesystem', -- 必选: > 指定连接器类型 > 'path' = > 'hdfs://ark1:8020/tmp/usertag/20211029/db_31abd9593e9983ec/orcfile/smarttag_base_table_5/', > -- 必选: 指向目录的路径 > 'format' = > 'orc' > -- 必选: 文件系统连接器需要指定格式,请查阅 表格式 部分以获取更多细节 > ) > > > > > > 报错:Unsupport vector: org.apache.hadoop.hive.ql.exec.vector.ListColumnVector > 我看是因为array<string>字段不支持导致的 但是 为什么会报hive的异常 > source到hdfs的一张orc的表 > > 陈卓宇 > > > > > > > > ------------------ 原始邮件 ------------------ > 发件人: > "user-zh" > > <[email protected]>; > 发送时间: 2021年11月12日(星期五) 上午10:59 > 收件人: "flink中文邮件组"<[email protected]>; > > 主题: Re: Flinksql 多表进行full join 出现异常 > > > > Hi! > > 感谢反馈问题。这看起来其实和 join 无关,应该是与 source 有关。方便的话,能否把 source 表的 > ddl(包含每个字段的类型,字段名如果敏感可以重命名一下)和其他信息(例如 source 表以什么格式存储)分享在邮件里? > > 陈卓宇 <[email protected]> 于2021年11月11日周四 下午9:44写道: > > > 场景:进行多表的full join失败 > > > > > > 报错: > > java.lang.RuntimeException: Failed to fetch next result > > > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:109) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultIterator.hasNext(CollectResultIterator.java:80) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.table.planner.sinks.SelectTableSinkBase$RowIteratorWrapper.hasNext(SelectTableSinkBase.java:117) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.table.api.internal.TableResultImpl$CloseableRowIteratorWrapper.hasNext(TableResultImpl.java:350) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.table.utils.PrintUtils.printAsTableauForm(PrintUtils.java:149) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.table.api.internal.TableResultImpl.print(TableResultImpl.java:154) > > &nbsp;&nbsp; &nbsp;at > TableAPI.envReadFileSysteam(TableAPI.java:441) > > &nbsp;&nbsp; &nbsp;at > sun.reflect.NativeMethodAccessorImpl.invoke0(Native > > Method) > > &nbsp;&nbsp; &nbsp;at > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > > &nbsp;&nbsp; &nbsp;at > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > &nbsp;&nbsp; &nbsp;at > java.lang.reflect.Method.invoke(Method.java:498) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) > > &nbsp;&nbsp; &nbsp;at > > > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) > > &nbsp;&nbsp; &nbsp;at > > org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54) > > &nbsp;&nbsp; &nbsp;at > org.junit.rules.RunRules.evaluate(RunRules.java:20) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) > > &nbsp;&nbsp; &nbsp;at > > org.junit.runners.ParentRunner.run(ParentRunner.java:413) > > &nbsp;&nbsp; &nbsp;at > org.junit.runner.JUnitCore.run(JUnitCore.java:137) > > &nbsp;&nbsp; &nbsp;at > > > com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) > > &nbsp;&nbsp; &nbsp;at > > > com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33) > > &nbsp;&nbsp; &nbsp;at > > > com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:221) > > &nbsp;&nbsp; &nbsp;at > > com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54) > > &nbsp;&nbsp; &nbsp;at > sun.reflect.NativeMethodAccessorImpl.invoke0(Native > > Method) > > &nbsp;&nbsp; &nbsp;at > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > > &nbsp;&nbsp; &nbsp;at > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > &nbsp;&nbsp; &nbsp;at > java.lang.reflect.Method.invoke(Method.java:498) > > &nbsp;&nbsp; &nbsp;at > > > com.intellij.rt.execution.CommandLineWrapper.main(CommandLineWrapper.java:64) > > Caused by: java.io.IOException: Failed to fetch job execution result > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:169) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.next(CollectResultFetcher.java:118) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:106) > > &nbsp;&nbsp; &nbsp;... 39 more > > Caused by: java.util.concurrent.ExecutionException: > > org.apache.flink.runtime.client.JobExecutionException: Job execution > failed. > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) > > &nbsp;&nbsp; &nbsp;at > > java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:167) > > &nbsp;&nbsp; &nbsp;... 41 more > > Caused by: org.apache.flink.runtime.client.JobExecutionException: Job > > execution failed. > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$2(MiniClusterJobClient.java:117) > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602) > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614) > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.minicluster.MiniClusterJobClient.getJobExecutionResult(MiniClusterJobClient.java:114) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:166) > > &nbsp;&nbsp; &nbsp;... 41 more > > Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed > > by NoRestartBackoffTimeStrategy > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:118) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:80) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:233) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:224) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:215) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:666) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:89) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:446) > > &nbsp;&nbsp; &nbsp;at > sun.reflect.GeneratedMethodAccessor23.invoke(Unknown > > Source) > > &nbsp;&nbsp; &nbsp;at > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > &nbsp;&nbsp; &nbsp;at > java.lang.reflect.Method.invoke(Method.java:498) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158) > > &nbsp;&nbsp; &nbsp;at akka.japi.pf > > .UnitCaseStatement.apply(CaseStatements.scala:26) > > &nbsp;&nbsp; &nbsp;at akka.japi.pf > > .UnitCaseStatement.apply(CaseStatements.scala:21) > > &nbsp;&nbsp; &nbsp;at > > scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) > > &nbsp;&nbsp; &nbsp;at akka.japi.pf > > .UnitCaseStatement.applyOrElse(CaseStatements.scala:21) > > &nbsp;&nbsp; &nbsp;at > > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) > > &nbsp;&nbsp; &nbsp;at > > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) > > &nbsp;&nbsp; &nbsp;at > > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) > > &nbsp;&nbsp; &nbsp;at > akka.actor.Actor$class.aroundReceive(Actor.scala:517) > > &nbsp;&nbsp; &nbsp;at > > akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) > > &nbsp;&nbsp; &nbsp;at > > akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) > > &nbsp;&nbsp; &nbsp;at > akka.actor.ActorCell.invoke(ActorCell.scala:561) > > &nbsp;&nbsp; &nbsp;at > > akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) > > &nbsp;&nbsp; &nbsp;at > akka.dispatch.Mailbox.run(Mailbox.scala:225) > > &nbsp;&nbsp; &nbsp;at > akka.dispatch.Mailbox.exec(Mailbox.scala:235) > > &nbsp;&nbsp; &nbsp;at > > akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > > &nbsp;&nbsp; &nbsp;at > > > akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > > &nbsp;&nbsp; &nbsp;at > > akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > > &nbsp;&nbsp; &nbsp;at > > > akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > > Caused by: java.lang.RuntimeException: One or more fetchers have > > encountered exception > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:199) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:154) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:116) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:275) > > &nbsp;&nbsp; &nbsp;at org.apache.flink.streaming.runtime.io > > .StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:67) > > &nbsp;&nbsp; &nbsp;at org.apache.flink.streaming.runtime.io > > .StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:398) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:191) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:619) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:583) > > &nbsp;&nbsp; &nbsp;at > > org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:758) > > &nbsp;&nbsp; &nbsp;at > > org.apache.flink.runtime.taskmanager.Task.run(Task.java:573) > > &nbsp;&nbsp; &nbsp;at java.lang.Thread.run(Thread.java:748) > > Caused by: java.lang.RuntimeException: SplitFetcher thread 0 received > > unexpected exception while polling the records > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:146) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:101) > > &nbsp;&nbsp; &nbsp;at > > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > &nbsp;&nbsp; &nbsp;at > > java.util.concurrent.FutureTask.run(FutureTask.java:266) > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > > &nbsp;&nbsp; &nbsp;at > > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > > &nbsp;&nbsp; &nbsp;... 1 more > > Caused by: java.lang.UnsupportedOperationException: Unsupport vector: > > org.apache.hadoop.hive.ql.exec.vector.ListColumnVector > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.vector.AbstractOrcColumnVector.createFlinkVector(AbstractOrcColumnVector.java:73) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.OrcColumnarRowFileInputFormat.lambda$createPartitionedFormat$84717d21$1(OrcColumnarRowFileInputFormat.java:161) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.OrcColumnarRowFileInputFormat.createReaderBatch(OrcColumnarRowFileInputFormat.java:88) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.AbstractOrcFileInputFormat.createPoolOfBatches(AbstractOrcFileInputFormat.java:157) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.AbstractOrcFileInputFormat.createReader(AbstractOrcFileInputFormat.java:103) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.orc.AbstractOrcFileInputFormat.createReader(AbstractOrcFileInputFormat.java:52) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.file.src.impl.FileSourceSplitReader.checkSplitOrStartNext(FileSourceSplitReader.java:112) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.file.src.impl.FileSourceSplitReader.fetch(FileSourceSplitReader.java:65) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:56) > > &nbsp;&nbsp; &nbsp;at > > > org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:138) > > &nbsp;&nbsp; &nbsp;... 6 more > > > > 2021-11-11 > > > 21:30:32.431|INFO|org.apache.flink.runtime.blob.AbstractBlobCache|TransientBlobCache > > shutdown hook|close|240|Shutting down BLOB cache > > 2021-11-11 > > > 21:30:32.433|INFO|org.apache.flink.runtime.blob.AbstractBlobCache|PermanentBlobCache > > shutdown hook|close|240|Shutting down BLOB cache > > 2021-11-11 > > 21:30:32.447|INFO|org.apache.flink.runtime.blob.BlobServer|BlobServer > > shutdown hook|close|345|Stopped BLOB server at 0.0.0.0:60726 > > > > 进程已结束,退出代码为 -1 > > > > > > > > > > > > > > sql: > > select * from smarttag_base_table_3 FULL JOIN smarttag_base_table_2 on > > smarttag_base_table_3.distinct_id=smarttag_base_table_2.distinct_id > > &nbsp;FULL JOIN smarttag_derived_table_4 on > > smarttag_base_table_2.distinct_id=smarttag_derived_table_4.distinct_id > > &nbsp;FULL JOIN smarttag_derived_table_1 on > > smarttag_derived_table_4.distinct_id=smarttag_derived_table_1.distinct_id > > &nbsp;FULL JOIN smarttag_base_table_5 on > > smarttag_derived_table_1.distinct_id=smarttag_base_table_5.distinct_id > > > > > > > > 陈 > > > > > > &nbsp; -- Best, Jingsong Lee
