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

Support cluster mode in PySpark #2197

Merged
merged 3 commits into from
Aug 9, 2017
Merged
Show file tree
Hide file tree
Changes from 2 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
16 changes: 10 additions & 6 deletions luigi/contrib/spark.py
Original file line number Diff line number Diff line change
Expand Up @@ -237,8 +237,6 @@ class PySparkTask(SparkSubmitTask):

# Path to the pyspark program passed to spark-submit
app = os.path.join(os.path.dirname(__file__), 'pyspark_runner.py')
# Python only supports the client deploy mode, force it
deploy_mode = "client"
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Was this an intentional deletion? Why not just allow overwrite of deploy_mode?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

deploy_mode = "client" overwrites deploy_mode in SparkSubmitTask. It was here to force deploy mode to be client since previously cluster deploy mode wasn't supported. Now that we do support it, there is no need to pin it to client only.


@property
def name(self):
Expand All @@ -250,6 +248,11 @@ def py_packages(self):
if packages:
return map(lambda s: s.strip(), packages.split(','))

@property
def files(self):
if self.deploy_mode == "cluster":
return [self.run_pickle]

def setup(self, conf):
"""
Called by the pyspark_runner with a SparkConf instance that will be used to instantiate the SparkContext
Expand All @@ -269,11 +272,12 @@ def main(self, sc, *args):
"""
raise NotImplementedError("subclass should define a main method")

def program_args(self):
return self.spark_command() + self.app_command()

def app_command(self):
return [self.app, self.run_pickle] + self.app_options()
if self.deploy_mode == "cluster":
pickle_loc = os.path.basename(self.run_pickle)
else:
pickle_loc = self.run_pickle
return [self.app, pickle_loc] + self.app_options()

def run(self):
self.run_path = tempfile.mkdtemp(prefix=self.name)
Expand Down
15 changes: 13 additions & 2 deletions test/contrib/spark_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ def interrupt():
class PySparkTaskTest(unittest.TestCase):
ss = 'ss-stub'

@with_config({'spark': {'spark-submit': ss, 'master': "spark://host:7077"}})
@with_config({'spark': {'spark-submit': ss, 'master': "spark://host:7077", 'deploy-mode': 'client'}})
@patch('luigi.contrib.external_program.subprocess.Popen')
def test_run(self, proc):
setup_run_process(proc)
Expand All @@ -199,7 +199,7 @@ def test_run(self, proc):
self.assertTrue(os.path.exists(proc_arg_list[7]))
self.assertTrue(proc_arg_list[8].endswith('TestPySparkTask.pickle'))

@with_config({'spark': {'spark-submit': ss, 'master': "spark://host:7077"}})
@with_config({'spark': {'spark-submit': ss, 'master': "spark://host:7077", 'deploy-mode': 'client'}})
@patch('luigi.contrib.external_program.subprocess.Popen')
def test_run_with_pickle_dump(self, proc):
setup_run_process(proc)
Expand All @@ -211,6 +211,17 @@ def test_run_with_pickle_dump(self, proc):
self.assertTrue(os.path.exists(proc_arg_list[7]))
self.assertTrue(proc_arg_list[8].endswith('TestPySparkTask.pickle'))

@with_config({'spark': {'spark-submit': ss, 'master': "spark://host:7077", 'deploy-mode': 'cluster'}})
@patch('luigi.contrib.external_program.subprocess.Popen')
def test_run_with_cluster(self, proc):
setup_run_process(proc)
job = TestPySparkTask()
job.run()
proc_arg_list = proc.call_args[0][0]
self.assertEqual(proc_arg_list[0:7], ['ss-stub', '--master', 'spark://host:7077', '--deploy-mode', 'cluster', '--name', 'TestPySparkTask'])
self.assertTrue(os.path.exists(proc_arg_list[7]))
self.assertTrue(proc_arg_list[8].endswith('TestPySparkTask.pickle'))

@patch.dict('sys.modules', {'pyspark': MagicMock()})
@patch('pyspark.SparkContext')
def test_pyspark_runner(self, spark_context):
Expand Down