Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

PySparkTask fix for bytes / str type error and import error #2168

Merged
merged 4 commits into from
Jul 13, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions luigi/contrib/pyspark_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,15 @@
import pickle
import logging
import sys
import os


class PySparkRunner(object):

def __init__(self, job, *args):
# Append job directory to PYTHON_PATH to enable dynamic import
# of the module in which the class resides on unpickling
sys.path.append(os.path.dirname(job))
with open(job, "rb") as fd:
self.job = pickle.load(fd)
self.args = args
Expand Down
6 changes: 5 additions & 1 deletion luigi/contrib/spark.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import shutil
import importlib
import tarfile
import inspect
try:
import cPickle as pickle
except ImportError:
Expand Down Expand Up @@ -278,6 +279,9 @@ def run(self):
self.run_path = tempfile.mkdtemp(prefix=self.name)
self.run_pickle = os.path.join(self.run_path, '.'.join([self.name.replace(' ', '_'), 'pickle']))
with open(self.run_pickle, 'wb') as fd:
# Copy module file to run path.
module_path = os.path.abspath(inspect.getfile(self.__class__))
shutil.copy(module_path, os.path.join(self.run_path, '.'))
self._dump(fd)
try:
super(PySparkTask, self).run()
Expand All @@ -289,7 +293,7 @@ def _dump(self, fd):
if self.__module__ == '__main__':
d = pickle.dumps(self)
module_name = os.path.basename(sys.argv[0]).rsplit('.', 1)[0]
d = d.replace(b'(c__main__', "(c" + module_name)
d = d.replace(b'c__main__', b'c' + module_name.encode('ascii'))
fd.write(d)
else:
pickle.dump(self, fd)
Expand Down
10 changes: 10 additions & 0 deletions test/contrib/spark_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

import unittest
import os
import sys
import pickle
import luigi
import luigi.contrib.hdfs
from luigi import six
Expand Down Expand Up @@ -219,6 +221,14 @@ def mock_spark_submit(task):
PySparkRunner(*task.app_command()[1:]).run()
# Check py-package exists
self.assertTrue(os.path.exists(sc.addPyFile.call_args[0][0]))
# Check that main module containing the task exists.
run_path = os.path.dirname(task.app_command()[1])
self.assertTrue(os.path.exists(os.path.join(run_path, os.path.basename(__file__))))
# Check that the python path contains the run_path
self.assertTrue(run_path in sys.path)
# Check if find_class finds the class for the correct module name.
with open(task.app_command()[1], 'rb') as fp:
self.assertTrue(pickle.Unpickler(fp).find_class('spark_test', 'TestPySparkTask'))

with patch.object(SparkSubmitTask, 'run', mock_spark_submit):
job = TestPySparkTask()
Expand Down