Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-16687

Support Python UDF in Java Correlate

    XMLWordPrintableJSON

Details

    Description

       
      I try to run PyFlink UDF with SQL UNNEST, execution of job failed, I defined a source from element, and use UDF split the string to list.
      raise org.codehaus.commons.compiler.CompileException: Cannot determine simple type name "PythonScalarFunction$0"

      import os
      from pyflink.table.udf import udf
      from pyflink.datastream import StreamExecutionEnvironment
      from pyflink.table import StreamTableEnvironment, DataTypes, CsvTableSink
      
      @udf(input_types=[DataTypes.STRING()], result_type=DataTypes.ARRAY(DataTypes.STRING()))
      def format_string_to_array(item):
          return item.replace('[', '').replace(']', '').replace(', ', ',').split(',')
      
      if __name__ == '__main__':
          env = StreamExecutionEnvironment.get_execution_environment()            env.set_parallelism(1)
          st_env = StreamTableEnvironment.create(env)
          result_file = "result.csv"
          if os.path.exists(result_file):
              os.remove(result_file)
      
          st_env.register_table_sink("result_tab",  CsvTableSink(["id", "url"], [DataTypes.STRING(), DataTypes.STRING()],  result_file))
          st_env.register_function("format_string_to_array", format_string_to_array)
          tab = st_env.from_elements([("1", "['www.bing.com', 'www.google.com']"), ("2", "['www.taobao.com']")], ['id', 'urls'])
          st_env.register_table("temp_table", tab)
          st_env.sql_query("Select id, A.url from temp_table, UNNEST(format_string_to_array(temp_table.urls)) AS A(url)").insert_into("result_tab")    
          st_env.execute("udf")
      
      

       

      When I execute the program, I get the following exception:

       

      py4j.protocol.Py4JJavaError: An error occurred while calling o2.execute.
      : java.util.concurrent.ExecutionException: org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 5d63838ad2043bf4a5d0bca83623959d)
              at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
              at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
              at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1640)
              at org.apache.flink.streaming.api.environment.LocalStreamEnvironment.execute(LocalStreamEnvironment.java:74)
              at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1620)
              at org.apache.flink.table.executor.StreamExecutor.execute(StreamExecutor.java:50)
              at org.apache.flink.table.api.internal.TableEnvironmentImpl.execute(TableEnvironmentImpl.java:643)
              at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
              at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:498)
              at org.apache.flink.api.python.shaded.py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
              at org.apache.flink.api.python.shaded.py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
              at org.apache.flink.api.python.shaded.py4j.Gateway.invoke(Gateway.java:282)
              at org.apache.flink.api.python.shaded.py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
              at org.apache.flink.api.python.shaded.py4j.commands.CallCommand.execute(CallCommand.java:79)
              at org.apache.flink.api.python.shaded.py4j.GatewayConnection.run(GatewayConnection.java:238)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 5d63838ad2043bf4a5d0bca83623959d)
              at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:112)
              at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
              at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
              at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
              at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
              at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:874)
              at akka.dispatch.OnComplete.internal(Future.scala:264)
              at akka.dispatch.OnComplete.internal(Future.scala:261)
              at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191)
              at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188)
              at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
              at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:74)
              at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
              at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
              at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:572)
              at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:22)
              at akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:21)
              at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:436)
              at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:435)
              at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
              at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
              at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
              at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
              at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
              at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
              at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
              at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
              at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
              at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
              at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
              at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
              at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
              at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:147)
              at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:110)
              ... 31 more
      Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
              at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:110)
              at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:76)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:192)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:186)
              at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:180)
              at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:484)
              at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:380)
              at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
              at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:498)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:279)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:194)
              at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
              at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
              at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
              at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
              at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
              at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
              at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
              at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
              at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
              at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
              at akka.actor.ActorCell.invoke(ActorCell.scala:561)
              at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
              at akka.dispatch.Mailbox.run(Mailbox.scala:225)
              at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
              ... 4 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.codegen.Compiler$class.compile(Compiler.scala:36)
              at org.apache.flink.table.runtime.CRowCorrelateProcessRunner.compile(CRowCorrelateProcessRunner.scala:35)
              at org.apache.flink.table.runtime.CRowCorrelateProcessRunner.open(CRowCorrelateProcessRunner.scala:58)
              at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36)
              at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:102)
              at org.apache.flink.streaming.api.operators.ProcessOperator.open(ProcessOperator.java:56)
              at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007)
              at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454)
              at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94)
              at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449)
              at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461)
              at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)
              at org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.codehaus.commons.compiler.CompileException: Line 6, Column 31: Cannot determine simple type name "PythonScalarFunction$0"
              at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12124)
              at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6746)
              at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6507)
              at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6486)
              at org.codehaus.janino.UnitCompiler.access$13800(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$21$1.visitReferenceType(UnitCompiler.java:6394)
              at org.codehaus.janino.UnitCompiler$21$1.visitReferenceType(UnitCompiler.java:6389)
              at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3917)
              at org.codehaus.janino.UnitCompiler$21.visitType(UnitCompiler.java:6389)
              at org.codehaus.janino.UnitCompiler$21.visitType(UnitCompiler.java:6382)
              at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3916)
              at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6382)
              at org.codehaus.janino.UnitCompiler.access$1300(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$24.getType(UnitCompiler.java:8184)
              at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6786)
              at org.codehaus.janino.UnitCompiler.access$14300(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$21$2$1.visitFieldAccess(UnitCompiler.java:6412)
              at org.codehaus.janino.UnitCompiler$21$2$1.visitFieldAccess(UnitCompiler.java:6407)
              at org.codehaus.janino.Java$FieldAccess.accept(Java.java:4299)
              at org.codehaus.janino.UnitCompiler$21$2.visitLvalue(UnitCompiler.java:6407)
              at org.codehaus.janino.UnitCompiler$21$2.visitLvalue(UnitCompiler.java:6403)
              at org.codehaus.janino.Java$Lvalue.accept(Java.java:4137)
              at org.codehaus.janino.UnitCompiler$21.visitRvalue(UnitCompiler.java:6403)
              at org.codehaus.janino.UnitCompiler$21.visitRvalue(UnitCompiler.java:6382)
              at org.codehaus.janino.Java$Rvalue.accept(Java.java:4105)
              at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6382)
              at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:6768)
              at org.codehaus.janino.UnitCompiler.access$14100(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$21$2$1.visitAmbiguousName(UnitCompiler.java:6410)
              at org.codehaus.janino.UnitCompiler$21$2$1.visitAmbiguousName(UnitCompiler.java:6407)
              at org.codehaus.janino.Java$AmbiguousName.accept(Java.java:4213)
              at org.codehaus.janino.UnitCompiler$21$2.visitLvalue(UnitCompiler.java:6407)
              at org.codehaus.janino.UnitCompiler$21$2.visitLvalue(UnitCompiler.java:6403)
              at org.codehaus.janino.Java$Lvalue.accept(Java.java:4137)
              at org.codehaus.janino.UnitCompiler$21.visitRvalue(UnitCompiler.java:6403)
              at org.codehaus.janino.UnitCompiler$21.visitRvalue(UnitCompiler.java:6382)
              at org.codehaus.janino.Java$Rvalue.accept(Java.java:4105)
              at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:6382)
              at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8939)
              at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5060)
              at org.codehaus.janino.UnitCompiler.access$9100(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4421)
              at org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4394)
              at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5062)
              at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4394)
              at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5575)
              at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3781)
              at org.codehaus.janino.UnitCompiler.access$5900(UnitCompiler.java:215)
              at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3760)
              at org.codehaus.janino.UnitCompiler$13.visitMethodInvocation(UnitCompiler.java:3732)
              ... 13 more
      

      Attachments

        Issue Links

          Activity

            People

              hxbks2ks Huang Xingbo
              maynewong mayne wong
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 20m
                  20m