Skip to content

Commit

Permalink
Merge pull request #834 from spotify/erikbern/mock-target
Browse files Browse the repository at this point in the history
Renamed MockFile -> MockTarget
  • Loading branch information
Erik Bernhardsson committed Mar 9, 2015
2 parents ffc5c63 + d830e78 commit 10064c8
Show file tree
Hide file tree
Showing 16 changed files with 95 additions and 92 deletions.
15 changes: 11 additions & 4 deletions luigi/mock.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,16 @@
# 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.
"""

import multiprocessing
from io import BytesIO

import sys
import warnings

from luigi import six
import luigi.util
Expand All @@ -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):
"""
Expand Down Expand Up @@ -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):
Expand Down Expand Up @@ -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)
12 changes: 5 additions & 7 deletions test/cmdline_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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')
Expand Down Expand Up @@ -99,19 +99,17 @@ class FooSubClass(FooBaseClass):
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):
Expand Down
14 changes: 7 additions & 7 deletions test/decorator_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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')
Expand All @@ -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):
Expand All @@ -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):
Expand Down
22 changes: 9 additions & 13 deletions test/fib_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,7 @@

import luigi
import luigi.interface
from luigi.mock import MockFile

LocalTarget = MockFile
from luigi.mock import MockTarget

# Calculates Fibonacci numbers :)

Expand All @@ -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:
Expand All @@ -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):
Expand All @@ -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()
2 changes: 1 addition & 1 deletion test/file_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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))
7 changes: 3 additions & 4 deletions test/hadoop_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand All @@ -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):
Expand Down Expand Up @@ -236,7 +235,7 @@ def test_instantiate_job(self):
MyStreamingJob('param_value')

def setUp(self):
MockFile.fs.clear()
MockTarget.fs.clear()


@attr('minicluster')
Expand Down
5 changes: 2 additions & 3 deletions test/instance_wrap_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand All @@ -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):
Expand Down
17 changes: 12 additions & 5 deletions test/mock_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand All @@ -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")

Expand All @@ -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")

Expand All @@ -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

Expand All @@ -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))
10 changes: 5 additions & 5 deletions test/optparse_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,16 @@

import luigi
from fib_test import FibTestBase
from luigi.mock import MockFile
from luigi.mock import MockTarget


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
Expand All @@ -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')
10 changes: 5 additions & 5 deletions test/parameter_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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()
Expand All @@ -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()
Expand Down Expand Up @@ -310,11 +310,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'])
Expand Down
Loading

0 comments on commit 10064c8

Please sign in to comment.