jscheffl commented on code in PR #42048:
URL: https://github.com/apache/airflow/pull/42048#discussion_r1769649484


##########
airflow/providers/edge/executors/edge_executor.py:
##########
@@ -0,0 +1,175 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from datetime import datetime, timedelta
+from typing import TYPE_CHECKING, Any
+
+from sqlalchemy import delete
+
+from airflow.cli.cli_config import GroupCommand
+from airflow.configuration import conf
+from airflow.executors.base_executor import BaseExecutor
+from airflow.models.abstractoperator import DEFAULT_QUEUE
+from airflow.models.taskinstance import TaskInstanceState
+from airflow.providers.edge.models.edge_job import EdgeJobModel
+from airflow.providers.edge.models.edge_logs import EdgeLogsModel
+from airflow.providers.edge.models.edge_worker import EdgeWorkerModel
+from airflow.utils.db import DBLocks, create_global_lock
+from airflow.utils.session import NEW_SESSION, provide_session
+
+if TYPE_CHECKING:
+    import argparse
+
+    from sqlalchemy.orm import Session
+
+    from airflow.executors.base_executor import CommandType
+    from airflow.models.taskinstance import TaskInstance
+    from airflow.models.taskinstancekey import TaskInstanceKey
+
+PARALLELISM: int = conf.getint("core", "PARALLELISM")
+
+
+class EdgeExecutor(BaseExecutor):
+    """Implementation of the EdgeExecutor to distribute work to Edge Workers 
via HTTP."""
+
+    def __init__(self, parallelism: int = PARALLELISM):
+        super().__init__(parallelism=parallelism)
+        self.last_reported_state: dict[TaskInstanceKey, TaskInstanceState] = {}
+
+    @provide_session
+    def start(self, session: Session = NEW_SESSION):
+        """If EdgeExecutor provider is loaded first time, ensure table 
exists."""
+        with create_global_lock(session=session, lock=DBLocks.MIGRATIONS):
+            engine = session.get_bind().engine
+            EdgeJobModel.metadata.create_all(engine)
+            EdgeLogsModel.metadata.create_all(engine)
+            EdgeWorkerModel.metadata.create_all(engine)
+
+    @provide_session
+    def execute_async(
+        self,
+        key: TaskInstanceKey,
+        command: CommandType,
+        queue: str | None = None,
+        executor_config: Any | None = None,
+        session: Session = NEW_SESSION,
+    ) -> None:
+        """Execute asynchronously."""
+        self.validate_airflow_tasks_run_command(command)
+        session.add(
+            EdgeJobModel(
+                dag_id=key.dag_id,
+                task_id=key.task_id,
+                run_id=key.run_id,
+                map_index=key.map_index,
+                try_number=key.try_number,
+                state=TaskInstanceState.QUEUED,
+                queue=queue or DEFAULT_QUEUE,
+                command=str(command),
+            )
+        )
+
+    @provide_session
+    def sync(self, session: Session = NEW_SESSION) -> None:
+        """Sync will get called periodically by the heartbeat method."""
+        jobs: list[EdgeJobModel] = session.query(EdgeJobModel).all()
+        for job in jobs:
+            if job.key in self.running:
+                if job.state == TaskInstanceState.RUNNING:
+                    if (
+                        job.key not in self.last_reported_state
+                        or self.last_reported_state[job.key] != job.state
+                    ):
+                        self.running_state(job.key)
+                    self.last_reported_state[job.key] = job.state
+                elif job.state == TaskInstanceState.SUCCESS:
+                    if job.key in self.last_reported_state:
+                        del self.last_reported_state[job.key]
+                    self.success(job.key)
+                elif job.state == TaskInstanceState.FAILED:
+                    if job.key in self.last_reported_state:
+                        del self.last_reported_state[job.key]
+                    self.fail(job.key)
+                else:
+                    self.last_reported_state[job.key] = job.state
+            job_success_purge = conf.getint("edge", "job_success_purge")
+            job_fail_purge = conf.getint("edge", "job_fail_purge")
+            if (
+                job.state == TaskInstanceState.SUCCESS
+                and job.last_update_t < (datetime.now() - 
timedelta(minutes=job_success_purge)).timestamp()
+            ) or (
+                job.state == TaskInstanceState.FAILED
+                and job.last_update_t < (datetime.now() - 
timedelta(minutes=job_fail_purge)).timestamp()
+            ):
+                if job.key in self.last_reported_state:
+                    del self.last_reported_state[job.key]
+                session.delete(job)
+                session.execute(
+                    delete(EdgeLogsModel).where(
+                        EdgeLogsModel.dag_id == job.dag_id,
+                        EdgeLogsModel.run_id == job.run_id,
+                        EdgeLogsModel.task_id == job.task_id,
+                        EdgeLogsModel.map_index == job.map_index,
+                        EdgeLogsModel.try_number == job.try_number,
+                    )
+                )
+        session.commit()
+
+    def end(self) -> None:
+        """End the executor."""
+        self.log.info("Shutting down EdgeExecutor")

Review Comment:
   Hi @o-nikolas ... took a moment but now I was able to make the stability 
checks, this is the outcome:
   
   Tests were based in the "Integration Test" DAG which contains all tested 
functions and operators which is added to provider package - will be base of 
system test in future if I get to this point. As you asked for heartbeat - 
there is one "long running" tasks that sleeps 1 minute and sends some logs for 
15 minutes runtime - so a task that is hitting the heartbeat timeout. Added 
this before to ensure heartbeating is working and long running tasks can be 
made on the edge.
   
   - [x] Positive case - let it run - SUCCESS - long running task reported OK
   - [x] Kill -9 the "task run" process - FAILED as expected - task was 
reported as failed (immediately). Logs are expected like:
         `INFO - Task exited with return code -9For more information, see 
https://airflow.apache.org/docs/apache-airflow/stable/troubleshooting.html#LocalTaskJob-killed`
   - [x] Kill -9 edge worker, leave forked long task process running - SUCCESS 
(because task completes also w/o worker and updates the state itself via task 
runner API)
         Side effect is that logs are not uploaded as this is done by the 
worker which is dead.
         After the restart of Edge Worker, the DAG picked the last task and 
ended with success.
         Future improvement will be to upload leftover logs after restart of 
edge worker
   - [x] Kill -9 edge worker and task runner (like if a container or server 
dies - FAILED as expected
         After 300(+) seconds - the task was detected as zombie and was set to 
failed by the scheduler.
         No need to have something special implemented on the EdgeExecutor or 
Edge Worker (which was dead anyway)
   - [x] Cut network connectivity (Actually stop webserver/API) - FAILED - as 
expected
         Edge worker as well as task runner tried via @tenacity.retry() to 
retry 10 times which took ~9minutes. Then gave-up and terminated.
         If the webserver/connection recovers within the retry attempts the 
task will continue and all problems are masked.
         In my case I did not restart webserver. So Edge Worker and task runner 
terminated/gave-up calling.
         In the scheduler the running task was reported as zombie as exected.
   
   As all Executors work on the common DB table and API calls go indirect via 
DB... in the Executor there is no real ownership needed in my view. Also no 
adoption. Based on the DB table Executors just report back the "running" tasks 
for the planning of PARALLELISM.
   
   Would this be OK for you? Or do you see an task adoption is needed?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to