Skip to content

Commit 99ee040

Browse files
Redwan Rahmanaoen
Redwan Rahman
authored andcommitted
[TWTR][CX-14365] additional logging to help debug scheduler hang, can be removed/reverted once issue is resolved
1 parent f4915fc commit 99ee040

File tree

5 files changed

+26
-1
lines changed

5 files changed

+26
-1
lines changed

airflow/jobs.py

+12
Original file line numberDiff line numberDiff line change
@@ -365,7 +365,10 @@ def helper():
365365
stdout = StreamLogWriter(log, logging.INFO)
366366
stderr = StreamLogWriter(log, logging.WARN)
367367

368+
log.info("Setting log context for file {}".format(file_path))
369+
# log file created here
368370
set_context(log, file_path)
371+
log.info("Successfully set log context for file {}".format(file_path))
369372

370373
try:
371374
# redirect stdout/stderr to log
@@ -384,6 +387,7 @@ def helper():
384387
log.info("Started process (PID=%s) to work on %s",
385388
os.getpid(), file_path)
386389
scheduler_job = SchedulerJob(dag_ids=dag_id_white_list, log=log)
390+
log.info("Processing file {}".format(file_path))
387391
result = scheduler_job.process_file(file_path,
388392
pickle_dags)
389393
result_queue.put(result)
@@ -412,6 +416,7 @@ def start(self):
412416
"""
413417
Launch the process and start processing the DAG.
414418
"""
419+
self.log.info("Launching process to process DAG at {}".format(self.file_path))
415420
self._process = DagFileProcessor._launch_process(
416421
self._result_queue,
417422
self.file_path,
@@ -1671,6 +1676,7 @@ def _execute_helper(self, processor_manager):
16711676
# Kick of new processes and collect results from finished ones
16721677
self.log.debug("Heartbeating the process manager")
16731678
simple_dags = processor_manager.heartbeat()
1679+
self.log.debug("Finished process manager heartbeat")
16741680

16751681
if self.using_sqlite:
16761682
# For the sqlite case w/ 1 thread, wait until the processor
@@ -1700,6 +1706,9 @@ def _execute_helper(self, processor_manager):
17001706
State.SCHEDULED],
17011707
State.NONE)
17021708

1709+
scheduled_dag_ids = ", ".join(simple_dag_bag.dag_ids)
1710+
self.log.info('DAGs to be executed: {}'.format(scheduled_dag_ids))
1711+
17031712
self._execute_task_instances(simple_dag_bag,
17041713
(State.SCHEDULED,))
17051714

@@ -1741,14 +1750,17 @@ def _execute_helper(self, processor_manager):
17411750
break
17421751

17431752
# Stop any processors
1753+
self.log.info("Terminating DAG processors")
17441754
processor_manager.terminate()
1755+
self.log.info("All DAG processors terminated")
17451756

17461757
# Verify that all files were processed, and if so, deactivate DAGs that
17471758
# haven't been touched by the scheduler as they likely have been
17481759
# deleted.
17491760
all_files_processed = True
17501761
for file_path in known_file_paths:
17511762
if processor_manager.get_last_finish_time(file_path) is None:
1763+
self.log.info("File {} not processed".format(file_path))
17521764
all_files_processed = False
17531765
break
17541766
if all_files_processed:

airflow/models.py

+6
Original file line numberDiff line numberDiff line change
@@ -4185,6 +4185,8 @@ def sync_to_db(self, owner=None, sync_time=None, session=None):
41854185
:return: None
41864186
"""
41874187

4188+
self.log.info("Attempting to sync DAG {} to DB".format(self._dag_id))
4189+
41884190
if owner is None:
41894191
owner = self.owner
41904192
if sync_time is None:
@@ -4203,8 +4205,12 @@ def sync_to_db(self, owner=None, sync_time=None, session=None):
42034205
session.merge(orm_dag)
42044206
session.commit()
42054207

4208+
self.log.info("Synced DAG %s to DB", self._dag_id)
4209+
42064210
for subdag in self.subdags:
4211+
self.log.info("Syncing SubDAG %s", subdag._dag_id)
42074212
subdag.sync_to_db(owner=owner, sync_time=sync_time, session=session)
4213+
self.log.info("Successfully synced SubDAG %s", subdag._dag_id)
42084214

42094215
@staticmethod
42104216
@provide_session

airflow/utils/dag_processing.py

+5
Original file line numberDiff line numberDiff line change
@@ -565,10 +565,15 @@ def heartbeat(self):
565565
processor.pid, file_path
566566
)
567567
self._processors[file_path] = processor
568+
569+
self.log.info("Number of active file processors: {}".format(len(self._processors)))
568570

569571
# Update scheduler heartbeat count.
570572
self._run_count[self._heart_beat_key] += 1
571573

574+
simple_dag_ids = ", ".join([simple_dag.dag_id for simple_dag in simple_dags])
575+
self.log.info("Processed DAGs: {}".format(simple_dag_ids))
576+
572577
return simple_dags
573578

574579
def max_runs_reached(self):

airflow/utils/log/file_processor_handler.py

+2
Original file line numberDiff line numberDiff line change
@@ -138,9 +138,11 @@ def _init_file(self, filename):
138138
directory = os.path.dirname(full_path)
139139

140140
if not os.path.exists(directory):
141+
logging.info("Creating directory {}".format(directory))
141142
os.makedirs(directory)
142143

143144
if not os.path.exists(full_path):
145+
logging.info("Creating file {}".format(full_path))
144146
open(full_path, "a").close()
145147

146148
return full_path

airflow/version.py

+1-1
Original file line numberDiff line numberDiff line change
@@ -18,4 +18,4 @@
1818
# under the License.
1919
#
2020

21-
version = '1.10.0+twtr11'
21+
version = '1.10.0+twtr12'

0 commit comments

Comments
 (0)