spark-instrumented-optimizer/python/pyspark/context.py

531 lines
22 KiB
Python
Raw Normal View History

#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
2012-08-10 04:10:02 -04:00
import os
import shutil
import sys
from threading import Lock
2012-08-10 04:10:02 -04:00
from tempfile import NamedTemporaryFile
from collections import namedtuple
2012-08-10 04:10:02 -04:00
2013-01-20 04:57:44 -05:00
from pyspark import accumulators
from pyspark.accumulators import Accumulator
2012-08-25 16:59:01 -04:00
from pyspark.broadcast import Broadcast
2013-12-29 14:03:39 -05:00
from pyspark.conf import SparkConf
from pyspark.files import SparkFiles
2012-08-10 04:10:02 -04:00
from pyspark.java_gateway import launch_gateway
from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \
PairDeserializer
2013-09-07 17:41:31 -04:00
from pyspark.storagelevel import StorageLevel
from pyspark import rdd
from pyspark.rdd import RDD
2012-08-10 04:10:02 -04:00
from py4j.java_collections import ListConverter
2012-08-10 04:10:02 -04:00
class SparkContext(object):
"""
Main entry point for Spark functionality. A SparkContext represents the
connection to a Spark cluster, and can be used to create L{RDD}s and
broadcast variables on that cluster.
"""
2012-08-10 04:10:02 -04:00
_gateway = None
_jvm = None
_writeToFile = None
2013-01-20 04:57:44 -05:00
_next_accum_id = 0
_active_spark_context = None
_lock = Lock()
_python_includes = None # zip and egg files that need to be added to PYTHONPATH
2012-08-10 04:10:02 -04:00
2013-12-29 14:03:39 -05:00
def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None,
gateway=None):
"""
2013-12-29 14:46:59 -05:00
Create a new SparkContext. At least the master and app name should be set,
either through the named parameters here or through C{conf}.
@param master: Cluster URL to connect to
(e.g. mesos://host:port, spark://host:port, local[4]).
2013-12-29 14:03:39 -05:00
@param appName: A name for your job, to display on the cluster web UI.
@param sparkHome: Location where Spark is installed on cluster nodes.
@param pyFiles: Collection of .zip or .py files to send to the cluster
and add to PYTHONPATH. These can be paths on the local file
system or HDFS, HTTP, HTTPS, or FTP URLs.
@param environment: A dictionary of environment variables to set on
worker nodes.
@param batchSize: The number of Python objects represented as a single
Java object. Set 1 to disable batching or -1 to use an
unlimited batch size.
@param serializer: The serializer for RDDs.
2013-12-29 14:03:39 -05:00
@param conf: A L{SparkConf} object setting Spark properties.
@param gateway: Use an existing gateway and JVM, otherwise a new JVM
will be instatiated.
>>> from pyspark.context import SparkContext
>>> sc = SparkContext('local', 'test')
>>> sc2 = SparkContext('local', 'test2') # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
ValueError:...
"""
if rdd._extract_concise_traceback() is not None:
self._callsite = rdd._extract_concise_traceback()
else:
tempNamedTuple = namedtuple("Callsite", "function file linenum")
self._callsite = tempNamedTuple(function=None, file=None, linenum=None)
SparkContext._ensure_initialized(self, gateway=gateway)
self.environment = environment or {}
self._conf = conf or SparkConf(_jvm=self._jvm)
self._batchSize = batchSize # -1 represents an unlimited batch size
self._unbatched_serializer = serializer
if batchSize == 1:
self.serializer = self._unbatched_serializer
else:
self.serializer = BatchedSerializer(self._unbatched_serializer,
batchSize)
# Set any parameters passed directly to us on the conf
if master:
self._conf.setMaster(master)
if appName:
self._conf.setAppName(appName)
if sparkHome:
self._conf.setSparkHome(sparkHome)
if environment:
for key, value in environment.iteritems():
self._conf.setExecutorEnv(key, value)
2013-12-29 14:03:39 -05:00
# Check that we have at least the required parameters
if not self._conf.contains("spark.master"):
2013-12-29 14:03:39 -05:00
raise Exception("A master URL must be set in your configuration")
if not self._conf.contains("spark.app.name"):
2013-12-29 14:03:39 -05:00
raise Exception("An application name must be set in your configuration")
# Read back our properties from the conf in case we loaded some of them from
# the classpath or an external config file
self.master = self._conf.get("spark.master")
self.appName = self._conf.get("spark.app.name")
self.sparkHome = self._conf.get("spark.home", None)
for (k, v) in self._conf.getAll():
if k.startswith("spark.executorEnv."):
varName = k[len("spark.executorEnv."):]
self.environment[varName] = v
2013-12-29 14:03:39 -05:00
# Create the Java SparkContext through Py4J
self._jsc = self._initialize_context(self._conf._jconf)
2013-01-20 04:57:44 -05:00
# Create a single Accumulator in Java that we'll send all our updates through;
# they will be passed back to us through a TCP server
self._accumulatorServer = accumulators._start_update_server()
(host, port) = self._accumulatorServer.server_address
self._javaAccumulator = self._jsc.accumulator(
self._jvm.java.util.ArrayList(),
self._jvm.PythonAccumulatorParam(host, port))
2013-01-20 04:57:44 -05:00
self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
2013-12-29 14:03:39 -05:00
2012-08-25 16:59:01 -04:00
# Broadcast's __reduce__ method stores Broadcast instances here.
# This allows other code to determine which Broadcast instances have
# been pickled, so it can determine which Java broadcast objects to
# send.
self._pickled_broadcast_vars = set()
2012-08-10 04:10:02 -04:00
SparkFiles._sc = self
root_dir = SparkFiles.getRootDirectory()
sys.path.append(root_dir)
# Deploy any code dependencies specified in the constructor
self._python_includes = list()
for path in (pyFiles or []):
self.addPyFile(path)
[SPARK-1549] Add Python support to spark-submit This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN. This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging. In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit. In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0. Author: Matei Zaharia <matei@databricks.com> Closes #664 from mateiz/py-submit and squashes the following commits: 15e9669 [Matei Zaharia] Fix some uses of path.separator property 051278c [Matei Zaharia] Small style fixes 0afe886 [Matei Zaharia] Add license headers 4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests 15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside 47c0655 [Matei Zaharia] More work to make spark-submit work with Python: d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 18:12:35 -04:00
# Deploy code dependencies set by spark-submit; these will already have been added
# with SparkContext.addFile, so we just need to add them
for path in self._conf.get("spark.submit.pyFiles", "").split(","):
if path != "":
self._python_includes.append(os.path.basename(path))
# Create a temporary directory inside spark.local.dir:
2013-12-29 00:11:36 -05:00
local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
self._temp_dir = \
self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
# Initialize SparkContext in function to allow subclass specific initialization
def _initialize_context(self, jconf):
return self._jvm.JavaSparkContext(jconf)
@classmethod
def _ensure_initialized(cls, instance=None, gateway=None):
with SparkContext._lock:
if not SparkContext._gateway:
SparkContext._gateway = gateway or launch_gateway()
SparkContext._jvm = SparkContext._gateway.jvm
SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile
if instance:
if SparkContext._active_spark_context and SparkContext._active_spark_context != instance:
currentMaster = SparkContext._active_spark_context.master
currentAppName = SparkContext._active_spark_context.appName
callsite = SparkContext._active_spark_context._callsite
# Raise error if there is already a running Spark context
raise ValueError("Cannot run multiple SparkContexts at once; existing SparkContext(app=%s, master=%s)" \
" created by %s at %s:%s " \
% (currentAppName, currentMaster, callsite.function, callsite.file, callsite.linenum))
else:
SparkContext._active_spark_context = instance
@classmethod
def setSystemProperty(cls, key, value):
"""
2013-12-29 14:03:39 -05:00
Set a Java system property, such as spark.executor.memory. This must
must be invoked before instantiating SparkContext.
"""
SparkContext._ensure_initialized()
SparkContext._jvm.java.lang.System.setProperty(key, value)
@property
def defaultParallelism(self):
"""
Default level of parallelism to use when not given by user (e.g. for
reduce tasks)
"""
return self._jsc.sc().defaultParallelism()
2012-08-10 04:10:02 -04:00
def __del__(self):
self.stop()
2012-08-10 04:10:02 -04:00
def stop(self):
"""
Shut down the SparkContext.
"""
if self._jsc:
self._jsc.stop()
self._jsc = None
if self._accumulatorServer:
self._accumulatorServer.shutdown()
self._accumulatorServer = None
with SparkContext._lock:
SparkContext._active_spark_context = None
2012-08-10 04:10:02 -04:00
def parallelize(self, c, numSlices=None):
"""
Distribute a local Python collection to form an RDD.
>>> sc.parallelize(range(5), 5).glom().collect()
[[0], [1], [2], [3], [4]]
"""
2012-08-10 04:10:02 -04:00
numSlices = numSlices or self.defaultParallelism
# Calling the Java parallelize() method with an ArrayList is too slow,
# because it sends O(n) Py4J commands. As an alternative, serialized
# objects are written to a file and loaded through textFile().
tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir)
# Make sure we distribute data evenly if it's smaller than self.batchSize
if "__len__" not in dir(c):
c = list(c) # Make it a list so we can compute its length
batchSize = min(len(c) // numSlices, self._batchSize)
if batchSize > 1:
serializer = BatchedSerializer(self._unbatched_serializer,
batchSize)
else:
serializer = self._unbatched_serializer
serializer.dump_stream(c, tempFile)
2012-08-10 04:10:02 -04:00
tempFile.close()
readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile
jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices)
return RDD(jrdd, self, serializer)
2012-08-10 04:10:02 -04:00
def textFile(self, name, minPartitions=None):
"""
Read a text file from HDFS, a local file system (available on all
nodes), or any Hadoop-supported file system URI, and return it as an
RDD of Strings.
"""
minPartitions = minPartitions or min(self.defaultParallelism, 2)
return RDD(self._jsc.textFile(name, minPartitions), self,
UTF8Deserializer())
2012-08-25 16:59:01 -04:00
def wholeTextFiles(self, path):
"""
Read a directory of text files from HDFS, a local file system
(available on all nodes), or any Hadoop-supported file system
URI. Each file is read as a single record and returned in a
key-value pair, where the key is the path of each file, the
value is the content of each file.
For example, if you have the following files::
hdfs://a-hdfs-path/part-00000
hdfs://a-hdfs-path/part-00001
...
hdfs://a-hdfs-path/part-nnnnn
Do C{rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")},
then C{rdd} contains::
(a-hdfs-path/part-00000, its content)
(a-hdfs-path/part-00001, its content)
...
(a-hdfs-path/part-nnnnn, its content)
NOTE: Small files are preferred, as each file will be loaded
fully in memory.
>>> dirPath = os.path.join(tempdir, "files")
>>> os.mkdir(dirPath)
>>> with open(os.path.join(dirPath, "1.txt"), "w") as file1:
... file1.write("1")
>>> with open(os.path.join(dirPath, "2.txt"), "w") as file2:
... file2.write("2")
>>> textFiles = sc.wholeTextFiles(dirPath)
>>> sorted(textFiles.collect())
[(u'.../1.txt', u'1'), (u'.../2.txt', u'2')]
"""
return RDD(self._jsc.wholeTextFiles(path), self,
PairDeserializer(UTF8Deserializer(), UTF8Deserializer()))
def _checkpointFile(self, name, input_deserializer):
jrdd = self._jsc.checkpointFile(name)
return RDD(jrdd, self, input_deserializer)
def union(self, rdds):
"""
Build the union of a list of RDDs.
This supports unions() of RDDs with different serialized formats,
although this forces them to be reserialized using the default
serializer:
>>> path = os.path.join(tempdir, "union-text.txt")
>>> with open(path, "w") as testFile:
... testFile.write("Hello")
>>> textFile = sc.textFile(path)
>>> textFile.collect()
[u'Hello']
>>> parallelized = sc.parallelize(["World!"])
>>> sorted(sc.union([textFile, parallelized]).collect())
[u'Hello', 'World!']
"""
first_jrdd_deserializer = rdds[0]._jrdd_deserializer
if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds):
rdds = [x._reserialize() for x in rdds]
first = rdds[0]._jrdd
rest = [x._jrdd for x in rdds[1:]]
rest = ListConverter().convert(rest, self._gateway._gateway_client)
return RDD(self._jsc.union(first, rest), self,
rdds[0]._jrdd_deserializer)
2012-08-25 16:59:01 -04:00
def broadcast(self, value):
"""
Broadcast a read-only variable to the cluster, returning a
L{Broadcast<pyspark.broadcast.Broadcast>}
object for reading it in distributed functions. The variable will be
sent to each cluster only once.
"""
pickleSer = PickleSerializer()
pickled = pickleSer.dumps(value)
jbroadcast = self._jsc.broadcast(bytearray(pickled))
return Broadcast(jbroadcast.id(), value, jbroadcast,
2012-08-25 16:59:01 -04:00
self._pickled_broadcast_vars)
2013-01-20 04:57:44 -05:00
def accumulator(self, value, accum_param=None):
"""
Create an L{Accumulator} with the given initial value, using a given
L{AccumulatorParam} helper object to define how to add values of the
data type if provided. Default AccumulatorParams are used for integers
and floating-point numbers if you do not provide one. For other types,
a custom AccumulatorParam can be used.
2013-01-20 04:57:44 -05:00
"""
if accum_param is None:
2013-01-20 04:57:44 -05:00
if isinstance(value, int):
accum_param = accumulators.INT_ACCUMULATOR_PARAM
elif isinstance(value, float):
accum_param = accumulators.FLOAT_ACCUMULATOR_PARAM
elif isinstance(value, complex):
accum_param = accumulators.COMPLEX_ACCUMULATOR_PARAM
else:
raise Exception("No default accumulator param for type %s" % type(value))
SparkContext._next_accum_id += 1
return Accumulator(SparkContext._next_accum_id - 1, value, accum_param)
def addFile(self, path):
"""
Add a file to be downloaded with this Spark job on every node.
The C{path} passed can be either a local file, a file in HDFS
(or other Hadoop-supported filesystems), or an HTTP, HTTPS or
FTP URI.
To access the file in Spark jobs, use
L{SparkFiles.get(path)<pyspark.files.SparkFiles.get>} to find its
download location.
>>> from pyspark import SparkFiles
>>> path = os.path.join(tempdir, "test.txt")
>>> with open(path, "w") as testFile:
... testFile.write("100")
>>> sc.addFile(path)
>>> def func(iterator):
... with open(SparkFiles.get("test.txt")) as testFile:
... fileVal = int(testFile.readline())
... return [x * 100 for x in iterator]
>>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect()
[100, 200, 300, 400]
"""
self._jsc.sc().addFile(path)
def clearFiles(self):
"""
Clear the job's list of files added by L{addFile} or L{addPyFile} so
that they do not get downloaded to any new nodes.
"""
# TODO: remove added .py or .zip files from the PYTHONPATH?
self._jsc.sc().clearFiles()
def addPyFile(self, path):
"""
Add a .py or .zip dependency for all tasks to be executed on this
SparkContext in the future. The C{path} passed can be either a local
file, a file in HDFS (or other Hadoop-supported filesystems), or an
HTTP, HTTPS or FTP URI.
"""
self.addFile(path)
(dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix
if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'):
self._python_includes.append(filename)
sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) # for tests in local mode
2013-01-16 22:15:14 -05:00
def setCheckpointDir(self, dirName):
2013-01-16 22:15:14 -05:00
"""
Set the directory under which RDDs are going to be checkpointed. The
directory must be a HDFS path if running on a cluster.
2013-01-16 22:15:14 -05:00
"""
self._jsc.sc().setCheckpointDir(dirName)
2013-09-07 17:41:31 -04:00
def _getJavaStorageLevel(self, storageLevel):
"""
Returns a Java StorageLevel based on a pyspark.StorageLevel.
"""
if not isinstance(storageLevel, StorageLevel):
raise Exception("storageLevel must be of type pyspark.StorageLevel")
newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
return newStorageLevel(storageLevel.useDisk,
storageLevel.useMemory,
storageLevel.useOffHeap,
storageLevel.deserialized,
storageLevel.replication)
2013-09-07 17:41:31 -04:00
def setJobGroup(self, groupId, description, interruptOnCancel=False):
"""
Assigns a group ID to all the jobs started by this thread until the group ID is set to a
different value or cleared.
Often, a unit of execution in an application consists of multiple Spark actions or jobs.
Application programmers can use this method to group all those jobs together and give a
group description. Once set, the Spark web UI will associate such jobs with this group.
The application can use L{SparkContext.cancelJobGroup} to cancel all
running jobs in this group.
>>> import thread, threading
>>> from time import sleep
>>> result = "Not Set"
>>> lock = threading.Lock()
>>> def map_func(x):
... sleep(100)
SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions This patch includes several cleanups to PythonRDD, focused around fixing [SPARK-1579](https://issues.apache.org/jira/browse/SPARK-1579) cleanly. Listed in order of approximate importance: - The Python daemon waits for Spark to close the socket before exiting, in order to avoid causing spurious IOExceptions in Spark's `PythonRDD::WriterThread`. - Removes the Python Monitor Thread, which polled for task cancellations in order to kill the Python worker. Instead, we do this in the onCompleteCallback, since this is guaranteed to be called during cancellation. - Adds a "completed" variable to TaskContext to avoid the issue noted in [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), where onCompleteCallbacks may be execution-order dependent. Along with this, I removed the "context.interrupted = true" flag in the onCompleteCallback. - Extracts PythonRDD::WriterThread to its own class. Since this patch provides an alternative solution to [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), I did test it with ``` sc.textFile("latlon.tsv").take(5) ``` many times without error. Additionally, in order to test the unswallowed exceptions, I performed ``` sc.textFile("s3n://<big file>").count() ``` and cut my internet during execution. Prior to this patch, we got the "stdin writer exited early" message, which was unhelpful. Now, we get the SocketExceptions propagated through Spark to the user and get proper (though unsuccessful) task retries. Author: Aaron Davidson <aaron@databricks.com> Closes #640 from aarondav/pyspark-io and squashes the following commits: b391ff8 [Aaron Davidson] Detect "clean socket shutdowns" and stop waiting on the socket c0c49da [Aaron Davidson] SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions
2014-05-07 12:48:31 -04:00
... raise Exception("Task should have been cancelled")
>>> def start_job(x):
... global result
... try:
... sc.setJobGroup("job_to_cancel", "some description")
... result = sc.parallelize(range(x)).map(map_func).collect()
... except Exception as e:
... result = "Cancelled"
... lock.release()
>>> def stop_job():
... sleep(5)
... sc.cancelJobGroup("job_to_cancel")
>>> supress = lock.acquire()
>>> supress = thread.start_new_thread(start_job, (10,))
>>> supress = thread.start_new_thread(stop_job, tuple())
>>> supress = lock.acquire()
>>> print result
Cancelled
If interruptOnCancel is set to true for the job group, then job cancellation will result
in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure
that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208,
where HDFS may respond to Thread.interrupt() by marking nodes as dead.
"""
self._jsc.setJobGroup(groupId, description, interruptOnCancel)
def setLocalProperty(self, key, value):
"""
Set a local property that affects jobs submitted from this thread, such as the
Spark fair scheduler pool.
"""
self._jsc.setLocalProperty(key, value)
def getLocalProperty(self, key):
"""
Get a local property set in this thread, or null if it is missing. See
L{setLocalProperty}
"""
return self._jsc.getLocalProperty(key)
def sparkUser(self):
"""
Get SPARK_USER for user who is running SparkContext.
"""
return self._jsc.sc().sparkUser()
def cancelJobGroup(self, groupId):
"""
Cancel active jobs for the specified group. See L{SparkContext.setJobGroup}
for more information.
"""
self._jsc.sc().cancelJobGroup(groupId)
def cancelAllJobs(self):
"""
Cancel all jobs that have been scheduled or are running.
"""
self._jsc.sc().cancelAllJobs()
def _test():
import atexit
import doctest
import tempfile
globs = globals().copy()
globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
globs['tempdir'] = tempfile.mkdtemp()
atexit.register(lambda: shutil.rmtree(globs['tempdir']))
(failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
globs['sc'].stop()
if failure_count:
exit(-1)
if __name__ == "__main__":
_test()