Hot-keys on this page
r m x p toggle line displays
j k next/prev highlighted chunk
0 (zero) top of page
1 (one) first highlighted chunk
# # 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. #
PairDeserializer, AutoBatchedSerializer, NoOpSerializer, ChunkedStream
# These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. "spark.serializer.objectStreamReset": 100, "spark.rdd.compress": True, }
""" Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create :class:`RDD` and broadcast variables on that cluster.
When you create a new SparkContext, at least the master and app name should be set, either through the named parameters here or through `conf`.
Parameters ---------- master : str, optional Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). appName : str, optional A name for your job, to display on the cluster web UI. sparkHome : str, optional Location where Spark is installed on cluster nodes. pyFiles : list, optional 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. environment : dict, optional A dictionary of environment variables to set on worker nodes. batchSize : int, optional The number of Python objects represented as a single Java object. Set 1 to disable batching, 0 to automatically choose the batch size based on object sizes, or -1 to use an unlimited batch size serializer : :class:`pyspark.serializers.Serializer`, optional The serializer for RDDs. conf : :py:class:`pyspark.SparkConf`, optional An object setting Spark properties. gateway : :py:class:`py4j.java_gateway.JavaGateway`, optional Use an existing gateway and JVM, otherwise a new JVM will be instantiated. This is only used internally. jsc : :py:class:`py4j.java_gateway.JavaObject`, optional The JavaSparkContext instance. This is only used internally. profiler_cls : type, optional A class of custom Profiler used to do profiling (default is :class:`pyspark.profiler.BasicProfiler`).
Notes ----- Only one :class:`SparkContext` should be active per JVM. You must `stop()` the active :class:`SparkContext` before creating a new one.
:class:`SparkContext` instance is not supported to share across multiple processes out of the box, and PySpark does not guarantee multi-processing execution. Use threads instead for concurrent processing purpose.
Examples -------- >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') >>> sc2 = SparkContext('local', 'test2') # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... ValueError: ... """
environment=None, batchSize=0, serializer=PickleSerializer(), conf=None, gateway=None, jsc=None, profiler_cls=BasicProfiler): conf.get("spark.executor.allowSparkContext", "false").lower() != "true"): # In order to prevent SparkContext from being created in executors.
"You are trying to pass an insecure Py4j gateway to Spark. This" " is not allowed as it is a security risk.")
conf, jsc, profiler_cls) # If an error occurs, clean up in order to allow future SparkContext creation:
conf, jsc, profiler_cls): # java gateway must have been launched at this point. # conf has been initialized in JVM properly, so use conf directly. This represents the # scenario that JVM has been launched before SparkConf is created (e.g. SparkContext is # created and then stopped, and we create a new SparkConf and new SparkContext again) else:
else: batchSize)
# Set any parameters passed directly to us on the conf self._conf.setSparkHome(sparkHome) for key, value in environment.items(): self._conf.setExecutorEnv(key, value)
# Check that we have at least the required parameters raise RuntimeError("A master URL must be set in your configuration") raise RuntimeError("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
varName = k[len("spark.executorEnv."):] self.environment[varName] = v
# Create the Java SparkContext through Py4J # Reset the SparkConf to the one actually used by the SparkContext in JVM.
# 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
# If encryption is enabled, we need to setup a server in the jvm to read broadcast # data via a socket. # scala's mangled names w/ $ in them require special treatment. str(self._jvm.PythonUtils.getPythonAuthSocketTimeout(self._jsc)) str(self._jvm.PythonUtils.getSparkBufferSize(self._jsc))
"Python 3.6 support is deprecated in Spark 3.2.", FutureWarning )
# 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.
# Deploy any code dependencies specified in the constructor self.addPyFile(path)
# Deploy code dependencies set by spark-submit; these will already have been added # with SparkContext.addFile, so we just need to add them to the PYTHONPATH (dirname, filename) = os.path.split(path) try: filepath = os.path.join(SparkFiles.getRootDirectory(), filename) if not os.path.exists(filepath): # In case of YARN with shell mode, 'spark.submit.pyFiles' files are # not added via SparkContext.addFile. Here we check if the file exists, # try to copy and then add it to the path. See SPARK-21945. shutil.copyfile(path, filepath) if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) sys.path.insert(1, filepath) except Exception: warnings.warn( "Failed to add file [%s] specified in 'spark.submit.pyFiles' to " "Python path:\n %s" % (path, "\n ".join(sys.path)), RuntimeWarning)
# Create a temporary directory inside spark.local.dir: self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir, "pyspark") \ .getAbsolutePath()
# profiling stats collected for each PythonRDD else:
# create a signal handler which would be invoked on receiving SIGINT self.cancelAllJobs() raise KeyboardInterrupt()
# see http://stackoverflow.com/questions/23206787/
return "<SparkContext master={master} appName={appName}>".format( master=self.master, appName=self.appName, )
return """ <div> <p><b>SparkContext</b></p>
<p><a href="{sc.uiWebUrl}">Spark UI</a></p>
<dl> <dt>Version</dt> <dd><code>v{sc.version}</code></dd> <dt>Master</dt> <dd><code>{sc.master}</code></dd> <dt>AppName</dt> <dd><code>{sc.appName}</code></dd> </dl> </div> """.format( sc=self )
""" Initialize SparkContext in function to allow subclass specific initialization """
""" Checks whether a SparkContext is initialized or not. Throws error if a SparkContext is already running. """
SparkContext._active_spark_context != instance):
# Raise error if there is already a running Spark context "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:
# This method is called when attempting to pickle SparkContext, which is always an error: "It appears that you are attempting to reference SparkContext from a broadcast " "variable, action, or transformation. SparkContext can only be used on the driver, " "not in code that it run on workers. For more information, see SPARK-5063." )
""" Enable 'with SparkContext(...) as sc: app(sc)' syntax. """
""" Enable 'with SparkContext(...) as sc: app' syntax.
Specifically stop the context on exit of the with block. """
""" Get or instantiate a SparkContext and register it as a singleton object.
Parameters ---------- conf : :py:class:`pyspark.SparkConf`, optional """
""" Control our logLevel. This overrides any user-defined log settings. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN """ self._jsc.setLogLevel(logLevel)
def setSystemProperty(cls, key, value): """ 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)
def version(self): """ The version of Spark on which this application is running. """
def applicationId(self): """ A unique identifier for the Spark application. Its format depends on the scheduler implementation.
* in case of local spark app something like 'local-1433865536131' * in case of YARN something like 'application_1433865536131_34483'
Examples -------- >>> sc.applicationId # doctest: +ELLIPSIS 'local-...' """
def uiWebUrl(self): """Return the URL of the SparkUI instance started by this SparkContext""" return self._jsc.sc().uiWebUrl().get()
def startTime(self): """Return the epoch time when the Spark Context was started."""
def defaultParallelism(self): """ Default level of parallelism to use when not given by user (e.g. for reduce tasks) """
def defaultMinPartitions(self): """ Default min number of partitions for Hadoop RDDs when not given by user """
""" Shut down the SparkContext. """ except Py4JError: # Case: SPARK-18523 warnings.warn( 'Unable to cleanly shutdown Spark JVM process.' ' It is possible that the process has crashed,' ' been killed or may also be in a zombie state.', RuntimeWarning ) finally:
""" Create an RDD that has no partitions or elements. """
""" Create a new RDD of int containing elements from `start` to `end` (exclusive), increased by `step` every element. Can be called the same way as python's built-in range() function. If called with a single argument, the argument is interpreted as `end`, and `start` is set to 0.
Parameters ---------- start : int the start value end : int, optional the end value (exclusive) step : int, optional the incremental step (default: 1) numSlices : int, optional the number of partitions of the new RDD
Returns ------- :py:class:`pyspark.RDD` An RDD of int
Examples -------- >>> sc.range(5).collect() [0, 1, 2, 3, 4] >>> sc.range(2, 4).collect() [2, 3] >>> sc.range(1, 7, 2).collect() [1, 3, 5] """
""" Distribute a local Python collection to form an RDD. Using range is recommended if the input represents a range for performance.
Examples -------- >>> sc.parallelize([0, 2, 3, 4, 6], 5).glom().collect() [[0], [2], [3], [4], [6]] >>> sc.parallelize(range(0, 6, 2), 5).glom().collect() [[], [0], [], [2], [4]] """
# it's an empty iterator here but we need this line for triggering the # logic of signal handling in FramedSerializer.load_stream, for instance, # SpecialLengths.END_OF_DATA_SECTION in _read_with_length. Since # FramedSerializer.load_stream produces a generator, the control should # at least be in that function once. Here we do it by explicitly converting # the empty iterator to a list, thus make sure worker reuse takes effect. # See more details in SPARK-26549.
# Make sure we distribute data evenly if it's smaller than self.batchSize
""" Using py4j to send a large dataset to the jvm is really slow, so we use either a file or a socket if we have encryption enabled.
Examples -------- data object to be serialized serializer : :py:class:`pyspark.serializers.Serializer` reader_func : function A function which takes a filename and reads in the data in the jvm and returns a JavaRDD. Only used when encryption is disabled. createRDDServer : function A function which creates a PythonRDDServer in the jvm to accept the serialized data, for use when encryption is enabled. """ # with encryption, we open a server in java and send the data directly # this call will block until the server has read all the data and processed it (or # throws an exception) else: # without encryption, we serialize to a file, and we read the file in java and # parallelize from there. finally: finally: # we eagerly reads the file so we can delete right after.
""" Load an RDD previously saved using :meth:`RDD.saveAsPickleFile` method.
Examples -------- >>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile.close() >>> sc.parallelize(range(10)).saveAsPickleFile(tmpFile.name, 5) >>> sorted(sc.pickleFile(tmpFile.name, 3).collect()) [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """
""" 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. The text files must be encoded as UTF-8.
If use_unicode is False, the strings will be kept as `str` (encoding as `utf-8`), which is faster and smaller than unicode. (Added in Spark 1.2)
Examples -------- >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() ['Hello world!'] """ UTF8Deserializer(use_unicode))
""" 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. The text files must be encoded as UTF-8.
If `use_unicode` is False, the strings will be kept as `str` (encoding as `utf-8`), which is faster and smaller than unicode. (Added in Spark 1.2)
For example, if you have the following files:
.. code-block:: text
hdfs://a-hdfs-path/part-00000 hdfs://a-hdfs-path/part-00001 ... hdfs://a-hdfs-path/part-nnnnn
Do ``rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")``, then ``rdd`` contains:
.. code-block:: text
(a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content)
Notes ----- Small files are preferred, as each file will be loaded fully in memory.
Examples -------- >>> 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()) [('.../1.txt', '1'), ('.../2.txt', '2')] """ PairDeserializer(UTF8Deserializer(use_unicode), UTF8Deserializer(use_unicode)))
""" Read a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI as a byte array. 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.
Notes ----- Small files are preferred, large file is also allowable, but may cause bad performance. """ PairDeserializer(UTF8Deserializer(), NoOpSerializer()))
""" Load data from a flat binary file, assuming each record is a set of numbers with the specified numerical format (see ByteBuffer), and the number of bytes per record is constant.
Parameters ---------- path : str Directory to the input data files recordLength : int The length at which to split the records """
valueConverter=None, minSplits=None, batchSize=0): """ Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is as follows:
1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes 2. Serialization is attempted via Pyrolite pickling 3. If this fails, the fallback is to call 'toString' on each key and value 4. :class:`PickleSerializer` is used to deserialize pickled objects on the Python side
Parameters ---------- path : str path to sequencefile keyClass: str, optional fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") valueClass : str, optional fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") keyConverter : str, optional fully qualified name of a function returning key WritableConverter valueConverter : str, optional fully qualifiedname of a function returning value WritableConverter minSplits : int, optional minimum splits in dataset (default min(2, sc.defaultParallelism)) batchSize : int, optional The number of Python objects represented as a single Java object. (default 0, choose batchSize automatically) """ keyConverter, valueConverter, minSplits, batchSize)
valueConverter=None, conf=None, batchSize=0): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for :py:meth:`SparkContext.sequenceFile`.
A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java
Parameters ---------- path : str path to Hadoop file inputFormatClass : str fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") keyClass : str fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") valueClass : str fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") keyConverter : str, optional fully qualified name of a function returning key WritableConverter None by default valueConverter : str, optional fully qualified name of a function returning value WritableConverter None by default conf : dict, optional Hadoop configuration, passed in as a dict None by default batchSize : int, optional The number of Python objects represented as a single Java object. (default 0, choose batchSize automatically) """ valueClass, keyConverter, valueConverter, jconf, batchSize)
valueConverter=None, conf=None, batchSize=0): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for :py:meth:`SparkContext.sequenceFile`.
Parameters ---------- inputFormatClass : str fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") keyClass : str fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") valueClass : str fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") keyConverter : str, optional fully qualified name of a function returning key WritableConverter (None by default) valueConverter : str, optional fully qualified name of a function returning value WritableConverter (None by default) conf : dict, optional Hadoop configuration, passed in as a dict (None by default) batchSize : int, optional The number of Python objects represented as a single Java object. (default 0, choose batchSize automatically) """ valueClass, keyConverter, valueConverter, jconf, batchSize)
valueConverter=None, conf=None, batchSize=0): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for :py:meth:`SparkContext.sequenceFile`.
A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java.
path : str path to Hadoop file inputFormatClass : str fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") keyClass : str fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") valueClass : str fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") keyConverter : str, optional fully qualified name of a function returning key WritableConverter (None by default) valueConverter : str, optional fully qualified name of a function returning value WritableConverter (None by default) conf : dict, optional Hadoop configuration, passed in as a dict (None by default) batchSize : int, optional The number of Python objects represented as a single Java object. (default 0, choose batchSize automatically) """ valueClass, keyConverter, valueConverter, jconf, batchSize)
valueConverter=None, conf=None, batchSize=0): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for :py:meth:`SparkContext.sequenceFile`.
Parameters ---------- inputFormatClass : str fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") keyClass : str fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") valueClass : str fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") keyConverter : str, optional fully qualified name of a function returning key WritableConverter (None by default) valueConverter : str, optional fully qualified name of a function returning value WritableConverter (None by default) conf : dict, optional Hadoop configuration, passed in as a dict (None by default) batchSize : int, optional The number of Python objects represented as a single Java object. (default 0, choose batchSize automatically) """ valueClass, keyConverter, valueConverter, jconf, batchSize)
""" 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:
Examples -------- >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() ['Hello'] >>> parallelized = sc.parallelize(["World!"]) >>> sorted(sc.union([textFile, parallelized]).collect()) ['Hello', 'World!'] """ elif is_instance_of(gw, rdds[0]._jrdd, jdouble_rdd_cls): cls = jdouble_rdd_cls else: cls_name = rdds[0]._jrdd.getClass().getCanonicalName() raise TypeError("Unsupported Java RDD class %s" % cls_name)
""" Broadcast a read-only variable to the cluster, returning a :class:`Broadcast` object for reading it in distributed functions. The variable will be sent to each cluster only once. """
""" Create an :class:`Accumulator` with the given initial value, using a given :class:`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. """ else:
""" Add a file to be downloaded with this Spark job on every node. The `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 :meth:`SparkFiles.get` with the filename to find its download location.
A directory can be given if the recursive option is set to True. Currently directories are only supported for Hadoop-supported filesystems.
Notes ----- A path can be added only once. Subsequent additions of the same path are ignored.
Examples -------- >>> 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 * fileVal for x in iterator] >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect() [100, 200, 300, 400] """
""" Add a .py or .zip dependency for all tasks to be executed on this SparkContext in the future. The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, HTTPS or FTP URI.
Notes ----- A path can be added only once. Subsequent additions of the same path are ignored. """ # for tests in local mode
""" Set the directory under which RDDs are going to be checkpointed. The directory must be an HDFS path if running on a cluster. """
def getCheckpointDir(self): """ Return the directory where RDDs are checkpointed. Returns None if no checkpoint directory has been set. """ return None
""" Returns a Java StorageLevel based on a pyspark.StorageLevel. """ raise TypeError("storageLevel must be of type pyspark.StorageLevel")
storageLevel.useMemory, storageLevel.useOffHeap, storageLevel.deserialized, storageLevel.replication)
""" 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 :meth:`SparkContext.cancelJobGroup` to cancel all running jobs in this group.
Notes ----- 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.
If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread local inheritance, and preventing resource leak.
Examples -------- >>> import threading >>> from time import sleep >>> from pyspark import InheritableThread >>> result = "Not Set" >>> lock = threading.Lock() >>> def map_func(x): ... sleep(100) ... raise RuntimeError("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") >>> suppress = lock.acquire() >>> suppress = InheritableThread(target=start_job, args=(10,)).start() >>> suppress = InheritableThread(target=stop_job).start() >>> suppress = lock.acquire() >>> print(result) Cancelled """
""" Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool.
Notes ----- If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread local inheritance, and preventing resource leak. """
""" Get a local property set in this thread, or null if it is missing. See :meth:`setLocalProperty`. """ return self._jsc.getLocalProperty(key)
""" Set a human readable description of the current job.
Notes ----- If you run jobs in parallel, use :class:`pyspark.InheritableThread` for thread local inheritance, and preventing resource leak. """ self._jsc.setJobDescription(value)
""" Get SPARK_USER for user who is running SparkContext. """ return self._jsc.sc().sparkUser()
""" Cancel active jobs for the specified group. See :meth:`SparkContext.setJobGroup`. for more information. """
""" Cancel all jobs that have been scheduled or are running. """
""" Return :class:`StatusTracker` object """
""" Executes the given partitionFunc on the specified set of partitions, returning the result as an array of elements.
If 'partitions' is not specified, this will run over all partitions.
Examples -------- >>> myRDD = sc.parallelize(range(6), 3) >>> sc.runJob(myRDD, lambda part: [x * x for x in part]) [0, 1, 4, 9, 16, 25]
>>> myRDD = sc.parallelize(range(6), 3) >>> sc.runJob(myRDD, lambda part: [x * x for x in part], [0, 2], True) [0, 1, 16, 25] """
# Implementation note: This is implemented as a mapPartitions followed # by runJob() in order to avoid having to pass a Python lambda into # SparkContext#runJob.
""" Print the profile stats to stdout """ else: "to 'true' to enable Python profile.")
""" Dump the profile stats into directory `path` """ else: "to 'true' to enable Python profile.")
def resources(self):
def _assert_on_driver(): """ Called to ensure that SparkContext is created only on the Driver.
Throws an exception if a SparkContext is about to be created in executors. """ raise RuntimeError("SparkContext should only be created and accessed on the driver.")
sys.exit(-1)
|