[ 
https://issues.apache.org/jira/browse/BEAM-7224?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16896831#comment-16896831
 ] 

Valentyn Tymofieiev commented on BEAM-7224:
-------------------------------------------

It seems that portable wordcount on Spark runner works in Loopback mode only. 
If we use a container environment, the wordcount fails on Spark runner (both on 
Python 2 and Python 3), but passes on Flink runner.

./gradlew :sdks:python:container:py3:docker
./gradlew :runners:spark:job-server:runShadow    # replace to 
:runners:flink:1.5:job-server:runShadow  for WC to pass.
./gradlew :sdks:python:test-suites:portable:py35:portableWordCountBatch  
-PjobEndpoint=localhost:8099
{noformat} 
 Task :sdks:python:test-suites:portable:py35:portableWordCountBatch
/usr/local/google/home/valentyn/projects/beam/cleanflink/beam/sdks/python/apache_beam/__init__.py:84:
 UserWarning: Some syntactic constructs of Python 3 are not yet fully supported 
by Apache Beam.
  'Some syntactic constructs of Python 3 are not yet fully supported by '
INFO:root:Using latest locally built Python SDK docker image: 
valentyn-docker-apache.bintray.io/beam/python3:latest.
INFO:root:==================== <function lift_combiners at 0x7fb979b6d840> 
====================
INFO:root:==================== <function expand_sdf at 0x7fb979b6d8c8> 
====================
WARNING:root:Discarding unparseable args: ['--parallelism=2', 
'--shutdown_sources_on_final_watermark']
INFO:root:Job state changed to RUNNING
ERROR:root:java.lang.RuntimeException: Error received from SDK harness for 
instruction 2: Traceback (most recent call last):
  File "apache_beam/runners/common.py", line 782, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 594, in 
apache_beam.runners.common.PerWindowInvoker.invoke_process
  File "apache_beam/runners/common.py", line 666, in 
apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/iobase.py", line 
1041, in process
    self.writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
186, in open_writer
    return FileBasedSinkWriter(self, writer_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
390, in __init__
    self.temp_handle = self.sink.open(temp_shard_path)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/textio.py", line 
391, in open
    file_handle = super(_TextSink, self).open(temp_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
129, in open
    return FileSystems.create(temp_path, self.mime_type, self.compression_type)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/filesystems.py", 
line 203, in create
    return filesystem.create(path, mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 151, in create
    return self._path_open(path, 'wb', mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 134, in _path_open
    raw_file = open(path, mode)
FileNotFoundError: [Errno 2] No such file or directory: 
'/tmp/beam-temp-py-wordcount-direct-1590df80b36011e98bf3f4939fefdbd5/ab0591ee-f8b9-471f-8edd-4b4a89481bc2.py-wordcount-direct'

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 157, in _execute
    response = task()
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 190, in <lambda>
    self._execute(lambda: worker.do_instruction(work), work)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 342, in do_instruction
    request.instruction_id)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 368, in process_bundle
    bundle_processor.process_bundle(instruction_id))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 593, in process_bundle
    data.ptransform_id].process_encoded(data.data)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 143, in process_encoded
    self.output(decoded_value)
  File "apache_beam/runners/worker/operations.py", line 255, in 
apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 256, in 
apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 143, in 
apache_beam.runners.worker.operations.SingletonConsumerSet.receive
  File "apache_beam/runners/worker/operations.py", line 593, in 
apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/worker/operations.py", line 594, in 
apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/common.py", line 778, in 
apache_beam.runners.common.DoFnRunner.receive
  File "apache_beam/runners/common.py", line 784, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 851, in 
apache_beam.runners.common.DoFnRunner._reraise_augmented
  File "/usr/local/lib/python3.5/site-packages/future/utils/__init__.py", line 
419, in raise_with_traceback
    raise exc.with_traceback(traceback)
  File "apache_beam/runners/common.py", line 782, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 594, in 
apache_beam.runners.common.PerWindowInvoker.invoke_process
  File "apache_beam/runners/common.py", line 666, in 
apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/iobase.py", line 
1041, in process
    self.writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
186, in open_writer
    return FileBasedSinkWriter(self, writer_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
390, in __init__
    self.temp_handle = self.sink.open(temp_shard_path)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/textio.py", line 
391, in open
    file_handle = super(_TextSink, self).open(temp_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
129, in open
    return FileSystems.create(temp_path, self.mime_type, self.compression_type)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/filesystems.py", 
line 203, in create
    return filesystem.create(path, mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 151, in create
    return self._path_open(path, 'wb', mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 134, in _path_open
    raw_file = open(path, mode)
RuntimeError: FileNotFoundError: [Errno 2] No such file or directory: 
'/tmp/beam-temp-py-wordcount-direct-1590df80b36011e98bf3f4939fefdbd5/ab0591ee-f8b9-471f-8edd-4b4a89481bc2.py-wordcount-direct'
 [while running 'write/Write/WriteImpl/WriteBundles']

INFO:root:Job state changed to FAILED
Traceback (most recent call last):
  File "/usr/lib/python3.5/runpy.py", line 193, in _run_module_as_main
    "__main__", mod_spec)
  File "/usr/lib/python3.5/runpy.py", line 85, in _run_code
    exec(code, run_globals)
  File 
"/usr/local/google/home/valentyn/projects/beam/cleanflink/beam/sdks/python/apache_beam/examples/wordcount.py",
 line 135, in <module>
    run()
  File 
"/usr/local/google/home/valentyn/projects/beam/cleanflink/beam/sdks/python/apache_beam/examples/wordcount.py",
 line 115, in run
    result.wait_until_finish()
  File 
"/usr/local/google/home/valentyn/projects/beam/cleanflink/beam/sdks/python/apache_beam/runners/portability/portable_runner.py",
 line 454, in wait_until_finish
    self._job_id, self._state, self._last_error_message()))
RuntimeError: Pipeline 
BeamApp-valentyn-0731065440-26f48096_3137c576-3c34-4ce2-8aab-e62822b177a6 
failed in state FAILED: java.lang.RuntimeException: Error received from SDK 
harness for instruction 2: Traceback (most recent call last):
  File "apache_beam/runners/common.py", line 782, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 594, in 
apache_beam.runners.common.PerWindowInvoker.invoke_process
  File "apache_beam/runners/common.py", line 666, in 
apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/iobase.py", line 
1041, in process
    self.writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
186, in open_writer
    return FileBasedSinkWriter(self, writer_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
390, in __init__
    self.temp_handle = self.sink.open(temp_shard_path)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/textio.py", line 
391, in open
    file_handle = super(_TextSink, self).open(temp_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
129, in open
    return FileSystems.create(temp_path, self.mime_type, self.compression_type)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/filesystems.py", 
line 203, in create
    return filesystem.create(path, mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 151, in create
    return self._path_open(path, 'wb', mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 134, in _path_open
    raw_file = open(path, mode)
FileNotFoundError: [Errno 2] No such file or directory: 
'/tmp/beam-temp-py-wordcount-direct-1590df80b36011e98bf3f4939fefdbd5/ab0591ee-f8b9-471f-8edd-4b4a89481bc2.py-wordcount-direct'

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 157, in _execute
    response = task()
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 190, in <lambda>
    self._execute(lambda: worker.do_instruction(work), work)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 342, in do_instruction
    request.instruction_id)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/sdk_worker.py",
 line 368, in process_bundle
    bundle_processor.process_bundle(instruction_id))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 593, in process_bundle
    data.ptransform_id].process_encoded(data.data)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/runners/worker/bundle_processor.py",
 line 143, in process_encoded
    self.output(decoded_value)
  File "apache_beam/runners/worker/operations.py", line 255, in 
apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 256, in 
apache_beam.runners.worker.operations.Operation.output
  File "apache_beam/runners/worker/operations.py", line 143, in 
apache_beam.runners.worker.operations.SingletonConsumerSet.receive
  File "apache_beam/runners/worker/operations.py", line 593, in 
apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/worker/operations.py", line 594, in 
apache_beam.runners.worker.operations.DoOperation.process
  File "apache_beam/runners/common.py", line 778, in 
apache_beam.runners.common.DoFnRunner.receive
  File "apache_beam/runners/common.py", line 784, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 851, in 
apache_beam.runners.common.DoFnRunner._reraise_augmented
  File "/usr/local/lib/python3.5/site-packages/future/utils/__init__.py", line 
419, in raise_with_traceback
    raise exc.with_traceback(traceback)
  File "apache_beam/runners/common.py", line 782, in 
apache_beam.runners.common.DoFnRunner.process
  File "apache_beam/runners/common.py", line 594, in 
apache_beam.runners.common.PerWindowInvoker.invoke_process
  File "apache_beam/runners/common.py", line 666, in 
apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/iobase.py", line 
1041, in process
    self.writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
186, in open_writer
    return FileBasedSinkWriter(self, writer_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
390, in __init__
    self.temp_handle = self.sink.open(temp_shard_path)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/textio.py", line 
391, in open
    file_handle = super(_TextSink, self).open(temp_path)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/options/value_provider.py", 
line 137, in _f
    return fnc(self, *args, **kwargs)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/filebasedsink.py", line 
129, in open
    return FileSystems.create(temp_path, self.mime_type, self.compression_type)
  File "/usr/local/lib/python3.5/site-packages/apache_beam/io/filesystems.py", 
line 203, in create
    return filesystem.create(path, mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 151, in create
    return self._path_open(path, 'wb', mime_type, compression_type)
  File 
"/usr/local/lib/python3.5/site-packages/apache_beam/io/localfilesystem.py", 
line 134, in _path_open
    raw_file = open(path, mode)
RuntimeError: FileNotFoundError: [Errno 2] No such file or directory: 
'/tmp/beam-temp-py-wordcount-direct-1590df80b36011e98bf3f4939fefdbd5/ab0591ee-f8b9-471f-8edd-4b4a89481bc2.py-wordcount-direct'
 [while running 'write/Write/WriteImpl/WriteBundles']
{noformat}


> Enable Python3 tests for Spark
> ------------------------------
>
>                 Key: BEAM-7224
>                 URL: https://issues.apache.org/jira/browse/BEAM-7224
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-spark
>            Reporter: Ankur Goenka
>            Assignee: Kyle Weaver
>            Priority: Major
>              Labels: portability-spark
>
> Add py3 integration tests for Spark



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to