[ 
https://issues.apache.org/jira/browse/BEAM-5633?focusedWorklogId=151879&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-151879
 ]

ASF GitHub Bot logged work on BEAM-5633:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Oct/18 21:38
            Start Date: 05/Oct/18 21:38
    Worklog Time Spent: 10m 
      Work Description: mwylde opened a new pull request #6593: [BEAM-5633] 
Adds reconnect logic to python logging client
URL: https://github.com/apache/beam/pull/6593
 
 
   This PR attempts to fix BEAM-5633 (Python SDK harness logging client 
failure) by adding reconnect logic to the logging client. With this patch, we 
haven't seen logging failures on our infrastructure.
   
   This issue manifests itself on the server (taskmanager) side with the 
exception
   
   ```
   [grpc-default-executor-0] WARN 
org.apache.beam.runners.fnexecution.logging.GrpcLoggingService - Logging client 
failed unexpectedly.
   org.apache.beam.vendor.grpc.v1.io.grpc.StatusRuntimeException: CANCELLED: 
cancelled before receiving half close
        at 
org.apache.beam.vendor.grpc.v1.io.grpc.Status.asRuntimeException(Status.java:517)
        at 
org.apache.beam.vendor.grpc.v1.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onCancel(ServerCalls.java:272)
   ```
   
   and on the client (python worker) side with
   
   ```
   E1004 22:08:02.139750634      38 chttp2_transport.cc:1097]   Received a 
GOAWAY with error code ENHANCE_YOUR_CALM and debug data equal to 
"too_many_pings"
   Exception in thread read_log_control_messages:
   Traceback (most recent call last):
     File "/usr/local/lib/python2.7/threading.py", line 801, in 
__bootstrap_inner
       self.run()
     File "/usr/local/lib/python2.7/threading.py", line 754, in run
       self.__target(*self.__args, **self.__kwargs)
     File 
"/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/log_handler.py",
 line 65, in <lambda>
       target=lambda: self._read_log_control_messages(log_control_messages),
     File 
"/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/log_handler.py",
 line 111, in _read_log_control_messages
       for _ in log_control_iterator:
     File "/usr/local/lib/python2.7/site-packages/grpc/_channel.py", line 350, 
in next
       return self._next()
     File "/usr/local/lib/python2.7/site-packages/grpc/_channel.py", line 341, 
in _next
       raise self
   _Rendezvous: <_Rendezvous of RPC that terminated with 
(StatusCode.UNAVAILABLE, Socket closed)>
   ```
   
   At a high-level, the issue is the client is sending the server too many 
illegal pings, and the server eventually closes the connection. As the client 
doesn't have any retry logic it never recovers from this.
   
   Ideally, there'd be two levels of fixes:
   
   1. Update the server or client gRPC configuration to prevent the client from 
sending bad pings or to prevent the server from killing the connection in this 
situation (see 
[here](https://github.com/grpc/grpc/blob/master/doc/keepalive.md) for more 
documentation of this behavior)
   2. Make the logging client more robust, reconnecting when the connection is 
lost
   
   I haven't been able to find a configuration that solves the first part, so 
that may need more investigation. Regardless, I think it's worthwhile having 
reconnection logic even if that issue is fixed.
   
   ------------------------
   
   Follow this checklist to help us incorporate your contribution quickly and 
easily:
   
    - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in 
ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA 
issue, if applicable. This will automatically link the pull request to the 
issue.
    - [ ] If this contribution is large, please file an Apache [Individual 
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   It will help us expedite review of your Pull Request if you tag someone 
(e.g. `@username`) to look at it.
   
   Post-Commit Tests Status (on master branch)
   
------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/)
 | --- | --- | --- | --- | --- | ---
   Java | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/)
   Python | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/)
 | --- | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)
 </br> [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/)
 | --- | --- | ---
   
   
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

            Worklog Id:     (was: 151879)
            Time Spent: 10m
    Remaining Estimate: 0h

> Python SDK harness logging client failure
> -----------------------------------------
>
>                 Key: BEAM-5633
>                 URL: https://issues.apache.org/jira/browse/BEAM-5633
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-harness
>            Reporter: Thomas Weise
>            Priority: Major
>              Labels: portability-flink
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> After running a test with synthetic source for a few minutes, the logging 
> client fails and all subsequent log output is not forwarded to the runner.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to