18
18
from __future__ import unicode_literals
19
19
20
20
import getpass
21
+ import logging
21
22
import multiprocessing
22
23
import os
23
24
import psutil
@@ -285,19 +286,16 @@ class DagFileProcessor(AbstractDagFileProcessor, LoggingMixin):
285
286
# Counter that increments everytime an instance of this class is created
286
287
class_creation_counter = 0
287
288
288
- def __init__ (self , file_path , pickle_dags , dag_id_white_list , log_file ):
289
+ def __init__ (self , file_path , pickle_dags , dag_id_white_list ):
289
290
"""
290
291
:param file_path: a Python file containing Airflow DAG definitions
291
292
:type file_path: unicode
292
293
:param pickle_dags: whether to serialize the DAG objects to the DB
293
294
:type pickle_dags: bool
294
295
:param dag_id_whitelist: If specified, only look at these DAG ID's
295
296
:type dag_id_whitelist: list[unicode]
296
- :param log_file: the path to the file where log lines should be output
297
- :type log_file: unicode
298
297
"""
299
298
self ._file_path = file_path
300
- self ._log_file = log_file
301
299
# Queue that's used to pass results from the child process.
302
300
self ._result_queue = multiprocessing .Queue ()
303
301
# The process that was launched to process the given .
@@ -319,17 +317,12 @@ def __init__(self, file_path, pickle_dags, dag_id_white_list, log_file):
319
317
def file_path (self ):
320
318
return self ._file_path
321
319
322
- @property
323
- def log_file (self ):
324
- return self ._log_file
325
-
326
320
@staticmethod
327
321
def _launch_process (result_queue ,
328
322
file_path ,
329
323
pickle_dags ,
330
324
dag_id_white_list ,
331
- thread_name ,
332
- log_file ):
325
+ thread_name ):
333
326
"""
334
327
Launch a process to process the given file.
335
328
@@ -345,35 +338,21 @@ def _launch_process(result_queue,
345
338
:type dag_id_white_list: list[unicode]
346
339
:param thread_name: the name to use for the process that is launched
347
340
:type thread_name: unicode
348
- :param log_file: the logging output for the process should be directed
349
- to this file
350
- :type log_file: unicode
351
341
:return: the process that was launched
352
342
:rtype: multiprocessing.Process
353
343
"""
354
344
def helper ():
355
345
# This helper runs in the newly created process
356
-
357
- # Re-direct stdout and stderr to a separate log file. Otherwise,
358
- # the main log becomes too hard to read. No buffering to enable
359
- # responsive file tailing
360
- parent_dir , _ = os .path .split (log_file )
361
-
362
- _log = LoggingMixin ().log
363
-
364
- # Create the parent directory for the log file if necessary.
365
- if not os .path .isdir (parent_dir ):
366
- os .makedirs (parent_dir )
367
-
368
- f = open (log_file , "a" )
369
- original_stdout = sys .stdout
370
- original_stderr = sys .stderr
371
-
372
- sys .stdout = f
373
- sys .stderr = f
346
+ log = logging .getLogger ("airflow.processor" )
347
+ for handler in log .handlers :
348
+ try :
349
+ handler .set_context (file_path )
350
+ except AttributeError :
351
+ # Not all handlers need to have context passed in so we ignore
352
+ # the error when handlers do not have set_context defined.
353
+ pass
374
354
375
355
try :
376
- configure_logging ()
377
356
# Re-configure the ORM engine as there are issues with multiple processes
378
357
settings .configure_orm ()
379
358
@@ -383,26 +362,20 @@ def helper():
383
362
threading .current_thread ().name = thread_name
384
363
start_time = time .time ()
385
364
386
- _log .info ("Started process (PID=%s) to work on %s" ,
387
- os .getpid (),
388
- file_path )
389
- scheduler_job = SchedulerJob (dag_ids = dag_id_white_list )
365
+ log .info ("Started process (PID=%s) to work on %s" ,
366
+ os .getpid (), file_path )
367
+ scheduler_job = SchedulerJob (dag_ids = dag_id_white_list , log = log )
390
368
result = scheduler_job .process_file (file_path ,
391
369
pickle_dags )
392
370
result_queue .put (result )
393
371
end_time = time .time ()
394
- _log .info (
395
- "Processing %s took %.3f seconds" ,
396
- file_path , end_time - start_time
372
+ log .info (
373
+ "Processing %s took %.3f seconds" , file_path , end_time - start_time
397
374
)
398
375
except :
399
376
# Log exceptions through the logging framework.
400
- _log .exception ("Got an exception! Propagating..." )
377
+ log .exception ("Got an exception! Propagating..." )
401
378
raise
402
- finally :
403
- sys .stdout = original_stdout
404
- sys .stderr = original_stderr
405
- f .close ()
406
379
407
380
p = multiprocessing .Process (target = helper ,
408
381
args = (),
@@ -419,8 +392,7 @@ def start(self):
419
392
self .file_path ,
420
393
self ._pickle_dags ,
421
394
self ._dag_id_white_list ,
422
- "DagFileProcessor{}" .format (self ._instance_id ),
423
- self .log_file )
395
+ "DagFileProcessor{}" .format (self ._instance_id ))
424
396
self ._start_time = datetime .utcnow ()
425
397
426
398
def terminate (self , sigkill = False ):
@@ -538,6 +510,7 @@ def __init__(
538
510
processor_poll_interval = 1.0 ,
539
511
run_duration = None ,
540
512
do_pickle = False ,
513
+ log = None ,
541
514
* args , ** kwargs ):
542
515
"""
543
516
:param dag_id: if specified, only schedule tasks with this DAG ID
@@ -574,6 +547,10 @@ def __init__(
574
547
575
548
self .heartrate = conf .getint ('scheduler' , 'SCHEDULER_HEARTBEAT_SEC' )
576
549
self .max_threads = conf .getint ('scheduler' , 'max_threads' )
550
+
551
+ if log :
552
+ self ._log = log
553
+
577
554
self .using_sqlite = False
578
555
if 'sqlite' in conf .get ('core' , 'sql_alchemy_conn' ):
579
556
if self .max_threads > 1 :
@@ -591,9 +568,7 @@ def __init__(
591
568
# Parse and schedule each file no faster than this interval. Default
592
569
# to 3 minutes.
593
570
self .file_process_interval = file_process_interval
594
- # Directory where log files for the processes that scheduled the DAGs reside
595
- self .child_process_log_directory = conf .get ('scheduler' ,
596
- 'child_process_log_directory' )
571
+
597
572
self .max_tis_per_query = conf .getint ('scheduler' , 'max_tis_per_query' )
598
573
if run_duration is None :
599
574
self .run_duration = conf .getint ('scheduler' ,
@@ -1548,17 +1523,15 @@ def _execute(self):
1548
1523
known_file_paths = list_py_file_paths (self .subdir )
1549
1524
self .log .info ("There are %s files in %s" , len (known_file_paths ), self .subdir )
1550
1525
1551
- def processor_factory (file_path , log_file_path ):
1526
+ def processor_factory (file_path ):
1552
1527
return DagFileProcessor (file_path ,
1553
1528
pickle_dags ,
1554
- self .dag_ids ,
1555
- log_file_path )
1529
+ self .dag_ids )
1556
1530
1557
1531
processor_manager = DagFileProcessorManager (self .subdir ,
1558
1532
known_file_paths ,
1559
1533
self .max_threads ,
1560
1534
self .file_process_interval ,
1561
- self .child_process_log_directory ,
1562
1535
self .num_runs ,
1563
1536
processor_factory )
1564
1537
0 commit comments