Bugfix: resources in `executor_config` breaks Graph View in UI (#15199)
closes https://github.com/apache/airflow/issues/14327
When using `KubernetesExecutor` and the task as follows:
```python
PythonOperator(
task_id=f"sync_{table_name}",
python_callable=sync_table,
provide_context=True,
op_kwargs={"table_name": table_name},
executor_config={"KubernetesExecutor": {"request_cpu": "1"}},
retries=5,
dag=dag,
)
```
it breaks the UI as settings resources in such a way is only there
for backwards compatibility.
This commits fixes it.
(cherry picked from commit 7b577c35e2
)
This commit is contained in:
Родитель
a7e80b194f
Коммит
dc22f8cb71
|
@ -66,7 +66,7 @@ class AirflowJsonEncoder(JSONEncoder):
|
|||
obj, (np.float_, np.float16, np.float32, np.float64, np.complex_, np.complex64, np.complex128)
|
||||
):
|
||||
return float(obj)
|
||||
elif k8s is not None and isinstance(obj, k8s.V1Pod):
|
||||
elif k8s is not None and isinstance(obj, (k8s.V1Pod, k8s.V1ResourceRequirements)):
|
||||
from airflow.kubernetes.pod_generator import PodGenerator
|
||||
|
||||
return PodGenerator.serialize_pod(obj)
|
||||
|
|
Загрузка…
Ссылка в новой задаче