From f229eb320c05564bca5cb503e910f021e7ce6f0e Mon Sep 17 00:00:00 2001 From: Erik Bernhardsson Date: Sun, 8 Mar 2015 15:01:58 -0400 Subject: [PATCH] Renamed MockFile -> MockTarget I wonder if we should call it InMemoryTarget or something instead? Also deleted some LocalTarget->MockTarget swapping in the global scope Fixed an issue with the test coverage for MockFile (constructor wasn't run) --- luigi/mock.py | 15 +++++++++++---- test/cmdline_test.py | 14 ++++++-------- test/decorator_test.py | 14 +++++++------- test/fib_test.py | 22 +++++++++------------- test/file_test.py | 2 +- test/hadoop_test.py | 7 +++---- test/instance_wrap_test.py | 5 ++--- test/mock_test.py | 17 ++++++++++++----- test/optparse_test.py | 10 +++++----- test/parameter_test.py | 10 +++++----- test/range_test.py | 22 +++++++++++----------- test/recursion_test.py | 10 ++++------ test/spark_test.py | 14 +++++++------- test/test_event_callbacks.py | 4 ++-- test/worker_test.py | 6 +++--- test/wrap_test.py | 17 ++++++++--------- 16 files changed, 96 insertions(+), 93 deletions(-) diff --git a/luigi/mock.py b/luigi/mock.py index 1a8895bae8..5ed8362109 100644 --- a/luigi/mock.py +++ b/luigi/mock.py @@ -15,8 +15,8 @@ # limitations under the License. # """ -This moduel provides a class :class:`MockFile`, an implementation of :py:class:`~luigi.target.Target`. -:class:`MockFile` contains all data in-memory. +This moduel provides a class :class:`MockTarget`, an implementation of :py:class:`~luigi.target.Target`. +:class:`MockTarget` contains all data in-memory. The main purpose is unit testing workflows without writing to disk. """ @@ -24,6 +24,7 @@ from io import BytesIO import sys +import warnings from luigi import six import luigi.util @@ -47,7 +48,7 @@ def get_data(self, fn): return self.get_all_data()[fn] def exists(self, path): - return MockFile(path).exists() + return MockTarget(path).exists() def remove(self, path, recursive=True, skip_trash=True): """ @@ -80,7 +81,7 @@ def clear(self): self.get_all_data().clear() -class MockFile(target.FileSystemTarget): +class MockTarget(target.FileSystemTarget): fs = MockFileSystem() def __init__(self, fn, is_tmp=None, mirror_on_stderr=False, format=None): @@ -167,3 +168,9 @@ def seekable(self2): return wrapper else: return self.format.pipe_reader(Buffer(self.fs.get_all_data()[fn])) + + +class MockFile(MockTarget): + def __init__(self, *args, **kwargs): + warnings.warn("MockFile has been renamed MockTarget", DeprecationWarning, stacklevel=2) + super(MockFile, self).__init__(*args, **kwargs) diff --git a/test/cmdline_test.py b/test/cmdline_test.py index 3cca1fd049..819a910c08 100644 --- a/test/cmdline_test.py +++ b/test/cmdline_test.py @@ -30,14 +30,14 @@ import luigi import mock -from luigi.mock import MockFile +from luigi.mock import MockTarget class SomeTask(luigi.Task): n = luigi.IntParameter() def output(self): - return LocalTarget('/tmp/test_%d' % self.n) + return MockTarget('/tmp/test_%d' % self.n) def run(self): f = self.output().open('w') @@ -80,7 +80,7 @@ class WriteToFile(luigi.Task): filename = luigi.Parameter() def output(self): - return luigi.LocalTarget(self.filename) + return MockTarget(self.filename) def run(self): f = self.output().open('w') @@ -91,19 +91,17 @@ def run(self): class CmdlineTest(unittest.TestCase): def setUp(self): - global LocalTarget - LocalTarget = MockFile - MockFile.fs.clear() + MockTarget.fs.clear() @mock.patch("logging.getLogger") def test_cmdline_main_task_cls(self, logger): luigi.run(['--local-scheduler', '--no-lock', '--n', '100'], main_task_cls=SomeTask) - self.assertEqual(dict(MockFile.fs.get_all_data()), {'/tmp/test_100': b'done'}) + self.assertEqual(dict(MockTarget.fs.get_all_data()), {'/tmp/test_100': b'done'}) @mock.patch("logging.getLogger") def test_cmdline_other_task(self, logger): luigi.run(['--local-scheduler', '--no-lock', 'SomeTask', '--n', '1000']) - self.assertEqual(dict(MockFile.fs.get_all_data()), {'/tmp/test_1000': b'done'}) + self.assertEqual(dict(MockTarget.fs.get_all_data()), {'/tmp/test_1000': b'done'}) @mock.patch("logging.getLogger") def test_cmdline_ambiguous_class(self, logger): diff --git a/test/decorator_test.py b/test/decorator_test.py index bbf83cb3e0..7176584d9d 100644 --- a/test/decorator_test.py +++ b/test/decorator_test.py @@ -23,7 +23,7 @@ import luigi import luigi.notifications from luigi.interface import ArgParseInterface -from luigi.mock import MockFile +from luigi.mock import MockTarget from luigi.parameter import MissingParameterException from luigi.util import common_params, copies, delegates, inherits, requires @@ -291,7 +291,7 @@ class P(luigi.Task): date = luigi.DateParameter() def output(self): - return MockFile(self.date.strftime('/tmp/data-%Y-%m-%d.txt')) + return MockTarget(self.date.strftime('/tmp/data-%Y-%m-%d.txt')) def run(self): f = self.output().open('w') @@ -303,15 +303,15 @@ def run(self): class PCopy(luigi.Task): def output(self): - return MockFile(self.date.strftime('/tmp/copy-data-%Y-%m-%d.txt')) + return MockTarget(self.date.strftime('/tmp/copy-data-%Y-%m-%d.txt')) class CopyTest(unittest.TestCase): def test_copy(self): luigi.build([PCopy(date=datetime.date(2012, 1, 1))], local_scheduler=True) - self.assertEqual(MockFile.fs.get_data('/tmp/data-2012-01-01.txt'), b'hello, world\n') - self.assertEqual(MockFile.fs.get_data('/tmp/copy-data-2012-01-01.txt'), b'hello, world\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/data-2012-01-01.txt'), b'hello, world\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/copy-data-2012-01-01.txt'), b'hello, world\n') class PickleTest(unittest.TestCase): @@ -323,8 +323,8 @@ def test_pickle(self): p = pickle.loads(p_pickled) luigi.build([p], local_scheduler=True) - self.assertEqual(MockFile.fs.get_data('/tmp/data-2013-01-01.txt'), b'hello, world\n') - self.assertEqual(MockFile.fs.get_data('/tmp/copy-data-2013-01-01.txt'), b'hello, world\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/data-2013-01-01.txt'), b'hello, world\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/copy-data-2013-01-01.txt'), b'hello, world\n') class Subtask(luigi.Task): diff --git a/test/fib_test.py b/test/fib_test.py index 40d51b07e3..6fbc0bb6e5 100644 --- a/test/fib_test.py +++ b/test/fib_test.py @@ -19,9 +19,7 @@ import luigi import luigi.interface -from luigi.mock import MockFile - -LocalTarget = MockFile +from luigi.mock import MockTarget # Calculates Fibonacci numbers :) @@ -36,7 +34,7 @@ def requires(self): return [] def output(self): - return LocalTarget('/tmp/fib_%d' % self.n) + return MockTarget('/tmp/fib_%d' % self.n) def run(self): if self.n == 0: @@ -57,9 +55,7 @@ def run(self): class FibTestBase(unittest.TestCase): def setUp(self): - global LocalTarget - LocalTarget = MockFile - MockFile.fs.clear() + MockTarget.fs.clear() class FibTest(FibTestBase): @@ -69,20 +65,20 @@ def test_invoke(self): w.add(Fib(100)) w.run() w.stop() - self.assertEqual(MockFile.fs.get_data('/tmp/fib_10'), b'55\n') - self.assertEqual(MockFile.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_10'), b'55\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') def test_cmdline(self): luigi.run(['--local-scheduler', '--no-lock', 'Fib', '--n', '100']) - self.assertEqual(MockFile.fs.get_data('/tmp/fib_10'), b'55\n') - self.assertEqual(MockFile.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_10'), b'55\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') def test_build_internal(self): luigi.build([Fib(100)], local_scheduler=True) - self.assertEqual(MockFile.fs.get_data('/tmp/fib_10'), b'55\n') - self.assertEqual(MockFile.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_10'), b'55\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') if __name__ == '__main__': luigi.run() diff --git a/test/file_test.py b/test/file_test.py index 5315a6f185..3a8842731a 100644 --- a/test/file_test.py +++ b/test/file_test.py @@ -226,4 +226,4 @@ class TestImportFile(unittest.TestCase): def test_file(self): from luigi.file import File - self.assertTrue(issubclass(File, LocalTarget)) + self.assertTrue(isinstance(File('foo'), LocalTarget)) diff --git a/test/hadoop_test.py b/test/hadoop_test.py index 11f36b2762..be3b698c4f 100644 --- a/test/hadoop_test.py +++ b/test/hadoop_test.py @@ -27,11 +27,10 @@ import luigi.notifications import minicluster import mock -from luigi.mock import MockFile +from luigi.mock import MockTarget from nose.plugins.attrib import attr luigi.notifications.DEBUG = True -LocalTarget = MockFile luigi.hadoop.attach(minicluster) @@ -43,7 +42,7 @@ def get_output(self, fn): if self.use_hdfs: return luigi.hdfs.HdfsTarget('/tmp/' + fn, format=luigi.format.get_default_format() >> luigi.hdfs.PlainDir) else: - return LocalTarget(fn) + return MockTarget(fn) class HadoopJobTask(luigi.hadoop.JobTask, OutputMixin): @@ -236,7 +235,7 @@ def test_instantiate_job(self): MyStreamingJob('param_value') def setUp(self): - MockFile.fs.clear() + MockTarget.fs.clear() @attr('minicluster') diff --git a/test/instance_wrap_test.py b/test/instance_wrap_test.py index 43c878f53f..1dcce64d4e 100644 --- a/test/instance_wrap_test.py +++ b/test/instance_wrap_test.py @@ -21,10 +21,9 @@ import luigi import luigi.notifications -from luigi.mock import MockFile +from luigi.mock import MockTarget luigi.notifications.DEBUG = True -LocalTarget = MockFile class Report(luigi.Task): @@ -38,7 +37,7 @@ def run(self): f.close() def output(self): - return LocalTarget(self.date.strftime('/tmp/report-%Y-%m-%d')) + return MockTarget(self.date.strftime('/tmp/report-%Y-%m-%d')) class ReportReader(luigi.Task): diff --git a/test/mock_test.py b/test/mock_test.py index 29fa3bf6f0..45af3b358f 100644 --- a/test/mock_test.py +++ b/test/mock_test.py @@ -18,13 +18,13 @@ from helpers import unittest -from luigi.mock import MockFile, MockFileSystem +from luigi.mock import MockTarget, MockFileSystem class MockFileTest(unittest.TestCase): def test_1(self): - t = MockFile('test') + t = MockTarget('test') p = t.open('w') print('test', file=p) p.close() @@ -34,7 +34,7 @@ def test_1(self): q.close() def test_with(self): - t = MockFile("foo") + t = MockTarget("foo") with t.open('w') as b: b.write("bar") @@ -44,7 +44,7 @@ def test_with(self): # That should work in python2 because of the autocast # That should work in python3 because the default format is Text def test_unicode(self): - t = MockFile("foo") + t = MockTarget("foo") with t.open('w') as b: b.write(u"bar") @@ -56,7 +56,7 @@ class MockFileSystemTest(unittest.TestCase): fs = MockFileSystem() def _touch(self, path): - t = MockFile(path) + t = MockTarget(path) with t.open('w'): pass @@ -81,3 +81,10 @@ def test_remove_recursive(self): def test_listdir(self): self.assertEqual(sorted([self.path, self.path2]), sorted(self.fs.listdir("/tmp"))) + + +class TestImportMockFile(unittest.TestCase): + + def test_mockfile(self): + from luigi.mock import MockFile + self.assertTrue(isinstance(MockFile('foo'), MockTarget)) diff --git a/test/optparse_test.py b/test/optparse_test.py index dee889c3eb..3a105dccda 100644 --- a/test/optparse_test.py +++ b/test/optparse_test.py @@ -17,7 +17,7 @@ import luigi from fib_test import FibTestBase -from luigi.mock import MockFile +from luigi.mock import MockTarget class OptParseTest(FibTestBase): @@ -25,8 +25,8 @@ class OptParseTest(FibTestBase): def test_cmdline_optparse(self): luigi.run(['--local-scheduler', '--no-lock', '--task', 'Fib', '--n', '100'], use_optparse=True) - self.assertEqual(MockFile.fs.get_data('/tmp/fib_10'), b'55\n') - self.assertEqual(MockFile.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_10'), b'55\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') def test_cmdline_optparse_existing(self): import optparse @@ -35,5 +35,5 @@ def test_cmdline_optparse_existing(self): luigi.run(['--local-scheduler', '--no-lock', '--task', 'Fib', '--n', '100'], use_optparse=True, existing_optparse=parser) - self.assertEqual(MockFile.fs.get_data('/tmp/fib_10'), b'55\n') - self.assertEqual(MockFile.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_10'), b'55\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/fib_100'), b'354224848179261915075\n') diff --git a/test/parameter_test.py b/test/parameter_test.py index 3382828e0f..ba674f1d3c 100644 --- a/test/parameter_test.py +++ b/test/parameter_test.py @@ -24,7 +24,7 @@ import luigi.interface import luigi.notifications from helpers import with_config -from luigi.mock import MockFile, MockFileSystem +from luigi.mock import MockTarget, MockFileSystem from luigi.parameter import ParameterException from worker_test import email_patch @@ -110,7 +110,7 @@ class BananaDep(luigi.Task): y = luigi.Parameter(default='def') def output(self): - return MockFile('banana-dep-%s-%s' % (self.x, self.y)) + return MockTarget('banana-dep-%s-%s' % (self.x, self.y)) def run(self): self.output().open('w').close() @@ -134,7 +134,7 @@ def requires(self): raise Exception('unknown style') def output(self): - return MockFile('banana-%s-%s' % (self.x, self.y)) + return MockTarget('banana-%s-%s' % (self.x, self.y)) def run(self): self.output().open('w').close() @@ -318,11 +318,11 @@ class TestNewStyleGlobalParameters(unittest.TestCase): def setUp(self): super(TestNewStyleGlobalParameters, self).setUp() - MockFile.fs.clear() + MockTarget.fs.clear() BananaDep.y.reset_global() def expect_keys(self, expected): - self.assertEquals(set(MockFile.fs.get_all_data().keys()), set(expected)) + self.assertEquals(set(MockTarget.fs.get_all_data().keys()), set(expected)) def test_x_arg(self): luigi.run(['--local-scheduler', '--no-lock', 'Banana', '--x', 'foo', '--y', 'bar', '--style', 'x-arg']) diff --git a/test/range_test.py b/test/range_test.py index 4a45653b4c..ae9443d770 100644 --- a/test/range_test.py +++ b/test/range_test.py @@ -21,7 +21,7 @@ import luigi import mock -from luigi.mock import MockFile, MockFileSystem +from luigi.mock import MockTarget, MockFileSystem from luigi.tools.range import (RangeDaily, RangeDailyBase, RangeEvent, RangeHourly, RangeHourlyBase, _constrain_glob, _get_filesystems_and_globs) @@ -30,14 +30,14 @@ class CommonDateHourTask(luigi.Task): dh = luigi.DateHourParameter() def output(self): - return MockFile(self.dh.strftime('/n2000y01a05n/%Y_%m-_-%daww/21mm%Hdara21/ooo')) + return MockTarget(self.dh.strftime('/n2000y01a05n/%Y_%m-_-%daww/21mm%Hdara21/ooo')) class CommonDateTask(luigi.Task): d = luigi.DateParameter() def output(self): - return MockFile(self.d.strftime('/n2000y01a05n/%Y_%m-_-%daww/21mm01dara21/ooo')) + return MockTarget(self.d.strftime('/n2000y01a05n/%Y_%m-_-%daww/21mm01dara21/ooo')) task_a_paths = [ @@ -92,7 +92,7 @@ class TaskA(luigi.Task): dh = luigi.DateHourParameter() def output(self): - return MockFile(self.dh.strftime('TaskA/%Y-%m-%d/%H')) + return MockTarget(self.dh.strftime('TaskA/%Y-%m-%d/%H')) class TaskB(luigi.Task): @@ -100,14 +100,14 @@ class TaskB(luigi.Task): complicator = luigi.Parameter() def output(self): - return MockFile(self.dh.strftime('TaskB/%%s%Y-%m-%d/%H') % self.complicator) + return MockTarget(self.dh.strftime('TaskB/%%s%Y-%m-%d/%H') % self.complicator) class TaskC(luigi.Task): dh = luigi.DateHourParameter() def output(self): - return MockFile(self.dh.strftime('not/a/real/path/%Y-%m-%d/%H')) + return MockTarget(self.dh.strftime('not/a/real/path/%Y-%m-%d/%H')) class CommonWrapperTask(luigi.WrapperTask): @@ -502,11 +502,11 @@ class InconsistentlyOutputtingDateHourTask(luigi.Task): def output(self): base = self.dh.strftime('/even/%Y%m%d%H') if self.dh.hour % 2 == 0: - return MockFile(base) + return MockTarget(base) else: return { - 'spi': MockFile(base + '/something.spi'), - 'spl': MockFile(base + '/something.spl'), + 'spi': MockTarget(base + '/something.spi'), + 'spl': MockTarget(base + '/something.spl'), } def test_raise_not_implemented(): @@ -571,7 +571,7 @@ class SomeDailyTask(luigi.Task): d = luigi.DateParameter() def output(self): - return MockFile(self.d.strftime('/data/2014/p/v/z/%Y_/_%m-_-%doctor/20/ZOOO')) + return MockTarget(self.d.strftime('/data/2014/p/v/z/%Y_/_%m-_-%doctor/20/ZOOO')) task = RangeDaily(now=datetime_to_epoch(datetime.datetime(2016, 4, 1)), of='SomeDailyTask', @@ -594,7 +594,7 @@ class RangeHourlyTest(unittest.TestCase): new=mock_exists_always_true) def test_missing_tasks_correctly_required(self): for task_path in task_a_paths: - MockFile(task_path) + MockTarget(task_path) task = RangeHourly(now=datetime_to_epoch(datetime.datetime(2016, 4, 1)), of='TaskA', start=datetime.datetime(2014, 3, 20, 17), diff --git a/test/recursion_test.py b/test/recursion_test.py index a454059120..729021140e 100644 --- a/test/recursion_test.py +++ b/test/recursion_test.py @@ -22,16 +22,14 @@ import luigi import luigi.interface -from luigi.mock import MockFile - -LocalTarget = MockFile +from luigi.mock import MockTarget class Popularity(luigi.Task): date = luigi.DateParameter(default=datetime.date.today() - datetime.timedelta(1)) def output(self): - return LocalTarget('/tmp/popularity/%s.txt' % self.date.strftime('%Y-%m-%d')) + return MockTarget('/tmp/popularity/%s.txt' % self.date.strftime('%Y-%m-%d')) def requires(self): return Popularity(self.date - datetime.timedelta(1)) @@ -47,7 +45,7 @@ def run(self): class RecursionTest(unittest.TestCase): def setUp(self): - MockFile.fs.get_all_data()['/tmp/popularity/2009-01-01.txt'] = b'0\n' + MockTarget.fs.get_all_data()['/tmp/popularity/2009-01-01.txt'] = b'0\n' def test_invoke(self): w = luigi.worker.Worker() @@ -55,4 +53,4 @@ def test_invoke(self): w.run() w.stop() - self.assertEqual(MockFile.fs.get_data('/tmp/popularity/2010-01-01.txt'), b'365\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/popularity/2010-01-01.txt'), b'365\n') diff --git a/test/spark_test.py b/test/spark_test.py index 40b57474dd..ad56696f29 100644 --- a/test/spark_test.py +++ b/test/spark_test.py @@ -26,7 +26,7 @@ import luigi.hdfs from helpers import with_config from luigi.contrib.spark import PySpark1xJob, Spark1xJob, SparkJob, SparkJobError -from luigi.mock import MockFile +from luigi.mock import MockTarget from mock import patch @@ -89,9 +89,9 @@ def communicate(self): return p h, p = luigi.hdfs.HdfsTarget, subprocess.Popen - luigi.hdfs.HdfsTarget, subprocess.Popen = MockFile, Popen_fake + luigi.hdfs.HdfsTarget, subprocess.Popen = MockTarget, Popen_fake try: - MockFile.move = lambda *args, **kwargs: None + MockTarget.move = lambda *args, **kwargs: None job = TestJob() job.run() self.assertEqual(len(arglist_result), 1) @@ -195,9 +195,9 @@ def communicate(self): return p h, p = luigi.hdfs.HdfsTarget, subprocess.Popen - luigi.hdfs.HdfsTarget, subprocess.Popen = MockFile, Popen_fake + luigi.hdfs.HdfsTarget, subprocess.Popen = MockTarget, Popen_fake try: - MockFile.move = lambda *args, **kwargs: None + MockTarget.move = lambda *args, **kwargs: None job = Test1xJob() job.run() self.assertEqual(len(arglist_result), 1) @@ -300,9 +300,9 @@ def communicate(self): return p h, p = luigi.hdfs.HdfsTarget, subprocess.Popen - luigi.hdfs.HdfsTarget, subprocess.Popen = MockFile, Popen_fake + luigi.hdfs.HdfsTarget, subprocess.Popen = MockTarget, Popen_fake try: - MockFile.move = lambda *args, **kwargs: None + MockTarget.move = lambda *args, **kwargs: None job = TestPySpark1xJob() job.run() self.assertEqual(len(arglist_result), 1) diff --git a/test/test_event_callbacks.py b/test/test_event_callbacks.py index ea9e7919c0..07248c77d3 100644 --- a/test/test_event_callbacks.py +++ b/test/test_event_callbacks.py @@ -20,7 +20,7 @@ import luigi from luigi import Event, Task, build -from luigi.mock import MockFile, MockFileSystem +from luigi.mock import MockTarget, MockFileSystem from luigi.task import flatten from mock import patch @@ -131,7 +131,7 @@ class ConsistentMockOutput(object): param = luigi.IntParameter(default=1) def output(self): - return MockFile('/%s/%u' % (self.__class__.__name__, self.param)) + return MockTarget('/%s/%u' % (self.__class__.__name__, self.param)) def produce_output(self): with self.output().open('w') as o: diff --git a/test/worker_test.py b/test/worker_test.py index 14060aeef6..246133cfdb 100644 --- a/test/worker_test.py +++ b/test/worker_test.py @@ -31,7 +31,7 @@ import mock from helpers import with_config from luigi import ExternalTask, RemoteScheduler, Task -from luigi.mock import MockFile, MockFileSystem +from luigi.mock import MockTarget, MockFileSystem from luigi.scheduler import CentralPlannerScheduler from luigi.worker import Worker from luigi import six @@ -838,7 +838,7 @@ class Dummy2Task(Task): p = luigi.Parameter() def output(self): - return MockFile(self.p) + return MockTarget(self.p) def run(self): f = self.output().open('w') @@ -897,7 +897,7 @@ class ForkBombTask(luigi.Task): p = luigi.Parameter(default=(0, )) # ehm for some weird reason [0] becomes a tuple...? def output(self): - return MockFile('.'.join(map(str, self.p))) + return MockTarget('.'.join(map(str, self.p))) def run(self): with self.output().open('w') as f: diff --git a/test/wrap_test.py b/test/wrap_test.py index a6c0247bda..f861796ec4 100644 --- a/test/wrap_test.py +++ b/test/wrap_test.py @@ -21,17 +21,16 @@ import luigi import luigi.notifications -from luigi.mock import MockFile +from luigi.mock import MockTarget from luigi.util import inherits luigi.notifications.DEBUG = True -LocalTarget = MockFile class A(luigi.Task): def output(self): - return LocalTarget('/tmp/a.txt') + return MockTarget('/tmp/a.txt') def run(self): f = self.output().open('w') @@ -43,7 +42,7 @@ class B(luigi.Task): date = luigi.DateParameter() def output(self): - return LocalTarget(self.date.strftime('/tmp/b-%Y-%m-%d.txt')) + return MockTarget(self.date.strftime('/tmp/b-%Y-%m-%d.txt')) def run(self): f = self.output().open('w') @@ -72,13 +71,13 @@ def run(self): class AXML(XMLWrapper(A)): def output(self): - return LocalTarget('/tmp/a.xml') + return MockTarget('/tmp/a.xml') class BXML(XMLWrapper(B)): def output(self): - return LocalTarget(self.date.strftime('/tmp/b-%Y-%m-%d.xml')) + return MockTarget(self.date.strftime('/tmp/b-%Y-%m-%d.xml')) class WrapperTest(unittest.TestCase): @@ -89,15 +88,15 @@ class WrapperTest(unittest.TestCase): workers = 1 def setUp(self): - MockFile.fs.clear() + MockTarget.fs.clear() def test_a(self): luigi.build([AXML()], local_scheduler=True, no_lock=True, workers=self.workers) - self.assertEqual(MockFile.fs.get_data('/tmp/a.xml'), b'\nhello, world\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/a.xml'), b'\nhello, world\n') def test_b(self): luigi.build([BXML(datetime.date(2012, 1, 1))], local_scheduler=True, no_lock=True, workers=self.workers) - self.assertEqual(MockFile.fs.get_data('/tmp/b-2012-01-01.xml'), b'\ngoodbye, space\n') + self.assertEqual(MockTarget.fs.get_data('/tmp/b-2012-01-01.xml'), b'\ngoodbye, space\n') class WrapperWithMultipleWorkersTest(WrapperTest):