Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
1.13.1
Description
PartitionPruner compile code use AppClassLoader (Obtained by getClass.getClassLoader)
but org.apache.flink.table.functions.hive.HiveGenericUDF is in user's jar, so classloader is UserCodeClassLoader,
So compile fail.
we need change
val function = genFunction.newInstance(getClass.getClassLoader)
to
val function = genFunction.newInstance(Thread.currentThread().getContextClassLoader)
The following is the error message:
org.apache.flink.util.FlinkRuntimeException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:76) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:98) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:69) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.utils.PartitionPruner$.prunePartitions(PartitionPruner.scala:112) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.utils.PartitionPruner.prunePartitions(PartitionPruner.scala) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.rules.logical.PushPartitionIntoTableSourceScanRule.lambda$onMatch$3(PushPartitionIntoTableSourceScanRule.java:163) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.rules.logical.PushPartitionIntoTableSourceScanRule.readPartitionFromCatalogWithoutFilterAndPrune(PushPartitionIntoTableSourceScanRule.java:373) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.rules.logical.PushPartitionIntoTableSourceScanRule.readPartitionFromCatalogAndPrune(PushPartitionIntoTableSourceScanRule.java:351) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.rules.logical.PushPartitionIntoTableSourceScanRule.readPartitionsAndPrune(PushPartitionIntoTableSourceScanRule.java:303) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.rules.logical.PushPartitionIntoTableSourceScanRule.onMatch(PushPartitionIntoTableSourceScanRule.java:171) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.AbstractRelOptPlanner.fireRule(AbstractRelOptPlanner.java:333) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:542) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepPlanner.applyRules(HepPlanner.java:407) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepPlanner.executeInstruction(HepPlanner.java:243) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepInstruction$RuleInstance.execute(HepInstruction.java:127) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepPlanner.executeProgram(HepPlanner.java:202) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.calcite.plan.hep.HepPlanner.findBestExp(HepPlanner.java:189) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkHepProgram.optimize(FlinkHepProgram.scala:69) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgram.optimize(FlinkHepRuleSetProgram.scala:87) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:63) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:60) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.Iterator$class.foreach(Iterator.scala:891) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractIterable.foreach(Iterable.scala:54) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:60) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:55) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.immutable.Range.foreach(Range.scala:160) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram.optimize(FlinkGroupProgram.scala:55) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.Iterator$class.foreach(Iterator.scala:891) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractIterable.foreach(Iterable.scala:54) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:87) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:58) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at scala.collection.immutable.List.foreach(List.scala:392) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:46) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:279) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:163) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1518) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:740) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:856) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:730) ~[flink-table_2.11-1.13.1.jar:1.13.1] at batchProcess.BatchSqlJob.main(BatchSqlJob.java:74) ~[xxxxxx-batch-streaming-integration-1.0-SNAPSHOT_v_1.jar:1.0-SNAPSHOT] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_102] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_102] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_102] at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_102] at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:357) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:223) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:812) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:246) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1054) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1132) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at java.security.AccessController.doPrivileged(Native Method) ~[?:1.8.0_102] at javax.security.auth.Subject.doAs(Subject.java:422) [?:1.8.0_102] at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1796) [hadoop-common-2.6.0U41.3-cdh5.10.0-hotfix1.jar:?] at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) [flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1132) [flink-dist_2.11-1.13.1.jar:1.13.1] Caused by: org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:74) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] ... 77 more Caused by: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:89) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:74) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:74) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] ... 77 more Caused by: org.codehaus.commons.compiler.CompileException: Line 32, Column 30: Cannot determine simple type name "org" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6833) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6594) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$1300(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$25.getType(UnitCompiler.java:8271) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6873) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$14400(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2$1.visitFieldAccess(UnitCompiler.java:6499) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2$1.visitFieldAccess(UnitCompiler.java:6494) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$FieldAccess.accept(Java.java:4310) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6494) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6490) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$Lvalue.accept(Java.java:4148) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6490) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$Rvalue.accept(Java.java:4116) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6855) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$14200(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2$1.visitAmbiguousName(UnitCompiler.java:6497) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2$1.visitAmbiguousName(UnitCompiler.java:6494) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$AmbiguousName.accept(Java.java:4224) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6494) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22$2.visitLvalue(UnitCompiler.java:6490) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$Lvalue.accept(Java.java:4148) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6490) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$Rvalue.accept(Java.java:4116) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9026) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5062) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4423) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4396) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4396) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5662) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3783) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3762) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3734) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3734) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2360) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$1800(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1494) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1487) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2874) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1567) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3388) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75) ~[flink-table_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:86) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:74) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) ~[flink-dist_2.11-1.13.1.jar:1.13.1] at org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:74) ~[flink-table-blink_2.11-1.13.1.jar:1.13.1] ... 77 more 2021-11-04 11:51:51,230 DEBUG org.apache.flink.table.runtime.generated.CompileUtils [] - Compiling: PartitionPruner$23 Code: public class PartitionPruner$23 extends org.apache.flink.api.common.functions.RichMapFunction { private transient org.apache.flink.table.runtime.typeutils.StringDataSerializer typeSerializer$7; private final org.apache.flink.table.data.binary.BinaryStringData str$9 = org.apache.flink.table.data.binary.BinaryStringData.fromString("20210927"); private final org.apache.flink.table.data.binary.BinaryStringData str$12 = org.apache.flink.table.data.binary.BinaryStringData.fromString("product_series"); private transient org.apache.flink.table.functions.hive.HiveGenericUDF function_org$apache$flink$table$functions$hive$HiveGenericUDF$00ff86955cfcc960a686ea8c68f6af97; private transient org.apache.flink.table.data.conversion.StringStringConverter converter$15; private transient org.apache.flink.table.data.conversion.StringStringConverter converter$16; private final org.apache.flink.table.data.binary.BinaryStringData str$19 = org.apache.flink.table.data.binary.BinaryStringData.fromString("KS"); public PartitionPruner$23(Object[] references) throws Exception { typeSerializer$7 = (((org.apache.flink.table.runtime.typeutils.StringDataSerializer) references[0])); function_org$apache$flink$table$functions$hive$HiveGenericUDF$00ff86955cfcc960a686ea8c68f6af97 = (((org.apache.flink.table.functions.hive.HiveGenericUDF) references[1])); converter$15 = (((org.apache.flink.table.data.conversion.StringStringConverter) references[2])); converter$16 = (((org.apache.flink.table.data.conversion.StringStringConverter) references[3])); } @Override public void open(org.apache.flink.configuration.Configuration parameters) throws Exception { function_org$apache$flink$table$functions$hive$HiveGenericUDF$00ff86955cfcc960a686ea8c68f6af97.open(new org.apache.flink.table.functions.ConstantFunctionContext(parameters)); converter$15.open(this.getClass().getClassLoader()); converter$16.open(this.getClass().getClassLoader()); } @Override public Object map(Object _in1) throws Exception { org.apache.flink.table.data.RowData in1 = (org.apache.flink.table.data.RowData) _in1; org.apache.flink.table.data.binary.BinaryStringData field$6; boolean isNull$6; org.apache.flink.table.data.binary.BinaryStringData field$8; boolean isNull$10; boolean result$11; org.apache.flink.table.data.binary.BinaryStringData field$13; boolean isNull$13; org.apache.flink.table.data.binary.BinaryStringData field$14; java.lang.String externalResult$17; org.apache.flink.table.data.binary.BinaryStringData result$18; boolean isNull$18; boolean isNull$20; boolean result$21; isNull$6 = in1.isNullAt(0); field$6 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; if (!isNull$6) { field$6 = ((org.apache.flink.table.data.binary.BinaryStringData) in1.getString(0)); } field$8 = field$6; if (!isNull$6) { field$8 = (org.apache.flink.table.data.binary.BinaryStringData) (typeSerializer$7.copy(field$8)); } isNull$13 = in1.isNullAt(1); field$13 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; if (!isNull$13) { field$13 = ((org.apache.flink.table.data.binary.BinaryStringData) in1.getString(1)); } field$14 = field$13; if (!isNull$13) { field$14 = (org.apache.flink.table.data.binary.BinaryStringData) (typeSerializer$7.copy(field$14)); } isNull$10 = isNull$6 || false; result$11 = false; if (!isNull$10) { result$11 = field$8.equals(((org.apache.flink.table.data.binary.BinaryStringData) str$9)); } boolean result$22 = false; boolean isNull$22 = false; if (!isNull$10 && !result$11) { // left expr is false, skip right expr } else { externalResult$17 = (java.lang.String) function_org$apache$flink$table$functions$hive$HiveGenericUDF$00ff86955cfcc960a686ea8c68f6af97 .eval(false ? null : ((java.lang.String) converter$15.toExternal((org.apache.flink.table.data.binary.BinaryStringData) ((org.apache.flink.table.data.binary.BinaryStringData) str$12))), isNull$6 ? null : ((java.lang.String) converter$16.toExternal((org.apache.flink.table.data.binary.BinaryStringData) field$8)), isNull$13 ? null : ((java.lang.String) converter$16.toExternal((org.apache.flink.table.data.binary.BinaryStringData) field$14))); isNull$18 = externalResult$17 == null; result$18 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; if (!isNull$18) { result$18 = (org.apache.flink.table.data.binary.BinaryStringData) converter$16.toInternalOrNull((java.lang.String) externalResult$17); } isNull$20 = isNull$18 || false; result$21 = false; if (!isNull$20) { result$21 = result$18.equals(((org.apache.flink.table.data.binary.BinaryStringData) str$19)); } if (!isNull$10 && !isNull$20) { result$22 = result$11 && result$21; isNull$22 = false; } else if (!isNull$10 && result$11 && isNull$20) { result$22 = false; isNull$22 = true; } else if (!isNull$10 && !result$11 && isNull$20) { result$22 = false; isNull$22 = false; } else if (isNull$10 && !isNull$20 && result$21) { result$22 = false; isNull$22 = true; } else if (isNull$10 && !isNull$20 && !result$21) { result$22 = false; isNull$22 = false; } else { result$22 = false; isNull$22 = true; } } return result$22; } @Override public void close() throws Exception { function_org$apache$flink$table$functions$hive$HiveGenericUDF$00ff86955cfcc960a686ea8c68f6af97.close(); } }