Getting the tests to run
This commit is contained in:
Родитель
9954ee45f3
Коммит
4ab60cc4a7
|
@ -27,7 +27,7 @@ def ds_add(ds, days):
|
|||
|
||||
def ds_format(ds, input_format, output_format):
|
||||
'''
|
||||
Takes an input string and outputs another string
|
||||
Takes an input string and outputs another string
|
||||
as specified in the output format
|
||||
|
||||
:param ds: input string which contains a date
|
||||
|
@ -39,7 +39,7 @@ def ds_format(ds, input_format, output_format):
|
|||
|
||||
>>> ds_format('2015-01-01', "%Y-%m-%d", "%m-%d-%y")
|
||||
'01-01-15'
|
||||
>>> ds_format('2015-01-01', "%Y-%m-%d", "%Y-%m-%d")
|
||||
'2015-01-01'
|
||||
>>> ds_format('1/5/2015', "%m/%d/%Y", "%Y-%m-%d")
|
||||
'2015-01-05'
|
||||
'''
|
||||
return datetime.datetime.strptime(ds, input_format).strftime(output_format)
|
||||
return datetime.strptime(ds, input_format).strftime(output_format)
|
||||
|
|
|
@ -79,11 +79,12 @@ class DagBag(object):
|
|||
"""
|
||||
def __init__(
|
||||
self,
|
||||
dag_folder=DAGS_FOLDER,
|
||||
dag_folder=None,
|
||||
executor=DEFAULT_EXECUTOR,
|
||||
include_examples=True,
|
||||
sync_to_db=False):
|
||||
|
||||
dag_folder = dag_folder or DAGS_FOLDER
|
||||
logging.info("Filling up the DagBag from " + dag_folder)
|
||||
self.dag_folder = dag_folder
|
||||
self.dags = {}
|
||||
|
|
|
@ -167,8 +167,8 @@ class CoreTest(unittest.TestCase):
|
|||
job = jobs.LocalTaskJob(task_instance=ti, force=True)
|
||||
job.run()
|
||||
|
||||
def test_master_job(self):
|
||||
job = jobs.MasterJob(dag_id='example_bash_operator', test_mode=True)
|
||||
def test_scheduler_job(self):
|
||||
job = jobs.SchedulerJob(dag_id='example_bash_operator', test_mode=True)
|
||||
job.run()
|
||||
|
||||
def test_local_backfill_job(self):
|
||||
|
|
Загрузка…
Ссылка в новой задаче