You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "Christian Thiel (JIRA)" <ji...@apache.org> on 2018/12/17 16:16:00 UTC

[jira] [Created] (AIRFLOW-3534) KubernetesPodOperator breaks with active log-collection for long running tasks

Christian Thiel created AIRFLOW-3534:
----------------------------------------

             Summary: KubernetesPodOperator breaks with active log-collection for long running tasks
                 Key: AIRFLOW-3534
                 URL: https://issues.apache.org/jira/browse/AIRFLOW-3534
             Project: Apache Airflow
          Issue Type: Bug
          Components: kubernetes
    Affects Versions: 1.10.1
            Reporter: Christian Thiel


If a KubernetesPodOperator is started with get_logs=True, the pod breaks if no logs are produced after ~30 seconds due to http client timeout.

The error occurs in two ways:
1. If the script doesn't write anything to stdout there are three WARNINGS of the connectionpool trying to get the logs: 

{code:python}
// Some comments here
2018-12-17 15:23:15,092] {{logging_mixin.py:95}} WARNING - 2018-12-17 15:23:15,092 WARNING Retrying (Retry(total=2, connect=None, read=None, redirect=None, status=None)) after connection broken by 'ProtocolError('Connection aborted.', RemoteDisconnected('Remote end closed connection without response',))': /k8s/clusters/local/api/v1/namespaces/my-namespace/pods/my-pod/log?container=base&follow=True&tailLines=10
{code}
Followed by a {code:python}http.client.RemoteDisconnected: Remote end closed connection without response{code}
originating from _monitor_pod in /contrib/kubernetes/pod_launcher.py
Full Traceback:

{code:python}
Traceback (most recent call last):
  File "/opt/conda/lib/python3.6/site-packages/airflow/models.py", line 1659, in _run_raw_task
    result = task_copy.execute(context=context)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/operators/kubernetes_pod_operator.py", line 123, in execute
    get_logs=self.get_logs)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/kubernetes/pod_launcher.py", line 90, in run_pod
    return self._monitor_pod(pod, get_logs)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/kubernetes/pod_launcher.py", line 102, in _monitor_pod
    _preload_content=False)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/apis/core_v1_api.py", line 18583, in read_namespaced_pod_log
    (data) = self.read_namespaced_pod_log_with_http_info(name, namespace, **kwargs)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/apis/core_v1_api.py", line 18689, in read_namespaced_pod_log_with_http_info
    collection_formats=collection_formats)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/api_client.py", line 321, in call_api
    _return_http_data_only, collection_formats, _preload_content, _request_timeout)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/api_client.py", line 155, in __call_api
    _request_timeout=_request_timeout)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/api_client.py", line 342, in request
    headers=headers)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/rest.py", line 231, in GET
    query_params=query_params)
  File "/opt/conda/lib/python3.6/site-packages/kubernetes/client/rest.py", line 205, in request
    headers=headers)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/request.py", line 68, in request
    **urlopen_kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/request.py", line 89, in request_encode_url
    return self.urlopen(method, url, **extra_kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/poolmanager.py", line 322, in urlopen
    response = conn.urlopen(method, u.request_uri, **kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/connectionpool.py", line 667, in urlopen
    **response_kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/connectionpool.py", line 667, in urlopen
    **response_kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/connectionpool.py", line 667, in urlopen
    **response_kw)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/connectionpool.py", line 638, in urlopen
    _stacktrace=sys.exc_info()[2])
  File "/opt/conda/lib/python3.6/site-packages/urllib3/util/retry.py", line 398, in increment
    raise MaxRetryError(_pool, url, error or ResponseError(cause))
urllib3.exceptions.MaxRetryError: HTTPSConnectionPool(host='rancher.benteler.net', port=443): Max retries exceeded with url: /k8s/clusters/local/api/v1/namespaces/ou-seamless-airflow-ops/pods/sql-fmv-collector-s3-9074ac52/log?container=base&follow=True&tailLines=10 (Caused by ProtocolError('Connection aborted.', RemoteDisconnected('Remote end closed connection without response',)))

{code}

2. A similar issue occurs if the container started to print to stdout but then stopped for some time maybe due to a long running operation (SQL-query in my case). In this case the incremental read of the logs break with the following traceback:

{code:python}
Traceback (most recent call last):
  File "/opt/conda/lib/python3.6/http/client.py", line 546, in _get_chunk_left
    chunk_left = self._read_next_chunk_size()
  File "/opt/conda/lib/python3.6/http/client.py", line 513, in _read_next_chunk_size
    return int(line, 16)
ValueError: invalid literal for int() with base 16: b''

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/opt/conda/lib/python3.6/http/client.py", line 578, in _readinto_chunked
    chunk_left = self._get_chunk_left()
  File "/opt/conda/lib/python3.6/http/client.py", line 548, in _get_chunk_left
    raise IncompleteRead(b'')
http.client.IncompleteRead: IncompleteRead(0 bytes read)

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/opt/conda/lib/python3.6/site-packages/urllib3/response.py", line 331, in _error_catcher
    yield
  File "/opt/conda/lib/python3.6/site-packages/urllib3/response.py", line 413, in read
    data = self._fp.read(amt)
  File "/opt/conda/lib/python3.6/http/client.py", line 449, in read
    n = self.readinto(b)
  File "/opt/conda/lib/python3.6/http/client.py", line 483, in readinto
    return self._readinto_chunked(b)
  File "/opt/conda/lib/python3.6/http/client.py", line 594, in _readinto_chunked
    raise IncompleteRead(bytes(b[0:total_bytes]))
http.client.IncompleteRead: IncompleteRead(0 bytes read)

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/opt/conda/lib/python3.6/site-packages/airflow/models.py", line 1659, in _run_raw_task
    result = task_copy.execute(context=context)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/operators/kubernetes_pod_operator.py", line 123, in execute
    get_logs=self.get_logs)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/kubernetes/pod_launcher.py", line 90, in run_pod
    return self._monitor_pod(pod, get_logs)
  File "/opt/conda/lib/python3.6/site-packages/airflow/contrib/kubernetes/pod_launcher.py", line 103, in _monitor_pod
    for line in logs:
  File "/opt/conda/lib/python3.6/site-packages/urllib3/response.py", line 430, in read
    raise IncompleteRead(self._fp_bytes_read, self.length_remaining)
  File "/opt/conda/lib/python3.6/contextlib.py", line 99, in __exit__
    self.gen.throw(type, value, traceback)
  File "/opt/conda/lib/python3.6/site-packages/urllib3/response.py", line 349, in _error_catcher
    raise ProtocolError('Connection broken: %r' % e, e)
urllib3.exceptions.ProtocolError: ('Connection broken: IncompleteRead(0 bytes read)', IncompleteRead(0 bytes read))
{code}

Note the bit: {code:python} ValueError: invalid literal for int() with base 16: b''{code} indicating that the only problem is that there is no data.

I am not sure whether this bug is in airflow or the kubernetes library. Happy to open an issue there too. We should probably still set some kind of timeout in Airflow?

--------------------

In the meantime the issue can be solved by some more logging:
{code:python}
import threading


def _alive_log():
    while True:
        logger.info('Process is still alive.')
        time.sleep(5)


t = threading.Thread(name='alive-log', target=_alive_log)
t.setDaemon(True)
t.start()
{code}










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