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

test_dependency.py fails due to 'Failed to close remote bundle'

Attach filesAttach ScreenshotVotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    Description

      https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=23343&view=logs&j=bdd9ea51-4de2-506a-d4d9-f3930e4d2355&t=dd50312f-73b5-56b5-c172-4d81d03e2ef1&l=23922

      Caused by: java.lang.RuntimeException: Error while waiting for BeamPythonFunctionRunner flush
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator.invokeFinishBundle(AbstractPythonFunctionOperator.java:361)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator.checkInvokeFinishBundleByCount(AbstractPythonFunctionOperator.java:321)
      Sep 02 01:34:47 E                   	at org.apache.flink.table.runtime.operators.python.AbstractStatelessFunctionOperator.processElement(AbstractStatelessFunctionOperator.java:119)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
      Sep 02 01:34:47 E                   	at SourceConversion$38.processElement(Unknown Source)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:92)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:116)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:73)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:323)
      Sep 02 01:34:47 E                   Caused by: java.lang.RuntimeException: Failed to close remote bundle
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner.finishBundle(BeamPythonFunctionRunner.java:377)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner.flush(BeamPythonFunctionRunner.java:361)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator.lambda$invokeFinishBundle$2(AbstractPythonFunctionOperator.java:340)
      Sep 02 01:34:47 E                   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      Sep 02 01:34:47 E                   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      Sep 02 01:34:47 E                   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      Sep 02 01:34:47 E                   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      Sep 02 01:34:47 E                   	at java.lang.Thread.run(Thread.java:748)
      Sep 02 01:34:47 E                   Caused by: java.util.concurrent.ExecutionException: java.lang.RuntimeException: Error received from SDK harness for instruction 1: Traceback (most recent call last):
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 289, in _execute
      Sep 02 01:34:47 E                       response = task()
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 362, in <lambda>
      Sep 02 01:34:47 E                       lambda: self.create_worker().do_instruction(request), request)
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 606, in do_instruction
      Sep 02 01:34:47 E                       return getattr(self, request_type)(
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 644, in process_bundle
      Sep 02 01:34:47 E                       bundle_processor.process_bundle(instruction_id))
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 999, in process_bundle
      Sep 02 01:34:47 E                       input_op_by_transform_id[element.transform_id].process_encoded(
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 228, in process_encoded
      Sep 02 01:34:47 E                       self.output(decoded_value)
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 357, in apache_beam.runners.worker.operations.Operation.output
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 359, in apache_beam.runners.worker.operations.Operation.output
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 319, in apache_beam.runners.worker.operations.Operation.process
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/fn_execution/beam/beam_operations_slow.py", line 132, in process
      Sep 02 01:34:47 E                       self._output_processor.process_outputs(o, self.process_element(value))
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/fn_execution/table/operations.py", line 84, in process_element
      Sep 02 01:34:47 E                       return self.func(value)
      Sep 02 01:34:47 E                     File "<string>", line 1, in <lambda>
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/table/tests/test_dependency.py", line 53, in plus_two
      Sep 02 01:34:47 E                       from test_dependency_manage_lib import add_two
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 991, in _find_and_load
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 975, in _find_and_load_unlocked
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 671, in _load_unlocked
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/_pytest/assertion/rewrite.py", line 161, in exec_module
      Sep 02 01:34:47 E                       source_stat, co = _rewrite_test(fn, self.config)
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/_pytest/assertion/rewrite.py", line 351, in _rewrite_test
      Sep 02 01:34:47 E                       stat = os.stat(fn_)
      Sep 02 01:34:47 E                   FileNotFoundError: [Errno 2] No such file or directory: '/tmp/python-dist-266a4f9f-c350-41b8-b437-69b3c67435de/python-files/blob_p-0f11eb68b0611db5e1812d04788ca9c96d8e519c-4ad7d15b4215b93fd53b7ca4ab4bef4e/test_dependency_manage_lib.py'
      Sep 02 01:34:47 E                   
      Sep 02 01:34:47 E                   	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
      Sep 02 01:34:47 E                   	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
      Sep 02 01:34:47 E                   	at org.apache.beam.sdk.util.MoreFutures.get(MoreFutures.java:60)
      Sep 02 01:34:47 E                   	at org.apache.beam.runners.fnexecution.control.SdkHarnessClient$BundleProcessor$ActiveBundle.close(SdkHarnessClient.java:504)
      Sep 02 01:34:47 E                   	at org.apache.beam.runners.fnexecution.control.DefaultJobBundleFactory$SimpleStageBundleFactory$1.close(DefaultJobBundleFactory.java:555)
      Sep 02 01:34:47 E                   	at org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner.finishBundle(BeamPythonFunctionRunner.java:375)
      Sep 02 01:34:47 E                   	... 7 more
      Sep 02 01:34:47 E                   Caused by: java.lang.RuntimeException: Error received from SDK harness for instruction 1: Traceback (most recent call last):
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 289, in _execute
      Sep 02 01:34:47 E                       response = task()
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 362, in <lambda>
      Sep 02 01:34:47 E                       lambda: self.create_worker().do_instruction(request), request)
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 606, in do_instruction
      Sep 02 01:34:47 E                       return getattr(self, request_type)(
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 644, in process_bundle
      Sep 02 01:34:47 E                       bundle_processor.process_bundle(instruction_id))
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 999, in process_bundle
      Sep 02 01:34:47 E                       input_op_by_transform_id[element.transform_id].process_encoded(
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 228, in process_encoded
      Sep 02 01:34:47 E                       self.output(decoded_value)
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 357, in apache_beam.runners.worker.operations.Operation.output
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 359, in apache_beam.runners.worker.operations.Operation.output
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
      Sep 02 01:34:47 E                     File "apache_beam/runners/worker/operations.py", line 319, in apache_beam.runners.worker.operations.Operation.process
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/fn_execution/beam/beam_operations_slow.py", line 132, in process
      Sep 02 01:34:47 E                       self._output_processor.process_outputs(o, self.process_element(value))
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/fn_execution/table/operations.py", line 84, in process_element
      Sep 02 01:34:47 E                       return self.func(value)
      Sep 02 01:34:47 E                     File "<string>", line 1, in <lambda>
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/pyflink/table/tests/test_dependency.py", line 53, in plus_two
      Sep 02 01:34:47 E                       from test_dependency_manage_lib import add_two
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 991, in _find_and_load
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 975, in _find_and_load_unlocked
      Sep 02 01:34:47 E                     File "<frozen importlib._bootstrap>", line 671, in _load_unlocked
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/_pytest/assertion/rewrite.py", line 161, in exec_module
      Sep 02 01:34:47 E                       source_stat, co = _rewrite_test(fn, self.config)
      Sep 02 01:34:47 E                     File "/__w/2/s/flink-python/.tox/py38-cython/lib/python3.8/site-packages/_pytest/assertion/rewrite.py", line 351, in _rewrite_test
      Sep 02 01:34:47 E                       stat = os.stat(fn_)
      Sep 02 01:34:47 E                   FileNotFoundError: [Errno 2] No such file or directory: '/tmp/python-dist-266a4f9f-c350-41b8-b437-69b3c67435de/python-files/blob_p-0f11eb68b0611db5e1812d04788ca9c96d8e519c-4ad7d15b4215b93fd53b7ca4ab4bef4e/test_dependency_manage_lib.py'
      Sep 02 01:34:47 E                   
      Sep 02 01:34:47 E                   	at org.apache.beam.runners.fnexecution.control.FnApiControlClient$ResponseStreamObserver.onNext(FnApiControlClient.java:180)
      Sep 02 01:34:47 E                   	at org.apache.beam.runners.fnexecution.control.FnApiControlClient$ResponseStreamObserver.onNext(FnApiControlClient.java:160)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:251)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Contexts$ContextualizedServerCallListener.onMessage(Contexts.java:76)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:309)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:292)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:782)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
      Sep 02 01:34:47 E                   	at org.apache.beam.vendor.grpc.v1p26p0.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
      Sep 02 01:34:47 E                   	... 3 more
      

      Attachments

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            hxbks2ks Huang Xingbo
            xtsong Xintong Song
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment