Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix TaskNotFound in log endpoint #13872

Merged
merged 1 commit into from
Jan 24, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion airflow/api_connexion/endpoints/log_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
from airflow.api_connexion import security
from airflow.api_connexion.exceptions import BadRequest, NotFound
from airflow.api_connexion.schemas.log_schema import LogResponseObject, logs_schema
from airflow.exceptions import TaskNotFound
from airflow.models import DagRun
from airflow.security import permissions
from airflow.utils.log.log_reader import TaskLogReader
Expand Down Expand Up @@ -71,7 +72,10 @@ def get_log(session, dag_id, dag_run_id, task_id, task_try_number, full_content=

dag = current_app.dag_bag.get_dag(dag_id)
if dag:
ti.task = dag.get_task(ti.task_id)
try:
ti.task = dag.get_task(ti.task_id)
except TaskNotFound:
pass

return_type = request.accept_mimetypes.best_match(['text/plain', 'application/json'])

Expand Down
28 changes: 28 additions & 0 deletions tests/api_connexion/endpoints/test_log_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,34 @@ def test_should_respond_200_text_plain(self, session):
== f"\n*** Reading local file: {expected_filename}\nLog for testing.\n"
)

@provide_session
def test_get_logs_of_removed_task(self, session):
self._create_dagrun(session)

# Recreate DAG without tasks
dagbag = self.app.dag_bag # pylint: disable=no-member
dag = DAG(self.DAG_ID, start_date=timezone.parse(self.default_time))
dagbag.bag_dag(dag=dag, root_dag=dag)

key = self.app.config["SECRET_KEY"]
serializer = URLSafeSerializer(key)
token = serializer.dumps({"download_logs": True})

response = self.client.get(
f"api/v1/dags/{self.DAG_ID}/dagRuns/TEST_DAG_RUN_ID/"
f"taskInstances/{self.TASK_ID}/logs/1?token={token}",
headers={'Accept': 'text/plain'},
environ_overrides={'REMOTE_USER': "test"},
)
expected_filename = "{}/{}/{}/{}/1.log".format(
self.log_dir, self.DAG_ID, self.TASK_ID, self.default_time.replace(':', '.')
)
assert 200 == response.status_code
assert (
response.data.decode('utf-8')
== f"\n*** Reading local file: {expected_filename}\nLog for testing.\n"
)

@provide_session
def test_get_logs_response_with_ti_equal_to_none(self, session):
self._create_dagrun(session)
Expand Down