[ https://issues.apache.org/jira/browse/FLINK-35081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838424#comment-17838424 ]
xuyang commented on FLINK-35081: -------------------------------- I think this bug is same with FLINK-28693 > CompileException when watermark definition contains coalesce and to_timestamp > built-in functions > ------------------------------------------------------------------------------------------------ > > Key: FLINK-35081 > URL: https://issues.apache.org/jira/browse/FLINK-35081 > Project: Flink > Issue Type: Bug > Components: Table SQL / Runtime > Affects Versions: 1.17.1 > Reporter: Grzegorz Kołakowski > Priority: Major > > I have a data stream in which event-time column can have two data formats. To > be able to define watermark on the table, I used coalesce and to_timestamp > built-in functions as shown below: > {code:sql} > create table test ( > `@timestamp` VARCHAR, > __rowtime AS coalesce( > to_timestamp(`@timestamp`, 'yyyy-MM-dd''T''HH:mm:ss'), > to_timestamp(`@timestamp`, 'yyyy-MM-dd''T''HH:mm:ss.SSS') > ), > watermark for __rowtime as __rowtime - INTERVAL '30' SECOND, > ... > ) with ( ... ) > {code} > The job failed with the following stacktrace: > {noformat} > org.apache.flink.runtime.JobException: Recovery is suppressed by > NoRestartBackoffTimeStrategy > at > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:139) > at > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:83) > at > org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:258) > at > org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:249) > at > org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:242) > at > org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:748) > at > org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:725) > at > org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:80) > at > org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:479) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(Unknown > Source) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(Unknown > Source) > at java.base/java.lang.reflect.Method.invoke(Unknown Source) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:309) > at > org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:307) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:222) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:84) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:168) > at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) > at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) > at scala.PartialFunction.applyOrElse(PartialFunction.scala:127) > at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126) > at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) > at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175) > at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) > at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) > at akka.actor.Actor.aroundReceive(Actor.scala:537) > at akka.actor.Actor.aroundReceive$(Actor.scala:535) > at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:579) > at akka.actor.ActorCell.invoke(ActorCell.scala:547) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) > at akka.dispatch.Mailbox.run(Mailbox.scala:231) > at akka.dispatch.Mailbox.exec(Mailbox.scala:243) > at java.base/java.util.concurrent.ForkJoinTask.doExec(Unknown Source) > at > java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(Unknown > Source) > at java.base/java.util.concurrent.ForkJoinPool.scan(Unknown Source) > at java.base/java.util.concurrent.ForkJoinPool.runWorker(Unknown Source) > at java.base/java.util.concurrent.ForkJoinWorkerThread.run(Unknown Source) > Caused by: java.lang.RuntimeException: Could not instantiate generated class > 'WatermarkGenerator$32' > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:74) > at > org.apache.flink.table.runtime.generated.GeneratedWatermarkGeneratorSupplier.createWatermarkGenerator(GeneratedWatermarkGeneratorSupplier.java:62) > at > org.apache.flink.streaming.api.operators.source.ProgressiveTimestampsAndWatermarks.createMainOutput(ProgressiveTimestampsAndWatermarks.java:109) > at > org.apache.flink.streaming.api.operators.SourceOperator.initializeMainOutput(SourceOperator.java:460) > at > org.apache.flink.streaming.api.operators.SourceOperator.emitNextNotReading(SourceOperator.java:436) > at > org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:412) > at > org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68) > at > org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:550) > at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:839) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:788) > at > org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:952) > at > org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:931) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:745) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562) > at java.base/java.lang.Thread.run(Unknown Source) > Caused by: 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:94) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:101) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:68) > ... 16 more > Caused by: > org.apache.flink.shaded.guava30.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.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2051) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache.get(LocalCache.java:3962) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4859) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:92) > ... 18 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:107) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$0(CompileUtils.java:92) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4864) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155) > at > org.apache.flink.shaded.guava30.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045) > ... 21 more > Caused by: org.codehaus.commons.compiler.CompileException: Line 31, Column > 130: Cannot determine simple type name "org" > at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12211) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6833) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6594) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6607) > at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6573) > at org.codehaus.janino.UnitCompiler.access$13900(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6481) > at > org.codehaus.janino.UnitCompiler$22$1.visitReferenceType(UnitCompiler.java:6476) > at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3928) > at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6476) > at org.codehaus.janino.UnitCompiler$22.visitType(UnitCompiler.java:6469) > at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3927) > at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) > at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:7121) > at org.codehaus.janino.UnitCompiler.access$17000(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6529) > at > org.codehaus.janino.UnitCompiler$22$2.visitNewClassInstance(UnitCompiler.java:6490) > at org.codehaus.janino.Java$NewClassInstance.accept(Java.java:5190) > at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6490) > at org.codehaus.janino.UnitCompiler$22.visitRvalue(UnitCompiler.java:6469) > at org.codehaus.janino.Java$Rvalue.accept(Java.java:4116) > at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6469) > at > org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9237) > at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9123) > at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9025) > at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5062) > at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4423) > at > org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4396) > at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) > at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4396) > at > org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5662) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3783) > at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3762) > at > org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3734) > at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5073) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3734) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2360) > at org.codehaus.janino.UnitCompiler.access$1800(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1494) > at > org.codehaus.janino.UnitCompiler$6.visitExpressionStatement(UnitCompiler.java:1487) > at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2874) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1487) > at > org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1567) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3388) > at > org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1357) > at > org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1330) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:822) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:432) > at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:215) > at > org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:411) > at > org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:406) > at > org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1414) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:406) > at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:378) > at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:237) > at > org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:465) > at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:216) > at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:207) > at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:80) > at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:75) > at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:104) > ... 27 more > {noformat} > > > {code:java} > /* 1 */ > /* 2 */ public final class WatermarkGenerator$64 > /* 3 */ extends > org.apache.flink.table.runtime.generated.WatermarkGenerator { > /* 4 */ > /* 5 */ private transient > org.apache.flink.table.runtime.typeutils.StringDataSerializer > typeSerializer$66; > /* 6 */ > /* 7 */ private final > org.apache.flink.table.data.binary.BinaryStringData str$68 = > org.apache.flink.table.data.binary.BinaryStringData.fromString("yyyy-MM-dd'T'HH:mm:ss"); > /* 8 */ > /* 9 */ > /* 10 */ private final > org.apache.flink.table.data.binary.BinaryStringData str$72 = > org.apache.flink.table.data.binary.BinaryStringData.fromString("yyyy-MM-dd'T'HH:mm:ss.SSS"); > /* 11 */ > /* 12 */ private transient > org.apache.flink.table.runtime.functions.scalar.CoalesceFunction > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096; > /* 13 */ > /* 14 */ private transient > org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context > /* 15 */ context; > /* 16 */ > /* 17 */ > /* 18 */ public WatermarkGenerator$64(Object[] references) throws > Exception { > /* 19 */ typeSerializer$66 = > (((org.apache.flink.table.runtime.typeutils.StringDataSerializer) > references[0])); > /* 20 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096 > = (((org.apache.flink.table.runtime.functions.scalar.CoalesceFunction) > references[1])); > /* 21 */ > /* 22 */ int len = references.length; > /* 23 */ context = > /* 24 */ > (org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context) > references[len-1]; > /* 25 */ > /* 26 */ } > /* 27 */ > /* 28 */ @Override > /* 29 */ public void open(org.apache.flink.configuration.Configuration > parameters) throws Exception { > /* 30 */ > /* 31 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096.open(new > > org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGeneratorFunctionContextWrapper(context)); > /* 32 */ > /* 33 */ } > /* 34 */ > /* 35 */ @Override > /* 36 */ public Long > currentWatermark(org.apache.flink.table.data.RowData row) throws Exception { > /* 37 */ > /* 38 */ org.apache.flink.table.data.binary.BinaryStringData > field$65; > /* 39 */ boolean isNull$65; > /* 40 */ org.apache.flink.table.data.binary.BinaryStringData > field$67; > /* 41 */ boolean isNull$69; > /* 42 */ org.apache.flink.table.data.TimestampData result$70; > /* 43 */ boolean isNull$73; > /* 44 */ org.apache.flink.table.data.TimestampData result$74; > /* 45 */ org.apache.flink.table.data.TimestampData externalResult$76; > /* 46 */ org.apache.flink.table.data.TimestampData result$77; > /* 47 */ boolean isNull$77; > /* 48 */ boolean isNull$78; > /* 49 */ org.apache.flink.table.data.TimestampData result$79; > /* 50 */ > /* 51 */ isNull$65 = row.isNullAt(0); > /* 52 */ field$65 = > org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; > /* 53 */ if (!isNull$65) { > /* 54 */ field$65 = > ((org.apache.flink.table.data.binary.BinaryStringData) row.getString(0)); > /* 55 */ } > /* 56 */ field$67 = field$65; > /* 57 */ if (!isNull$65) { > /* 58 */ field$67 = > (org.apache.flink.table.data.binary.BinaryStringData) > (typeSerializer$66.copy(field$67)); > /* 59 */ } > /* 60 */ > /* 61 */ > /* 62 */ > /* 63 */ > /* 64 */ > /* 65 */ > /* 66 */ > /* 67 */ isNull$69 = isNull$65 || false; > /* 68 */ result$70 = null; > /* 69 */ if (!isNull$69) { > /* 70 */ > /* 71 */ try { > /* 72 */ > /* 73 */ result$70 = > /* 74 */ > org.apache.flink.table.utils.DateTimeUtils.parseTimestampData(field$67.toString(), > ((org.apache.flink.table.data.binary.BinaryStringData) str$68).toString()) > /* 75 */ ; > /* 76 */ } catch (Throwable ignored$71) { > /* 77 */ isNull$69 = true; > /* 78 */ result$70 = null; > /* 79 */ } > /* 80 */ > /* 81 */ isNull$69 = (result$70 == null); > /* 82 */ } > /* 83 */ > /* 84 */ > /* 85 */ > /* 86 */ > /* 87 */ isNull$73 = isNull$65 || false; > /* 88 */ result$74 = null; > /* 89 */ if (!isNull$73) { > /* 90 */ > /* 91 */ try { > /* 92 */ > /* 93 */ result$74 = > /* 94 */ > org.apache.flink.table.utils.DateTimeUtils.parseTimestampData(field$67.toString(), > ((org.apache.flink.table.data.binary.BinaryStringData) str$72).toString()) > /* 95 */ ; > /* 96 */ } catch (Throwable ignored$75) { > /* 97 */ isNull$73 = true; > /* 98 */ result$74 = null; > /* 99 */ } > /* 100 */ > /* 101 */ isNull$73 = (result$74 == null); > /* 102 */ } > /* 103 */ > /* 104 */ externalResult$76 = > (org.apache.flink.table.data.TimestampData) > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096 > /* 105 */ .eval(isNull$69 ? null : > ((org.apache.flink.table.data.TimestampData) result$70), isNull$73 ? null : > ((org.apache.flink.table.data.TimestampData) result$74)); > /* 106 */ > /* 107 */ isNull$77 = externalResult$76 == null; > /* 108 */ result$77 = null; > /* 109 */ if (!isNull$77) { > /* 110 */ result$77 = externalResult$76; > /* 111 */ } > /* 112 */ > /* 113 */ > /* 114 */ isNull$78 = isNull$77 || false; > /* 115 */ result$79 = null; > /* 116 */ if (!isNull$78) { > /* 117 */ > /* 118 */ > /* 119 */ result$79 = > org.apache.flink.table.data.TimestampData.fromEpochMillis(result$77.getMillisecond() > - ((long) 30000L), result$77.getNanoOfMillisecond()); > /* 120 */ > /* 121 */ > /* 122 */ } > /* 123 */ > /* 124 */ if (isNull$78) { > /* 125 */ return null; > /* 126 */ } else { > /* 127 */ return result$79.getMillisecond(); > /* 128 */ } > /* 129 */ } > /* 130 */ > /* 131 */ @Override > /* 132 */ public void close() throws Exception { > /* 133 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096.close(); > /* 134 */ } > /* 135 */ } > /* 136 */ > /* 1 */ > /* 2 */ public final class WatermarkGenerator$64 > /* 3 */ extends > org.apache.flink.table.runtime.generated.WatermarkGenerator { > /* 4 */ > /* 5 */ private transient > org.apache.flink.table.runtime.typeutils.StringDataSerializer > typeSerializer$66; > /* 6 */ > /* 7 */ private final > org.apache.flink.table.data.binary.BinaryStringData str$68 = > org.apache.flink.table.data.binary.BinaryStringData.fromString("yyyy-MM-dd'T'HH:mm:ss"); > /* 8 */ > /* 9 */ > /* 10 */ private final > org.apache.flink.table.data.binary.BinaryStringData str$72 = > org.apache.flink.table.data.binary.BinaryStringData.fromString("yyyy-MM-dd'T'HH:mm:ss.SSS"); > /* 11 */ > /* 12 */ private transient > org.apache.flink.table.runtime.functions.scalar.CoalesceFunction > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096; > /* 13 */ > /* 14 */ private transient > org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context > /* 15 */ context; > /* 16 */ > /* 17 */ > /* 18 */ public WatermarkGenerator$64(Object[] references) throws > Exception { > /* 19 */ typeSerializer$66 = > (((org.apache.flink.table.runtime.typeutils.StringDataSerializer) > references[0])); > /* 20 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096 > = (((org.apache.flink.table.runtime.functions.scalar.CoalesceFunction) > references[1])); > /* 21 */ > /* 22 */ int len = references.length; > /* 23 */ context = > /* 24 */ > (org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context) > references[len-1]; > /* 25 */ > /* 26 */ } > /* 27 */ > /* 28 */ @Override > /* 29 */ public void open(org.apache.flink.configuration.Configuration > parameters) throws Exception { > /* 30 */ > /* 31 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096.open(new > > org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGeneratorFunctionContextWrapper(context)); > /* 32 */ > /* 33 */ } > /* 34 */ > /* 35 */ @Override > /* 36 */ public Long > currentWatermark(org.apache.flink.table.data.RowData row) throws Exception { > /* 37 */ > /* 38 */ org.apache.flink.table.data.binary.BinaryStringData > field$65; > /* 39 */ boolean isNull$65; > /* 40 */ org.apache.flink.table.data.binary.BinaryStringData > field$67; > /* 41 */ boolean isNull$69; > /* 42 */ org.apache.flink.table.data.TimestampData result$70; > /* 43 */ boolean isNull$73; > /* 44 */ org.apache.flink.table.data.TimestampData result$74; > /* 45 */ org.apache.flink.table.data.TimestampData externalResult$76; > /* 46 */ org.apache.flink.table.data.TimestampData result$77; > /* 47 */ boolean isNull$77; > /* 48 */ boolean isNull$78; > /* 49 */ org.apache.flink.table.data.TimestampData result$79; > /* 50 */ > /* 51 */ isNull$65 = row.isNullAt(0); > /* 52 */ field$65 = > org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; > /* 53 */ if (!isNull$65) { > /* 54 */ field$65 = > ((org.apache.flink.table.data.binary.BinaryStringData) row.getString(0)); > /* 55 */ } > /* 56 */ field$67 = field$65; > /* 57 */ if (!isNull$65) { > /* 58 */ field$67 = > (org.apache.flink.table.data.binary.BinaryStringData) > (typeSerializer$66.copy(field$67)); > /* 59 */ } > /* 60 */ > /* 61 */ > /* 62 */ > /* 63 */ > /* 64 */ > /* 65 */ > /* 66 */ > /* 67 */ isNull$69 = isNull$65 || false; > /* 68 */ result$70 = null; > /* 69 */ if (!isNull$69) { > /* 70 */ > /* 71 */ try { > /* 72 */ > /* 73 */ result$70 = > /* 74 */ > org.apache.flink.table.utils.DateTimeUtils.parseTimestampData(field$67.toString(), > ((org.apache.flink.table.data.binary.BinaryStringData) str$68).toString()) > /* 75 */ ; > /* 76 */ } catch (Throwable ignored$71) { > /* 77 */ isNull$69 = true; > /* 78 */ result$70 = null; > /* 79 */ } > /* 80 */ > /* 81 */ isNull$69 = (result$70 == null); > /* 82 */ } > /* 83 */ > /* 84 */ > /* 85 */ > /* 86 */ > /* 87 */ isNull$73 = isNull$65 || false; > /* 88 */ result$74 = null; > /* 89 */ if (!isNull$73) { > /* 90 */ > /* 91 */ try { > /* 92 */ > /* 93 */ result$74 = > /* 94 */ > org.apache.flink.table.utils.DateTimeUtils.parseTimestampData(field$67.toString(), > ((org.apache.flink.table.data.binary.BinaryStringData) str$72).toString()) > /* 95 */ ; > /* 96 */ } catch (Throwable ignored$75) { > /* 97 */ isNull$73 = true; > /* 98 */ result$74 = null; > /* 99 */ } > /* 100 */ > /* 101 */ isNull$73 = (result$74 == null); > /* 102 */ } > /* 103 */ > /* 104 */ externalResult$76 = > (org.apache.flink.table.data.TimestampData) > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096 > /* 105 */ .eval(isNull$69 ? null : > ((org.apache.flink.table.data.TimestampData) result$70), isNull$73 ? null : > ((org.apache.flink.table.data.TimestampData) result$74)); > /* 106 */ > /* 107 */ isNull$77 = externalResult$76 == null; > /* 108 */ result$77 = null; > /* 109 */ if (!isNull$77) { > /* 110 */ result$77 = externalResult$76; > /* 111 */ } > /* 112 */ > /* 113 */ > /* 114 */ isNull$78 = isNull$77 || false; > /* 115 */ result$79 = null; > /* 116 */ if (!isNull$78) { > /* 117 */ > /* 118 */ > /* 119 */ result$79 = > org.apache.flink.table.data.TimestampData.fromEpochMillis(result$77.getMillisecond() > - ((long) 30000L), result$77.getNanoOfMillisecond()); > /* 120 */ > /* 121 */ > /* 122 */ } > /* 123 */ > /* 124 */ if (isNull$78) { > /* 125 */ return null; > /* 126 */ } else { > /* 127 */ return result$79.getMillisecond(); > /* 128 */ } > /* 129 */ } > /* 130 */ > /* 131 */ @Override > /* 132 */ public void close() throws Exception { > /* 133 */ > function_org$apache$flink$table$runtime$functions$scalar$CoalesceFunction$081acce440d9567a963370b8fb1c1096.close(); > /* 134 */ } > /* 135 */ } > /* 136 */ > {code} > -- This message was sent by Atlassian Jira (v8.20.10#820010)