[ https://issues.apache.org/jira/browse/FLINK-377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14301803#comment-14301803 ]
ASF GitHub Bot commented on FLINK-377: -------------------------------------- Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/202#issuecomment-72526181 I've tested the changes again, because I would really like to merge them The bin/pyflink3.sh script only works when called from the flink root dir ``` robert@robert-tower ...9-SNAPSHOT-bin/flink-0.9-SNAPSHOT/bin (git)-[papipr] % ./pyflink3.sh Error: Jar file: 'lib/flink-language-binding-0.9-SNAPSHOT.jar' does not exist. ``` This issue will be fixed soon because the `bin/flink` client will print all errors immediately (instead of asking the user to put a `-v`). For now, you can maybe add the `-v´ by default. ``` ./bin/pyflink3.sh pyflink.py Traceback (most recent call last): File "/tmp/flink_plan/plan.py", line 1, in <module> bullshit NameError: name 'bullshit' is not defined 20:16:20,658 WARN org.apache.hadoop.util.NativeCodeLoader - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Error: The main method caused an error. For a more detailed error message use the vebose output option '-v'. ``` The Python PlanBuilder seems to insist on using HDFS, even though I'm testing the code locally: ``` robert@robert-tower ...k-0.9-SNAPSHOT-bin/flink-0.9-SNAPSHOT (git)-[papipr] % ./bin/pyflink3.sh pyflink.py 20:25:57,440 WARN org.apache.hadoop.util.NativeCodeLoader - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Error: The main method caused an error. org.apache.flink.client.program.ProgramInvocationException: The main method caused an error. at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:449) at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:350) at org.apache.flink.client.program.Client.run(Client.java:242) at org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:389) at org.apache.flink.client.CliFrontend.run(CliFrontend.java:358) at org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1068) at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1092) Caused by: java.io.IOException: The given HDFS file URI (hdfs:/tmp/flink) did not describe the HDFS NameNode. The attempt to use a default HDFS configuration, as specified in the 'fs.hdfs.hdfsdefault' or 'fs.hdfs.hdfssite' config parameter failed due to the following problem: Either no default file system was registered, or the provided configuration contains no valid authority component (fs.default.name or fs.defaultFS) describing the (hdfs namenode) host and port. at org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFileSystem.java:287) at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:261) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.clearPath(PythonPlanBinder.java:135) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.distributeFiles(PythonPlanBinder.java:153) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.runPlan(PythonPlanBinder.java:101) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.main(PythonPlanBinder.java:78) 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:483) at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:434) ... 6 more ``` Apparently, using `env.execute(local=True)` resolves the problem. But leads to a new problem: ``` robert@robert-tower ...k-0.9-SNAPSHOT-bin/flink-0.9-SNAPSHOT (git)-[papipr] % ./bin/pyflink3.sh pyflink.py 02/02/2015 20:55:00 Job execution switched to status RUNNING. 02/02/2015 20:55:00 DataSource (ValueSource)(1/1) switched to SCHEDULED 02/02/2015 20:55:00 DataSource (ValueSource)(1/1) switched to DEPLOYING 02/02/2015 20:55:01 DataSource (ValueSource)(1/1) switched to RUNNING 02/02/2015 20:55:01 MapPartition (PythonFlatMap -> PythonCombine)(1/1) switched to SCHEDULED 02/02/2015 20:55:01 MapPartition (PythonFlatMap -> PythonCombine)(1/1) switched to DEPLOYING 02/02/2015 20:55:01 DataSource (ValueSource)(1/1) switched to FINISHED 02/02/2015 20:55:01 MapPartition (PythonFlatMap -> PythonCombine)(1/1) switched to RUNNING 02/02/2015 20:55:05 MapPartition (PythonFlatMap -> PythonCombine)(1/1) switched to FAILED java.lang.RuntimeException: External process for task MapPartition (PythonFlatMap -> PythonCombine) terminated prematurely due to an error. Check log-files for details. at org.apache.flink.languagebinding.api.java.common.streaming.Streamer.streamBufferWithoutGroups(Streamer.java:189) at org.apache.flink.languagebinding.api.java.python.functions.PythonMapPartition.mapPartition(PythonMapPartition.java:55) at org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:98) at org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:496) at org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360) at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204) at java.lang.Thread.run(Thread.java:745) 02/02/2015 20:55:05 Job execution switched to status FAILING. 02/02/2015 20:55:05 GroupReduce (PythonGroupReducePreStep)(1/1) switched to CANCELED 02/02/2015 20:55:05 MapPartition (PythonGroupReduce)(1/1) switched to CANCELED 02/02/2015 20:55:05 DataSink(PrintSink)(1/1) switched to CANCELED 02/02/2015 20:55:05 Job execution switched to status FAILED. Error: The program execution failed: java.lang.RuntimeException: External process for task MapPartition (PythonFlatMap -> PythonCombine) terminated prematurely due to an error. Check log-files for details. at org.apache.flink.languagebinding.api.java.common.streaming.Streamer.streamBufferWithoutGroups(Streamer.java:189) at org.apache.flink.languagebinding.api.java.python.functions.PythonMapPartition.mapPartition(PythonMapPartition.java:55) at org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:98) at org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:496) at org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360) at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204) at java.lang.Thread.run(Thread.java:745) org.apache.flink.client.program.ProgramInvocationException: The program execution failed: java.lang.RuntimeException: External process for task MapPartition (PythonFlatMap -> PythonCombine) terminated prematurely due to an error. Check log-files for details. at org.apache.flink.languagebinding.api.java.common.streaming.Streamer.streamBufferWithoutGroups(Streamer.java:189) at org.apache.flink.languagebinding.api.java.python.functions.PythonMapPartition.mapPartition(PythonMapPartition.java:55) at org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:98) at org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:496) at org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360) at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204) at java.lang.Thread.run(Thread.java:745) at org.apache.flink.client.program.Client.run(Client.java:337) at org.apache.flink.client.program.Client.run(Client.java:296) at org.apache.flink.client.program.Client.run(Client.java:290) at org.apache.flink.client.program.ContextEnvironment.execute(ContextEnvironment.java:55) at org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:675) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.runPlan(PythonPlanBinder.java:102) at org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.main(PythonPlanBinder.java:78) 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:483) at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:434) at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:350) at org.apache.flink.client.program.Client.run(Client.java:242) at org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:389) at org.apache.flink.client.CliFrontend.run(CliFrontend.java:358) at org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1068) at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1092) ``` The log output says ``` Traceback (most recent call last): File "/tmp/tmp_4b1632777c5777ace317d51ffd521adc/flink/executor.py", line 38, in <module> operator._go() File "/tmp/tmp_4b1632777c5777ace317d51ffd521adc/flink/flink/functions/Function.py", line 73, in _go self._run() File "/tmp/tmp_4b1632777c5777ace317d51ffd521adc/flink/flink/functions/FlatMapFunction.py", line 30, in _run result = function(value, collector) TypeError: <lambda>() takes 1 positional argument but 2 were given ``` (probably the wc example in the documentation is outdated). I'll add another comment once I've looked deeper through the code. > Create a general purpose framework for language bindings > -------------------------------------------------------- > > Key: FLINK-377 > URL: https://issues.apache.org/jira/browse/FLINK-377 > Project: Flink > Issue Type: Improvement > Reporter: GitHub Import > Assignee: Chesnay Schepler > Labels: github-import > Fix For: pre-apache > > > A general purpose API to run operators with arbitrary binaries. > This will allow to run Stratosphere programs written in Python, JavaScript, > Ruby, Go or whatever you like. > We suggest using Google Protocol Buffers for data serialization. This is the > list of languages that currently support ProtoBuf: > https://code.google.com/p/protobuf/wiki/ThirdPartyAddOns > Very early prototype with python: > https://github.com/rmetzger/scratch/tree/learn-protobuf (basically testing > protobuf) > For Ruby: https://github.com/infochimps-labs/wukong > Two new students working at Stratosphere (@skunert and @filiphaase) are > working on this. > The reference binding language will be for Python, but other bindings are > very welcome. > The best name for this so far is "stratosphere-lang-bindings". > I created this issue to track the progress (and give everybody a chance to > comment on this) > ---------------- Imported from GitHub ---------------- > Url: https://github.com/stratosphere/stratosphere/issues/377 > Created by: [rmetzger|https://github.com/rmetzger] > Labels: enhancement, > Assignee: [filiphaase|https://github.com/filiphaase] > Created at: Tue Jan 07 19:47:20 CET 2014 > State: open -- This message was sent by Atlassian JIRA (v6.3.4#6332)