Hi all, I´m currently working on an PR to enable the EdgeWorker with slot/resource handling. PR: https://github.com/apache/airflow/pull/43737.
In the PR we decided to make a devlist discussion about how to get additional task_instance data into the executor. This can be managed in different ways, and this is the idea of this discussion. What I´m talking about: Main idea is that the EdgeWorker supports a slot/resource handling. E.g.: A worker has 3 slots available and executes one task which needs 2 slots, cannot fetch a parallel task which needs 2 slots but can fetch a task which needs 1 slot. This allows the EdgeWorker to have a resource handling as a task which consumes more resources can block other tasks from running on worker. The handling follows same logic like the pools feature of Airflow. But for this the executor needs the information about how many slots are required to execute the task. My first idea was to add the number of slots which is needed by the task into the executor_config as the execute_async function of the BaseExecutor does only use the TaskInstanceKey to get task details. The KubernetesExecutor uses the executor_config parameter to allow some pod-overriding. (See: https://airflow.apache.org/docs/apache-airflow-providers-cncf-kubernetes/stable/kubernetes_executor.html#pod-override) But it feels like a misuse of executor_config parameter to add needed slots of a task into the EdgeExecutor. The discussion went into the direction to change the interface of the executor execute_async function to get more information about the task_instance into the executor. Currently we have the following options: 1) Add Executor specific task metadata into the executor like KubernetesExecutor does. 2) Enable the executor to access the task_instance properties. Regarding option 2: I prepared an example PR (https://github.com/apache/airflow/pull/44016) which adds a new execute function into the BaseExecutor. What is your opinion about this? Looking forward for your feedback. Marco