You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/07 22:16:23 UTC

[GitHub] [airflow] KIRY4 opened a new issue #10860: Timeouts in Airflow scheduler logs on AKS

KIRY4 opened a new issue #10860:
URL: https://github.com/apache/airflow/issues/10860


   **Apache Airflow version**:
   1.10.12
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):
   AKS 1.17.9
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**:
   Azure AKS
   - **OS** (e.g. from /etc/os-release):
   - **Kernel** (e.g. `uname -a`):
   - **Install tools**:
   - **Others**:
   I'm using AKS+KubernetesExecutor+KubernetesPodOperator+Helm stable Airflow chart
   
   **What happened**:
   
   **My log full of following errors:**
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 287, in run
       self.worker_uuid, self.kube_config)
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 323, in _run
       for event in list_worker_pods():
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 144, in stream
       for line in iter_resp_lines(resp):
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 46, in iter_resp_lines
       for seg in resp.read_chunked(decode_content=False):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 792, in read_chunked
       self._original_response.close()
     File "/usr/local/lib/python3.6/contextlib.py", line 99, in __exit__
       self.gen.throw(type, value, traceback)
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 441, in _error_catcher
       raise ReadTimeoutError(self._pool, None, "Read timed out.")
   urllib3.exceptions.ReadTimeoutError: HTTPSConnectionPool(host='10.158.0.1', port=443): Read timed out.
   [2020-09-10 13:39:23,281] {kubernetes_executor.py:302} INFO - Event: and now my watch begins starting at resource_version: 0
   
   **What you expected to happen**:
   
   Clean up log from this errors... I have suspicions that it could cause heartbeat problem witch scheduler, usually after 5-10 DAG's execution I'm getting error in UI: **The scheduler does not appear to be running. Last heartbeat was received X minutes ago.The DAGs list may not update, and new tasks will not be scheduled**. After that I can't schedule anything I'm not sure that error in logs and heartbeat problem are related but it could be...
   
   **How to reproduce it**:
   Schedule more or less long running DAG on following setup. AKS+KubernetesExecutor+KubernetesPodOperator+Helm stable Airflow chart.
   
   As minimally and precisely as possible. Keep in mind we do not have access to your cluster or dags.
   
   **Anything else we need to know**:
   
   One cool guy from Airflow Slack tell me that he knows this issue and he have solution: https://github.com/maganaluis/k8s-api-python in timeout-fix branch
   
   How often does this problem occur? Once? Every time etc?
   This error floods log every time...
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] arkadiuszbach commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
arkadiuszbach commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-791796674


   On Azure LoadBalancers have tcp idle timeout set as default to 4 minutes(it is visible in JSON View), so if you connect and don't interact with it in more than 4 minutes it will drop the connection.
   
   Airflow is using Kuberenetes watcher in order to monitor pod events and it is using it in stream mode.
   So It connects and waits for events, if there are no events in more than 4 minutes the LoadBalancer drops connection, but watcher is still listening for events and don't even know that connection was dropped
   
   When you add:
   ```
   -name: AIRFLOW__KUBERNETES__KUBE_CLIENT_REQUEST_ARGS
     value: '{"_request_timeout" : [60, 60]}'
   ```
   Then if the watcher did not get any events in more than 60 seconds the Read Timeout happens, but this time disconnect is one the client side(Airflow) - there is a `while(true)` in the code, so it will connect again and that is what you can see in the logs when it says "and now my watch begins"
   
   I tried the solution with _request_timeout, it works, but i didn't like these errors in the logs, so i looked and found following, which is pretty the same(it involves LoadBalancers) and describes it in more detail: https://www.finbourne.com/blog/the-mysterious-hanging-client-tcp-keep-alives
   
   So the solution is to add TCP keep alive, it will probe LoadBalancer and the idle timeout will not be triggered, even if for some reason LoadBalancer will disconnect,  keep alive will probe it and if it does not respond(for example 3 times over 60seconds) it will simply disconnect and connect again
   
   More information about keepalive:
   https://stackoverflow.com/questions/1480236/does-a-tcp-socket-connection-have-a-keep-alive
   
   With the help from solution above: https://github.com/maganaluis/k8s-api-python i was able to make it work.
   So i just downloaded the airflow version i had from pypi(Airflow 1.10.14) took the airflow file from `aiflow/bin` and after 
   `if __name__ == '__main__':` added:
   ```
       import socket
       from urllib3 import connection
       connection.HTTPConnection.default_socket_options += [
           (socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1),
           (socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 3)
       ]
   ``` 
   renamed this file to airflow_custom_start.py and added it in the AIRFLOW_HOME directory inside my Airflow Docker image, then in the entrypoint.sh i just started scheduler not by using:
    `airflow scheduler` command, 
   but:
    `python $AIRFLOW_HOME/airflow_custom_start.py`
   
   Also remember to remove request_timemout otherwise it will keep disconnecting


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kahidna commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
kahidna commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-735717893


   any update for this issue? I'm facing same with my airflow scheduler
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] arkadiuszbach commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
arkadiuszbach commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-886515641


   This is related to Azure LoadBalancers which has TCP_IDLE_TIMEOUT equal to 10minutes by default, which means that if your scheduler is not doing anything on kubernetes api for more than 10minutes connection will be killed.
   
   - In Airflow 2.x there is option:
     ```
       # Enables TCP keepalive mechanism. This prevents Kubernetes API requests to hang indefinitely
       # when idle connection is time-outed on services like cloud load balancers or firewalls.
       enable_tcp_keepalive = True
     ```
     for enabling TCP_KEEP_ALIVE in python urllib3 library, which is used by requests and kubernetes library, by default TCP_KEEP_ALIVE is disabled
   
   - In airflow 1.10.x you can copy airflow start script content to some other python script for example airflow_start_custom.py and add TCP_KEEP_ALIVE before `parser = CLIFactory.get_parser()` line:
     ```
       import socket
       from urllib3 import connection
       connection.HTTPConnection.default_socket_options += [
           (socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1),
           (socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 3)
       ]
     ```
     Then just start airflow scheduler not from airflow.py file but from airflow_start_custom.py
   
   
   PS: both of the above solutions will not work if you are using istio sidecar proxies, if that is the case you will need to enable TCP_KEEP_ALIVE on istio sidecars


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] gvazharov commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
gvazharov commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-701226788


   Hi I am experiencing the same problem and don't know how to fix it, is there any kind of update?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] vigneshs2893 commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
vigneshs2893 commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-832553978


   Any update on this issue? I am also facing it with my airflow scheduler.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] KIRY4 commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
KIRY4 commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-701314516


   > Hi I am experiencing the same problem and don't know how to fix it, is there any kind of update?
   
   Yeah I'm still experiencing it too. Also I'm struggling from following message in UI maybe it is related: `The scheduler does not appear to be running. Last heartbeat was received 11 minutes ago.The DAGs list may not update, and new tasks will not be scheduled` . One good man from Airflow channel proposed me following solution:  `on AKS you will have to fork Airflow and patch Kubernetes executor. https://github.com/maganaluis/k8s-api-python check the time out branch. You can open an issue for this on the Airflow repository, those changes are harmless for other platforms.` He told that this patch helped him with this timeout and looks like this issue exist only in AKS (not sure about that). But I'm not developer so I'm not sure what exactly should I patch and how. Maybe anyone from Airflow contributors can look into it?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] KIRY4 commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
KIRY4 commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-692101574


   Here is full logs from scheduler pod after restarting of scheduler POD. DAG's wasn't scheduled so it means that this errors producing whole time and not related to DAG's:
   ```
   *** running scheduler...
     ____________       _____________
    ____    |__( )_________  __/__  /________      __
   ____  /| |_  /__  ___/_  /_ __  /_  __ \_ | /| / /
   ___  ___ |  / _  /   _  __/ _  / / /_/ /_ |/ |/ /
    _/_/  |_/_/  /_/    /_/    /_/  \____/____/|__/
   /home/airflow/.local/lib/python3.6/site-packages/airflow/kubernetes/pod_generator.py:39: DeprecationWarning: This module is deprecated. Please use `airflow.kubernetes.pod`.
     from airflow.contrib.kubernetes.pod import _extract_volume_mounts
   [2020-09-14 14:30:02,552] {__init__.py:50} INFO - Using executor KubernetesExecutor
   [2020-09-14 14:30:02,571] {scheduler_job.py:1367} INFO - Starting the scheduler
   [2020-09-14 14:30:02,572] {scheduler_job.py:1375} INFO - Running execute loop for -1 seconds
   [2020-09-14 14:30:02,573] {scheduler_job.py:1376} INFO - Processing each file at most -1 times
   [2020-09-14 14:30:02,573] {scheduler_job.py:1379} INFO - Searching for files in /opt/airflow/dags
   [2020-09-14 14:30:02,905] {scheduler_job.py:1381} INFO - There are 4 files in /opt/airflow/dags
   [2020-09-14 14:30:02,905] {kubernetes_executor.py:770} INFO - Start Kubernetes executor
   [2020-09-14 14:30:02,958] {kubernetes_executor.py:302} INFO - Event: and now my watch begins starting at resource_version: 0
   [2020-09-14 14:30:02,965] {kubernetes_executor.py:698} INFO - When executor started up, found 0 queued task instances
   [2020-09-14 14:30:02,967] {scheduler_job.py:1438} INFO - Resetting orphaned tasks for active dag runs
   [2020-09-14 14:30:02,980] {dag_processing.py:562} INFO - Launched DagFileProcessorManager with pid: 81
   [2020-09-14 14:30:03,091] {settings.py:55} INFO - Configured default timezone <Timezone [UTC]>
   [2020-09-14 14:31:03,046] {kubernetes_executor.py:290} WARNING - There was a timeout error accessing the Kube API. Retrying request.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 436, in _error_catcher
       yield
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 763, in read_chunked
       self._update_chunk_length()
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 693, in _update_chunk_length
       line = self._fp.fp.readline()
     File "/usr/local/lib/python3.6/socket.py", line 586, in readinto
       return self._sock.recv_into(b)
     File "/usr/local/lib/python3.6/ssl.py", line 1012, in recv_into
       return self.read(nbytes, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 874, in read
       return self._sslobj.read(len, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 631, in read
       v = self._sslobj.read(len, buffer)
   socket.timeout: The read operation timed out
   
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 287, in run
       self.worker_uuid, self.kube_config)
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 323, in _run
       for event in list_worker_pods():
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 144, in stream
       for line in iter_resp_lines(resp):
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 46, in iter_resp_lines
       for seg in resp.read_chunked(decode_content=False):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 792, in read_chunked
       self._original_response.close()
     File "/usr/local/lib/python3.6/contextlib.py", line 99, in __exit__
       self.gen.throw(type, value, traceback)
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 441, in _error_catcher
       raise ReadTimeoutError(self._pool, None, "Read timed out.")
   urllib3.exceptions.ReadTimeoutError: HTTPSConnectionPool(host='10.158.0.1', port=443): Read timed out.
   [2020-09-14 14:31:04,050] {kubernetes_executor.py:302} INFO - Event: and now my watch begins starting at resource_version: 0
   [2020-09-14 14:32:04,132] {kubernetes_executor.py:290} WARNING - There was a timeout error accessing the Kube API. Retrying request.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 436, in _error_catcher
       yield
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 763, in read_chunked
       self._update_chunk_length()
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 693, in _update_chunk_length
       line = self._fp.fp.readline()
     File "/usr/local/lib/python3.6/socket.py", line 586, in readinto
       return self._sock.recv_into(b)
     File "/usr/local/lib/python3.6/ssl.py", line 1012, in recv_into
       return self.read(nbytes, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 874, in read
       return self._sslobj.read(len, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 631, in read
       v = self._sslobj.read(len, buffer)
   socket.timeout: The read operation timed out
   
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 287, in run
       self.worker_uuid, self.kube_config)
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 323, in _run
       for event in list_worker_pods():
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 144, in stream
       for line in iter_resp_lines(resp):
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 46, in iter_resp_lines
       for seg in resp.read_chunked(decode_content=False):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 792, in read_chunked
       self._original_response.close()
     File "/usr/local/lib/python3.6/contextlib.py", line 99, in __exit__
       self.gen.throw(type, value, traceback)
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 441, in _error_catcher
       raise ReadTimeoutError(self._pool, None, "Read timed out.")
   urllib3.exceptions.ReadTimeoutError: HTTPSConnectionPool(host='10.158.0.1', port=443): Read timed out.
   [2020-09-14 14:32:05,135] {kubernetes_executor.py:302} INFO - Event: and now my watch begins starting at resource_version: 0
   [2020-09-14 14:33:05,217] {kubernetes_executor.py:290} WARNING - There was a timeout error accessing the Kube API. Retrying request.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 436, in _error_catcher
       yield
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 763, in read_chunked
       self._update_chunk_length()
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 693, in _update_chunk_length
       line = self._fp.fp.readline()
     File "/usr/local/lib/python3.6/socket.py", line 586, in readinto
       return self._sock.recv_into(b)
     File "/usr/local/lib/python3.6/ssl.py", line 1012, in recv_into
       return self.read(nbytes, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 874, in read
       return self._sslobj.read(len, buffer)
     File "/usr/local/lib/python3.6/ssl.py", line 631, in read
       v = self._sslobj.read(len, buffer)
   socket.timeout: The read operation timed out
   
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 287, in run
       self.worker_uuid, self.kube_config)
     File "/home/airflow/.local/lib/python3.6/site-packages/airflow/executors/kubernetes_executor.py", line 323, in _run
       for event in list_worker_pods():
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 144, in stream
       for line in iter_resp_lines(resp):
     File "/home/airflow/.local/lib/python3.6/site-packages/kubernetes/watch/watch.py", line 46, in iter_resp_lines
       for seg in resp.read_chunked(decode_content=False):
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 792, in read_chunked
       self._original_response.close()
     File "/usr/local/lib/python3.6/contextlib.py", line 99, in __exit__
       self.gen.throw(type, value, traceback)
     File "/home/airflow/.local/lib/python3.6/site-packages/urllib3/response.py", line 441, in _error_catcher
       raise ReadTimeoutError(self._pool, None, "Read timed out.")
   urllib3.exceptions.ReadTimeoutError: HTTPSConnectionPool(host='10.158.0.1', port=443): Read timed out.
   [2020-09-14 14:33:06,218] {kubernetes_executor.py:302} INFO - Event: and now my watch begins starting at resource_version: 0
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-886974397


   Great comment ! Thanks for summarising it @arkadiuszbach !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk closed issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #10860:
URL: https://github.com/apache/airflow/issues/10860


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] KIRY4 closed issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
KIRY4 closed issue #10860:
URL: https://github.com/apache/airflow/issues/10860


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ardhar commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
ardhar commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-871413814


   Any solution to this ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-886234155


   I am afraid, this looks like a problem with deployment, not Airflow. What I can recommend is to install Airflow using the Official Helm Chart  https://airflow.apache.org/docs/helm-chart/stable/index.html. 
   
   Then we will be able to help with diagnosing the issues. I am afraid this is is the best we can do. You can also try to open an issue in the "stable" helm chart repo if you continue getting the same problem. 
   
   I am afraid this is the best we can do now,


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] arkadiuszbach edited a comment on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
arkadiuszbach edited a comment on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-791796674


   On Azure LoadBalancers have tcp idle timeout set as default to 4 minutes(it is visible in JSON View), so if you connect and don't interact with it in more than 4 minutes it will drop the connection.
   
   Airflow is using Kuberenetes watcher in order to monitor pod events and it is using it in stream mode.
   So It connects and waits for events, if there are no events in more than 4 minutes the LoadBalancer drops connection, but watcher is still listening for events and don't even know that connection was dropped
   
   When you add:
   ```
   -name: AIRFLOW__KUBERNETES__KUBE_CLIENT_REQUEST_ARGS
     value: '{"_request_timeout" : [60, 60]}'
   ```
   Then if the watcher did not get any events in more than 60 seconds the Read Timeout happens, but this time disconnect is one the client side(Airflow) - there is a `while(true)` in the code, so it will connect again and that is what you can see in the logs when it says "and now my watch begins"
   
   I tried the solution with _request_timeout, it works, but i didn't like these errors in the logs, so i looked and found following, which is pretty the same(it involves LoadBalancers) and describes it in more detail: https://www.finbourne.com/blog/the-mysterious-hanging-client-tcp-keep-alives
   
   So the solution is to add TCP keep alive, it will probe LoadBalancer and the idle timeout will not be triggered, even if for some reason LoadBalancer will disconnect,  keep alive will probe it and if it does not respond(for example 3 times over 60seconds) it will simply disconnect and connect again
   
   More information about keepalive:
   https://stackoverflow.com/questions/1480236/does-a-tcp-socket-connection-have-a-keep-alive
   
   With the help from solution above: https://github.com/maganaluis/k8s-api-python i was able to make it work.
   So i just downloaded the airflow version i had from pypi(Airflow 1.10.14) took the airflow file from `aiflow/bin` and after 
   `if __name__ == '__main__':` added:
   ```
       import socket
       from urllib3 import connection
       connection.HTTPConnection.default_socket_options += [
           (socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1),
           (socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 3)
       ]
   ``` 
   renamed this file to airflow_custom_start.py and added it in the AIRFLOW_HOME directory inside my Airflow Docker image, then in the entrypoint.sh i just started scheduler not by using:
    `airflow scheduler` command, 
   but:
    `python $AIRFLOW_HOME/airflow_custom_start.py scheduler`
   
   Also remember to remove request_timemout otherwise it will keep disconnecting


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] arkadiuszbach removed a comment on issue #10860: Timeouts in Airflow scheduler logs on AKS

Posted by GitBox <gi...@apache.org>.
arkadiuszbach removed a comment on issue #10860:
URL: https://github.com/apache/airflow/issues/10860#issuecomment-791796674


   On Azure LoadBalancers have tcp idle timeout set as default to 4 minutes(it is visible in JSON View), so if you connect and don't interact with it in more than 4 minutes it will drop the connection.
   
   Airflow is using Kuberenetes watcher in order to monitor pod events and it is using it in stream mode.
   So It connects and waits for events, if there are no events in more than 4 minutes the LoadBalancer drops connection, but watcher is still listening for events and don't even know that connection was dropped
   
   When you add:
   ```
   -name: AIRFLOW__KUBERNETES__KUBE_CLIENT_REQUEST_ARGS
     value: '{"_request_timeout" : [60, 60]}'
   ```
   Then if the watcher did not get any events in more than 60 seconds the Read Timeout happens, but this time disconnect is one the client side(Airflow) - there is a `while(true)` in the code, so it will connect again and that is what you can see in the logs when it says "and now my watch begins"
   
   I tried the solution with _request_timeout, it works, but i didn't like these errors in the logs, so i looked and found following, which is pretty the same(it involves LoadBalancers) and describes it in more detail: https://www.finbourne.com/blog/the-mysterious-hanging-client-tcp-keep-alives
   
   So the solution is to add TCP keep alive, it will probe LoadBalancer and the idle timeout will not be triggered, even if for some reason LoadBalancer will disconnect,  keep alive will probe it and if it does not respond(for example 3 times over 60seconds) it will simply disconnect and connect again
   
   More information about keepalive:
   https://stackoverflow.com/questions/1480236/does-a-tcp-socket-connection-have-a-keep-alive
   
   With the help from solution above: https://github.com/maganaluis/k8s-api-python i was able to make it work.
   So i just downloaded the airflow version i had from pypi(Airflow 1.10.14) took the airflow file from `aiflow/bin` and after 
   `if __name__ == '__main__':` added:
   ```
       import socket
       from urllib3 import connection
       connection.HTTPConnection.default_socket_options += [
           (socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1),
           (socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 60),
           (socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 3)
       ]
   ``` 
   renamed this file to airflow_custom_start.py and added it in the AIRFLOW_HOME directory inside my Airflow Docker image, then in the entrypoint.sh i just started scheduler not by using:
    `airflow scheduler` command, 
   but:
    `python $AIRFLOW_HOME/airflow_custom_start.py scheduler`
   
   Also remember to remove request_timemout otherwise it will keep disconnecting


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org