Skip to content

Conversation

@dstandish
Copy link
Contributor

There are many overlapping layers and strategies of retrying in this area of code. It appears this particular layer may be unnecessary. See discussion starting at #31622 (comment).

There are many overlapping layers and strategies of retrying in this area of code.  It appears this particular layer may be unnecessary.  See discussion starting at apache#31622 (comment).
@boring-cyborg boring-cyborg bot added area:providers provider:cncf-kubernetes Kubernetes (k8s) provider related issues labels Nov 7, 2023
@dstandish dstandish changed the title Remove tenancity on KPO logs inner func consume_logs Remove tenacity on KPO logs inner func consume_logs Nov 7, 2023
@dstandish dstandish merged commit d6c79ce into apache:main Nov 7, 2023
@dstandish dstandish deleted the remove-tenacity-wrapper-on-consume-logs branch November 7, 2023 18:41
romsharon98 pushed a commit to romsharon98/airflow that referenced this pull request Nov 10, 2023
There are many overlapping layers and strategies of retrying in this area of code.  It appears this particular layer may be unnecessary.  See discussion starting at apache#31622 (comment).
@o-nikolas
Copy link
Contributor

Unfortunately we're seeing periodic failures in our fargate system tests (see the original PR for more explanation here) after this retry was removed. @dstandish are you very opposed to my adding it back in?

CC @syedahsn @vincbeck

@dstandish
Copy link
Contributor Author

Could you share the traceback?

@o-nikolas
Copy link
Contributor

Could you share the traceback?

Sorry this took so long @dstandish! You can see the traceback that we see in the tests (for which Vincent's retry resolved and are now back). There is also some other diagnostic info from the logs included:

Traceback (most recent call last):
  File "/opt/airflow/airflow/providers/cncf/kubernetes/operators/pod.py", line 554, in execute_sync
    self.pod_manager.fetch_requested_container_logs(
  File "/opt/airflow/airflow/providers/cncf/kubernetes/utils/pod_manager.py", line 557, in fetch_requested_container_logs
    status = self.fetch_container_logs(pod=pod, container_name=c, follow=follow_logs)
  File "/opt/airflow/airflow/providers/cncf/kubernetes/utils/pod_manager.py", line 480, in fetch_container_logs
    last_log_time, exc = consume_logs(since_time=last_log_time)
  File "/opt/airflow/airflow/providers/cncf/kubernetes/utils/pod_manager.py", line 420, in consume_logs
    logs = self.read_pod_logs(
  File "/usr/local/lib/python3.8/site-packages/tenacity/__init__.py", line 289, in wrapped_f
    return self(f, *args, **kw)
  File "/usr/local/lib/python3.8/site-packages/tenacity/__init__.py", line 379, in __call__
    do = self.iter(retry_state=retry_state)
  File "/usr/local/lib/python3.8/site-packages/tenacity/__init__.py", line 325, in iter
    raise retry_exc.reraise()
  File "/usr/local/lib/python3.8/site-packages/tenacity/__init__.py", line 158, in reraise
    raise self.last_attempt.result()
  File "/usr/local/lib/python3.8/concurrent/futures/_base.py", line 437, in result
    return self.__get_result()
  File "/usr/local/lib/python3.8/concurrent/futures/_base.py", line 389, in __get_result
    raise self._exception
  File "/usr/local/lib/python3.8/site-packages/tenacity/__init__.py", line 382, in __call__
    result = fn(*args, **kwargs)
  File "/opt/airflow/airflow/providers/cncf/kubernetes/utils/pod_manager.py", line 645, in read_pod_logs
    logs = self._client.read_namespaced_pod_log(
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 23747, in read_namespaced_pod_log
    return self.read_namespaced_pod_log_with_http_info(name, namespace, **kwargs)  # noqa: E501
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/api/core_v1_api.py", line 23866, in read_namespaced_pod_log_with_http_info
    return self.api_client.call_api(
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 348, in call_api
    return self.__call_api(resource_path, method,
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 180, in __call_api
    response_data = self.request(
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/api_client.py", line 373, in request
    return self.rest_client.GET(url,
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 240, in GET
    return self.request("GET", url,
  File "/usr/local/lib/python3.8/site-packages/kubernetes/client/rest.py", line 234, in request
    raise ApiException(http_resp=r)
kubernetes.client.exceptions.ApiException: (500)
Reason: Internal Server Error
HTTP response headers: HTTPHeaderDict({'Audit-Id': '19856b72-9dba-40c5-ade3-fa2ea690d6f2', 'Cache-Control': 'no-cache, private', 'Content-Type': 'application/json', 'Date': 'Sun, 07 Jan 2024 17:12:17 GMT', 'Content-Length': '237'})
HTTP response body: b'{"kind":"Status","apiVersion":"v1","metadata":{},"status":"Failure","message":"Get \\"https://10.0.4.155:10250/containerLogs/default/run-pod-425a4bz9/base?follow=true\\u0026timestamps=true\\": remote error: tls: internal error","code":500}\n'
During handling of the above exception, another exception occurred:
Traceback (most recent call last):
  File "/opt/airflow/airflow/executors/debug_executor.py", line 86, in _run_task
    ti.run(job_id=ti.job_id, **params)
  File "/opt/airflow/airflow/utils/session.py", line 79, in wrapper
    return func(*args, session=session, **kwargs)
  File "/opt/airflow/airflow/models/taskinstance.py", line 2601, in run
    self._run_raw_task(
  File "/opt/airflow/airflow/utils/session.py", line 76, in wrapper
    return func(*args, **kwargs)
  File "/opt/airflow/airflow/models/taskinstance.py", line 2336, in _run_raw_task
    self._execute_task_with_callbacks(context, test_mode, session=session)
  File "/opt/airflow/airflow/models/taskinstance.py", line 2503, in _execute_task_with_callbacks
    result = self._execute_task(context, task_orig)
  File "/opt/airflow/airflow/models/taskinstance.py", line 2520, in _execute_task
    return _execute_task(self, context, task_orig)
  File "/opt/airflow/airflow/models/taskinstance.py", line 435, in _execute_task
    result = execute_callable(context=context, **execute_callable_kwargs)
  File "/opt/airflow/airflow/providers/amazon/aws/operators/eks.py", line 1089, in execute
    return super().execute(context)
  File "/opt/airflow/airflow/providers/cncf/kubernetes/operators/pod.py", line 534, in execute
    return self.execute_sync(context)
  File "/opt/airflow/airflow/providers/cncf/kubernetes/operators/pod.py", line 574, in execute_sync
    self.cleanup(
  File "/opt/airflow/airflow/providers/cncf/kubernetes/operators/pod.py", line 715, in cleanup
    raise AirflowException(
airflow.exceptions.AirflowException: Pod run-pod-425a4bz9 returned a failure.
remote_pod: {'api_version': None,
 'kind': None,
 'metadata': {'annotations': None,
              'cluster_name': None,
              'creation_timestamp': datetime.datetime(2024, 1, 7, 17, 11, 7, tzinfo=tzlocal()),
              'deletion_grace_period_seconds': None,
              'deletion_timestamp': None,
              'finalizers': None,
              'generate_name': None,
              'generation': None,
              'labels': {'airflow_kpo_in_cluster': 'False',
                         'airflow_version': '2.9.0.dev0',
                         'dag_id': 'example_eks_with_fargate_profile',
                         'demo': 'hello_world',
                         'eks.amazonaws.com/fargate-profile': 'env9f73f189-profile',
                         'kubernetes_pod_operator': 'True',
                         'run_id': 'backfill__2021-01-01T0000000000-493b0d6b0',
                         'task_id': 'run_pod',
                         'try_number': '1'},
              'managed_fields': [{'api_version': 'v1',
                                  'fields_type': 'FieldsV1',
                                  'fields_v1': {'f:metadata': {'f:labels': {'.': {},
                                                                            'f:airflow_kpo_in_cluster': 
{}
,
                                                                            'f:airflow_version': 
{}
,
                                                                            'f:dag_id': 
{}
,
                                                                            'f:demo': 
{}
,
                                                                            'f:kubernetes_pod_operator': 
{}
,
                                                                            'f:run_id': 
{}
,
                                                                            'f:task_id': 
{}
,
                                                                            'f:try_number': 
{}
}},
                                                'f:spec': {'f:affinity': {},
                                                           'f:containers': {'k:{"name":"base"}': {'.': {},
                                                                                                  'f:command': 
{}
,
                                                                                                  'f:image': 
{}
,
                                                                                                  'f:imagePullPolicy': 
{}
,
                                                                                                  'f:name': 
{}
,
                                                                                                  'f:resources': 
{}
,
                                                                                                  'f:terminationMessagePath': 
{}
,
                                                                                                  'f:terminationMessagePolicy': 
{}
}},
                                                           'f:dnsPolicy': 
{}
,
                                                           'f:enableServiceLinks': 
{}
,
                                                           'f:restartPolicy': 
{}
,
                                                           'f:schedulerName': 
{}
,
                                                           'f:securityContext': 
{}
,
                                                           'f:terminationGracePeriodSeconds': 
{}
}},
                                  'manager': 'OpenAPI-Generator',
                                  'operation': 'Update',
                                  'subresource': None,
                                  'time': datetime.datetime(2024, 1, 7, 17, 11, 7, tzinfo=tzlocal())}],
              'name': 'run-pod-425a4bz9',
              'namespace': 'default',
              'owner_references': None,
              'resource_version': '1170',
              'self_link': None,
              'uid': '4dec4471-6702-4673-8f7e-a05b1046e381'},
 'spec': {'active_deadline_seconds': None,
          'affinity': {'node_affinity': None,
                       'pod_affinity': None,
                       'pod_anti_affinity': None},
          'automount_service_account_token': None,
          'containers': [{'args': None,
                          'command': ['sh', '-c', 'echo Test Airflow; date'],
                          'env': None,
                          'env_from': None,
                          'image': 'amazon/aws-cli:latest',
                          'image_pull_policy': 'Always',
                          'lifecycle': None,
                          'liveness_probe': None,
                          'name': 'base',
                          'ports': None,
                          'readiness_probe': None,
                          'resources': {'limits': None, 'requests': None},
                          'security_context': None,
                          'startup_probe': None,
                          'stdin': None,
                          'stdin_once': None,
                          'termination_message_path': '/dev/termination-log',
                          'termination_message_policy': 'File',
                          'tty': None,
                          'volume_devices': None,
                          'volume_mounts': [{'mount_path': '/var/run/secrets/kubernetes.io/serviceaccount',
                                             'mount_propagation': None,
                                             'name': 'kube-api-access-bqqpg',
                                             'read_only': True,
                                             'sub_path': None,
                                             'sub_path_expr': None}],
                          'working_dir': None}],
          'dns_config': None,
          'dns_policy': 'ClusterFirst',
          'enable_service_links': True,
          'ephemeral_containers': None,
          'host_aliases': None,
          'host_ipc': None,
          'host_network': None,
          'host_pid': None,
          'hostname': None,
          'image_pull_secrets': None,
          'init_containers': None,
          'node_name': None,
          'node_selector': None,
          'os': None,
          'overhead': None,
          'preemption_policy': 'PreemptLowerPriority',
          'priority': 2000001000,
          'priority_class_name': 'system-node-critical',
          'readiness_gates': None,
          'restart_policy': 'Never',
          'runtime_class_name': None,
          'scheduler_name': 'fargate-scheduler',
          'security_context': {'fs_group': None,
                               'fs_group_change_policy': None,
                               'run_as_group': None,
                               'run_as_non_root': None,
                               'run_as_user': None,
                               'se_linux_options': None,
                               'seccomp_profile': None,
                               'supplemental_groups': None,
                               'sysctls': None,
                               'windows_options': None},
          'service_account': 'default',
          'service_account_name': 'default',
          'set_hostname_as_fqdn': None,
          'share_process_namespace': None,
          'subdomain': None,
          'termination_grace_period_seconds': 30,
          'tolerations': [{'effect': 'NoExecute',
                           'key': 'node.kubernetes.io/not-ready',
                           'operator': 'Exists',
                           'toleration_seconds': 300,
                           'value': None},
                          {'effect': 'NoExecute',
                           'key': 'node.kubernetes.io/unreachable',
                           'operator': 'Exists',
                           'toleration_seconds': 300,
                           'value': None}],
          'topology_spread_constraints': None,
          'volumes': [{'aws_elastic_block_store': None,
                       'azure_disk': None,
                       'azure_file': None,
                       'cephfs': None,
                       'cinder': None,
                       'config_map': None,
                       'csi': None,
                       'downward_api': None,
                       'empty_dir': None,
                       'ephemeral': None,
                       'fc': None,
                       'flex_volume': None,
                       'flocker': None,
                       'gce_persistent_disk': None,
                       'git_repo': None,
                       'glusterfs': None,
                       'host_path': None,
                       'iscsi': None,
                       'name': 'kube-api-access-bqqpg',
                       'nfs': None,
                       'persistent_volume_claim': None,
                       'photon_persistent_disk': None,
                       'portworx_volume': None,
                       'projected': {'default_mode': 420,
                                     'sources': [{'config_map': None,
                                                  'downward_api': None,
                                                  'secret': None,
                                                  'service_account_token': {'audience': None,
                                                                            'expiration_seconds': 3607,
                                                                            'path': 'token'}},
                                                 {'config_map': {'items': [{'key': 'ca.crt',
                                                                            'mode': None,
                                                                            'path': 'ca.crt'}],
                                                                 'name': 'kube-root-ca.crt',
                                                                 'optional': None},
                                                  'downward_api': None,
                                                  'secret': None,
                                                  'service_account_token': None},
                                                 {'config_map': None,
                                                  'downward_api': {'items': [{'field_ref': {'api_version': 'v1',
                                                                                            'field_path': 'metadata.namespace'},
                                                                              'mode': None,
                                                                              'path': 'namespace',
                                                                              'resource_field_ref': None}]},
                                                  'secret': None,
                                                  'service_account_token': None}]},
                       'quobyte': None,
                       'rbd': None,
                       'scale_io': None,
                       'secret': None,
                       'storageos': None,
                       'vsphere_volume': None}]},
 'status': {'conditions': None,
            'container_statuses': None,
            'ephemeral_container_statuses': None,
            'host_ip': None,
            'init_container_statuses': None,
            'message': None,
            'nominated_node_name': None,
            'phase': 'Pending',
            'pod_i_ps': None,
            'pod_ip': None,
            'qos_class': 'BestEffort',
            'reason': None,
            'start_time': None}}
ERROR [airflow.jobs.backfill_job_runner.BackfillJobRunner] Task instance <TaskInstance: example_eks_with_fargate_profile.run_pod backfill__2021-01-01T00:00:00+00:00 [failed]> failed
INFO  [airflow.jobs.backfill_job_runner.BackfillJobRunner] [backfill progress] | finished run 0 of 1 | tasks waiting: 6 | succeeded: 5 | running: 0 | failed: 1 | skipped: 0 | deadlocked: 0 | not ready: 6
INFO  [airflow.executors.debug_executor.DebugExecutor] Adding to queue: ['<TaskInstance: example_eks_with_fargate_profile.describe_pod backfill__2021-01-01T00:00:00+00:00 [queued]>']
INFO  [airflow.executors.debug_executor.DebugExecutor] Adding to queue: ['<TaskInstance: example_eks_with_fargate_profile.watcher backfill__2021-01-01T00:00:00+00:00 [queued]>']
INFO  [airflow.task] Dependencies all met for dep_context=non-requeueable deps ti=<TaskInstance: example_eks_with_fargate_profile.describe_pod backfill__2021-01-01T00:00:00+00:00 [queued]>
INFO  [airflow.task] Dependencies all met for dep_context=requeueable deps ti=<TaskInstance: example_eks_with_fargate_profile.describe_pod backfill__2021-01-01T00:00:00+00:00 [queued]>
INFO  [airflow.models.taskinstance.TaskInstance] Starting attempt 1 of 2
INFO  [airflow.models.taskinstance.TaskInstance] Executing <Task(BashOperator): describe_pod> on 2021-01-01 00:00:00+00:00
INFO  [airflow.task] Exporting env vars: AIRFLOW_CTX_DAG_OWNER='airflow' AIRFLOW_CTX_DAG_ID='example_eks_with_fargate_profile' AIRFLOW_CTX_TASK_ID='describe_pod' AIRFLOW_CTX_EXECUTION_DATE='2021-01-01T00:00:00+00:00' AIRFLOW_CTX_TRY_NUMBER='1' AIRFLOW_CTX_DAG_RUN_ID='backfill__2021-01-01T00:00:00+00:00'
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Tmp dir root location: /tmp
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Running command: ['/usr/bin/bash', '-c', '\n                install_aws.sh;\n                install_kubectl.sh;\n                # configure kubectl to hit the right cluster\n                aws eks update-kubeconfig --name env9f73f189-cluster;\n                # once all this setup is done, actually describe the pod\n                echo "vvv pod description below vvv";\n                kubectl describe pod run-pod-425a4bz9;\n                echo "^^^ pod description above ^^^" ']
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Output:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] /usr/local/bin/aws
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] The "aws" command found. Installation not needed. Run with --reinstall to reinstall
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Run with --reinstall to reinstall.
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Downloading from https://storage.googleapis.com/kubernetes-release/release/v1.29.0/bin/linux/amd64/kubectl
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] 
                                                                           0.1%
##################                                                        25.8%
##########################################                                58.8%
####################################################################      95.6%
######################################################################## 100.0%
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Installation complete.
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Added new context arn:aws:eks:us-west-2:558120655471:cluster/env9f73f189-cluster to /files/.kube/config
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] vvv pod description below vvv
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Name:                 run-pod-425a4bz9
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Namespace:            default
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Priority:             2000001000
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Priority Class Name:  system-node-critical
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Service Account:      default
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Node:                 fargate-ip-10-0-4-155.us-west-2.compute.internal/10.0.4.155
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Start Time:           Sun, 07 Jan 2024 17:11:57 +0000
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Labels:               airflow_kpo_in_cluster=False
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       airflow_version=2.9.0.dev0
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       already_checked=True
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       dag_id=example_eks_with_fargate_profile
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       demo=hello_world
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       eks.amazonaws.com/fargate-profile=env9f73f189-profile
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       kubernetes_pod_operator=True
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       run_id=backfill__2021-01-01T0000000000-493b0d6b0
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       task_id=run_pod
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       try_number=1
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Annotations:          CapacityProvisioned: 0.25vCPU 0.5GB
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                       Logging: LoggingDisabled: LOGGING_CONFIGMAP_NOT_FOUND
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Status:               Succeeded
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] IP:                   10.0.4.155
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] IPs:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   IP:  10.0.4.155
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Containers:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   base:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Container ID:  containerd://91e5ac18cc44dc6e8743d8ba8536138d4ae3f6bd05dd551a8169dc20a52d2941
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Image:         amazon/aws-cli:latest
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Image ID:      docker.io/amazon/aws-cli@sha256:e84594337c171be7aebc4992d1963dccb6c4bac51c7c32cfbbdac9f64842593b
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Port:          <none>
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Host Port:     <none>
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Command:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       sh
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       -c
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       echo Test Airflow; date
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     State:          Terminated
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       Reason:       Completed
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       Exit Code:    0
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       Started:      Sun, 07 Jan 2024 17:12:12 +0000
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       Finished:     Sun, 07 Jan 2024 17:12:12 +0000
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Ready:          False
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Restart Count:  0
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Environment:    <none>
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Mounts:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]       /var/run/secrets/kubernetes.io/serviceaccount from kube-api-access-bqqpg (ro)
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Conditions:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Type              Status
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Initialized       True
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Ready             False
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   ContainersReady   False
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   PodScheduled      True
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Volumes:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   kube-api-access-bqqpg:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     Type:                    Projected (a volume that contains injected data from multiple sources)
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     TokenExpirationSeconds:  3607
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     ConfigMapName:           kube-root-ca.crt
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     ConfigMapOptional:       <nil>
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]     DownwardAPI:             true
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] QoS Class:                   BestEffort
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Node-Selectors:              <none>
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Tolerations:                 node.kubernetes.io/not-ready:NoExecute op=Exists for 300s
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]                              node.kubernetes.io/unreachable:NoExecute op=Exists for 300s
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] Events:
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Type     Reason           Age   From               Message
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   ----     ------           ----  ----               -------
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Warning  LoggingDisabled  70s   fargate-scheduler  Disabled logging because aws-logging configmap was not found. configmap "aws-logging" not found
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Normal   Scheduled        23s   fargate-scheduler  Successfully assigned default/run-pod-425a4bz9 to fargate-ip-10-0-4-155.us-west-2.compute.internal
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Normal   Pulling          21s   kubelet            Pulling image "amazon/aws-cli:latest"
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Normal   Pulled           7s    kubelet            Successfully pulled image "amazon/aws-cli:latest" in 13.317s (13.317s including waiting)
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Normal   Created          7s    kubelet            Created container base
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook]   Normal   Started          7s    kubelet            Started container base
INFO  [airflow.task.hooks.airflow.hooks.subprocess.SubprocessHook] ^^^ pod description above ^^^

@dstandish
Copy link
Contributor Author

Sorry this took so long @dstandish! You can see the traceback that we see in the tests (for which Vincent's retry resolved and are now back). There is also some other diagnostic info from the logs included:

Sorry, which tests?

Thanks for posting the traceback i'll have a look

@dstandish
Copy link
Contributor Author

Hmm I see, looking at the traceback, that the error is raised from read_pod_logs, which is already wrapped with tenacity. Perhaps instead of adding "more tenacity" at higher level, we should just increase the number of allowable retries? or are we somehow getting a bad set of params that are causing the 500? i would recommend trying to simply increasing the number of retries -- it may be that that's all the reverted tenacity wrapper was accomplishing. lmk how it goes!

@vincbeck
Copy link
Contributor

Hmm I see, looking at the traceback, that the error is raised from read_pod_logs, which is already wrapped with tenacity. Perhaps instead of adding "more tenacity" at higher level, we should just increase the number of allowable retries? or are we somehow getting a bad set of params that are causing the 500? i would recommend trying to simply increasing the number of retries -- it may be that that's all the reverted tenacity wrapper was accomplishing. lmk how it goes!

Very good point! Yes I concur, we should try first to increase the number of attempts to see if it solves the issues

o-nikolas added a commit to aws-mwaa/upstream-to-airflow that referenced this pull request Jan 22, 2024
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache#35504
hussein-awala pushed a commit that referenced this pull request Jan 23, 2024
We're still seeing issues in kubernetes based system tests.

See context and discussions here: #35504
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request Jul 19, 2024
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request Sep 20, 2024
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request Nov 8, 2024
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request May 5, 2025
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request May 26, 2025
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request Sep 21, 2025
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this pull request Oct 19, 2025
We're still seeing issues in kubernetes based system tests.

See context and discussions here: apache/airflow#35504

GitOrigin-RevId: d78a114aa05bee0e02a09a43056827a03180844a
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:providers provider:cncf-kubernetes Kubernetes (k8s) provider related issues

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants