class JavaSparkContext extends Closeable
A Java-friendly version of org.apache.spark.SparkContext that returns org.apache.spark.api.java.JavaRDDs and works with Java collections instead of Scala ones.
- Source
- JavaSparkContext.scala
- Note
Only one
SparkContext
should be active per JVM. You muststop()
the activeSparkContext
before creating a new one.
- Alphabetic
- By Inheritance
- JavaSparkContext
- Closeable
- AutoCloseable
- AnyRef
- Any
- Hide All
- Show All
- Public
- Protected
Instance Constructors
- new JavaSparkContext(master: String, appName: String, sparkHome: String, jars: Array[String], environment: Map[String, String])
- master
Cluster URL to connect to (e.g. spark://host:port, local[4]).
- appName
A name for your application, to display on the cluster web UI
- sparkHome
The SPARK_HOME directory on the worker nodes
- jars
Collection of JARs to send to the cluster. These can be paths on the local file system or HDFS, HTTP, HTTPS, or FTP URLs.
- environment
Environment variables to set on worker nodes
- new JavaSparkContext(master: String, appName: String, sparkHome: String, jars: Array[String])
- master
Cluster URL to connect to (e.g. spark://host:port, local[4]).
- appName
A name for your application, to display on the cluster web UI
- sparkHome
The SPARK_HOME directory on the worker nodes
- jars
Collection of JARs to send to the cluster. These can be paths on the local file system or HDFS, HTTP, HTTPS, or FTP URLs.
- new JavaSparkContext(master: String, appName: String, sparkHome: String, jarFile: String)
- master
Cluster URL to connect to (e.g. spark://host:port, local[4]).
- appName
A name for your application, to display on the cluster web UI
- sparkHome
The SPARK_HOME directory on the worker nodes
- jarFile
JAR file to send to the cluster. This can be a path on the local file system or an HDFS, HTTP, HTTPS, or FTP URL.
- new JavaSparkContext(master: String, appName: String, conf: SparkConf)
- master
Cluster URL to connect to (e.g. spark://host:port, local[4]).
- appName
A name for your application, to display on the cluster web UI
- conf
a org.apache.spark.SparkConf object specifying other Spark parameters
- new JavaSparkContext(master: String, appName: String)
- master
Cluster URL to connect to (e.g. spark://host:port, local[4]).
- appName
A name for your application, to display on the cluster web UI
- new JavaSparkContext(conf: SparkConf)
- conf
a org.apache.spark.SparkConf object specifying Spark parameters
- new JavaSparkContext()
Create a JavaSparkContext that loads settings from system properties (for instance, when launching with ./bin/spark-submit).
- new JavaSparkContext(sc: SparkContext)
Value Members
- final def !=(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
- final def ##: Int
- Definition Classes
- AnyRef → Any
- final def ==(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
- def addFile(path: String, recursive: Boolean): Unit
Add a file to be downloaded with this Spark job on every node.
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, useSparkFiles.get(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.
- Note
A path can be added only once. Subsequent additions of the same path are ignored.
- def addFile(path: String): Unit
Add a file to be downloaded with this Spark job on every node.
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, useSparkFiles.get(fileName)
to find its download location.- Note
A path can be added only once. Subsequent additions of the same path are ignored.
- def addJar(path: String): Unit
Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
Adds a JAR 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.- Note
A path can be added only once. Subsequent additions of the same path are ignored.
- def addJobTag(tag: String): Unit
Add a tag to be assigned to all the jobs started by this thread.
Add a tag to be assigned to all the jobs started by this thread.
- tag
The tag to be added. Cannot contain ',' (comma) character.
- Since
3.5.0
- def appName: String
- final def asInstanceOf[T0]: T0
- Definition Classes
- Any
- def binaryFiles(path: String): JavaPairRDD[String, PortableDataStream]
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.
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.
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
JavaPairRDD<String, byte[]> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")
,
then
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; very large files but may cause bad performance.
- def binaryFiles(path: String, minPartitions: Int): JavaPairRDD[String, PortableDataStream]
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.
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.
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
JavaPairRDD<String, byte[]> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")
then
rdd
contains(a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content)
- minPartitions
A suggestion value of the minimal splitting number for input data.
- Note
Small files are preferred; very large files but may cause bad performance.
- def binaryRecords(path: String, recordLength: Int): JavaRDD[Array[Byte]]
Load data from a flat binary file, assuming the length of each record is constant.
Load data from a flat binary file, assuming the length of each record is constant.
- path
Directory to the input data files
- returns
An RDD of data with values, represented as byte arrays
- def broadcast[T](value: T): Broadcast[T]
Broadcast a read-only variable to the cluster, returning a org.apache.spark.broadcast.Broadcast object for reading it in distributed functions.
Broadcast a read-only variable to the cluster, returning a org.apache.spark.broadcast.Broadcast object for reading it in distributed functions. The variable will be sent to each cluster only once.
- def cancelAllJobs(): Unit
Cancel all jobs that have been scheduled or are running.
- def cancelJobGroup(groupId: String): Unit
Cancel active jobs for the specified group.
Cancel active jobs for the specified group. See
org.apache.spark.api.java.JavaSparkContext.setJobGroup
for more information.- groupId
the group ID to cancel
- def cancelJobGroup(groupId: String, reason: String): Unit
Cancel active jobs for the specified group.
Cancel active jobs for the specified group. See
org.apache.spark.api.java.JavaSparkContext.setJobGroup
for more information.- groupId
the group ID to cancel
- reason
reason for cancellation
- Since
4.0.0
- def cancelJobsWithTag(tag: String): Unit
Cancel active jobs that have the specified tag.
Cancel active jobs that have the specified tag. See
org.apache.spark.SparkContext.addJobTag
.- tag
The tag to be cancelled. Cannot contain ',' (comma) character.
- Since
3.5.0
- def cancelJobsWithTag(tag: String, reason: String): Unit
Cancel active jobs that have the specified tag.
Cancel active jobs that have the specified tag. See
org.apache.spark.SparkContext.addJobTag
.- tag
The tag to be cancelled. Cannot contain ',' (comma) character.
- reason
reason for cancellation
- Since
4.0.0
- def checkpointFile[T](path: String): JavaRDD[T]
- Attributes
- protected
- def clearCallSite(): Unit
Pass-through to SparkContext.setCallSite.
Pass-through to SparkContext.setCallSite. For API support only.
- def clearJobGroup(): Unit
Clear the current thread's job group ID and its description.
- def clearJobTags(): Unit
Clear the current thread's job tags.
Clear the current thread's job tags.
- Since
3.5.0
- def clone(): AnyRef
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
- def close(): Unit
- Definition Classes
- JavaSparkContext → Closeable → AutoCloseable
- def defaultMinPartitions: Integer
Default min number of partitions for Hadoop RDDs when not given by user
- def defaultParallelism: Integer
Default level of parallelism to use when not given by user (e.g.
Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD).
- def emptyRDD[T]: JavaRDD[T]
Get an RDD that has no partitions or elements.
- final def eq(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
- def equals(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef → Any
- def getCheckpointDir: Optional[String]
- final def getClass(): Class[_ <: AnyRef]
- Definition Classes
- AnyRef → Any
- Annotations
- @IntrinsicCandidate() @native()
- def getConf: SparkConf
Return a copy of this JavaSparkContext's configuration.
Return a copy of this JavaSparkContext's configuration. The configuration cannot be changed at runtime.
- def getJobTags(): Set[String]
Get the tags that are currently set to be assigned to all the jobs started by this thread.
Get the tags that are currently set to be assigned to all the jobs started by this thread.
- Since
3.5.0
- def getLocalProperty(key: String): String
Get a local property set in this thread, or null if it is missing.
Get a local property set in this thread, or null if it is missing. See
org.apache.spark.api.java.JavaSparkContext.setLocalProperty
. - def getPersistentRDDs: Map[Integer, JavaRDD[_]]
Returns a Java map of JavaRDDs that have marked themselves as persistent via cache() call.
Returns a Java map of JavaRDDs that have marked themselves as persistent via cache() call.
- Note
This does not necessarily mean the caching or computation was successful.
- def getReadOnlyConf: ReadOnlySparkConf
Return a read-only version of the spark conf.
- def getSparkHome(): Optional[String]
Get Spark's home location from either a value set through the constructor, or the spark.home Java property, or the SPARK_HOME environment variable (in that order of preference).
Get Spark's home location from either a value set through the constructor, or the spark.home Java property, or the SPARK_HOME environment variable (in that order of preference). If neither of these is set, return None.
- def hadoopConfiguration(): Configuration
Returns the Hadoop configuration used for the Hadoop code (e.g.
Returns the Hadoop configuration used for the Hadoop code (e.g. file systems) we reuse.
- Note
As it will be reused in all Hadoop RDDs, it's better not to modify it unless you plan to set some global configurations for all Hadoop RDDs.
- def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V]
Get an RDD for a Hadoop file with an arbitrary InputFormat
Get an RDD for a Hadoop file with an arbitrary InputFormat
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V], minPartitions: Int): JavaPairRDD[K, V]
Get an RDD for a Hadoop file with an arbitrary InputFormat.
Get an RDD for a Hadoop file with an arbitrary InputFormat.
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def hadoopRDD[K, V, F <: InputFormat[K, V]](conf: JobConf, inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V]
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any other necessary info (e.g.
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
- conf
JobConf for setting up the dataset. Note: This will be put into a Broadcast. Therefore if you plan to reuse this conf to create multiple RDDs, you need to make sure you won't modify the conf. A safe approach is always creating a new conf for a new RDD.
- inputFormatClass
Class of the InputFormat
- keyClass
Class of the keys
- valueClass
Class of the values
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def hadoopRDD[K, V, F <: InputFormat[K, V]](conf: JobConf, inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V], minPartitions: Int): JavaPairRDD[K, V]
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any other necessary info (e.g.
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, etc).
- conf
JobConf for setting up the dataset. Note: This will be put into a Broadcast. Therefore if you plan to reuse this conf to create multiple RDDs, you need to make sure you won't modify the conf. A safe approach is always creating a new conf for a new RDD.
- inputFormatClass
Class of the InputFormat
- keyClass
Class of the keys
- valueClass
Class of the values
- minPartitions
Minimum number of Hadoop Splits to generate.
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def hashCode(): Int
- Definition Classes
- AnyRef → Any
- Annotations
- @IntrinsicCandidate() @native()
- final def isInstanceOf[T0]: Boolean
- Definition Classes
- Any
- def isLocal: Boolean
- def jars: List[String]
- def master: String
- final def ne(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
- def newAPIHadoopFile[K, V, F <: InputFormat[K, V]](path: String, fClass: Class[F], kClass: Class[K], vClass: Class[V], conf: Configuration): JavaPairRDD[K, V]
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def newAPIHadoopRDD[K, V, F <: InputFormat[K, V]](conf: Configuration, fClass: Class[F], kClass: Class[K], vClass: Class[V]): JavaPairRDD[K, V]
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.
- conf
Configuration for setting up the dataset. Note: This will be put into a Broadcast. Therefore if you plan to reuse this conf to create multiple RDDs, you need to make sure you won't modify the conf. A safe approach is always creating a new conf for a new RDD.
- fClass
Class of the InputFormat
- kClass
Class of the keys
- vClass
Class of the values
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- final def notify(): Unit
- Definition Classes
- AnyRef
- Annotations
- @IntrinsicCandidate() @native()
- final def notifyAll(): Unit
- Definition Classes
- AnyRef
- Annotations
- @IntrinsicCandidate() @native()
- def objectFile[T](path: String): JavaRDD[T]
Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition.
Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition. This is still an experimental storage format and may not be supported exactly as is in future Spark releases. It will also be pretty slow if you use the default serializer (Java serialization), though the nice thing about it is that there's very little effort required to save arbitrary objects.
- def objectFile[T](path: String, minPartitions: Int): JavaRDD[T]
Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition.
Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition. This is still an experimental storage format and may not be supported exactly as is in future Spark releases. It will also be pretty slow if you use the default serializer (Java serialization), though the nice thing about it is that there's very little effort required to save arbitrary objects.
- def parallelize[T](list: List[T]): JavaRDD[T]
Distribute a local Scala collection to form an RDD.
- def parallelize[T](list: List[T], numSlices: Int): JavaRDD[T]
Distribute a local Scala collection to form an RDD.
- def parallelizeDoubles(list: List[Double]): JavaDoubleRDD
Distribute a local Scala collection to form an RDD.
- def parallelizeDoubles(list: List[Double], numSlices: Int): JavaDoubleRDD
Distribute a local Scala collection to form an RDD.
- def parallelizePairs[K, V](list: List[(K, V)]): JavaPairRDD[K, V]
Distribute a local Scala collection to form an RDD.
- def parallelizePairs[K, V](list: List[(K, V)], numSlices: Int): JavaPairRDD[K, V]
Distribute a local Scala collection to form an RDD.
- def removeJobTag(tag: String): Unit
Remove a tag previously added to be assigned to all the jobs started by this thread.
Remove a tag previously added to be assigned to all the jobs started by this thread. Noop if such a tag was not added earlier.
- tag
The tag to be removed. Cannot contain ',' (comma) character.
- Since
3.5.0
- def resources: Map[String, ResourceInformation]
- val sc: SparkContext
- def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V]
Get an RDD for a Hadoop SequenceFile.
Get an RDD for a Hadoop SequenceFile.
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], minPartitions: Int): JavaPairRDD[K, V]
Get an RDD for a Hadoop SequenceFile with given key and value types.
Get an RDD for a Hadoop SequenceFile with given key and value types.
- Note
Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD will create many references to the same object. If you plan to directly cache Hadoop writable objects, you should first copy them using a
map
function.
- def setCallSite(site: String): Unit
Pass-through to SparkContext.setCallSite.
Pass-through to SparkContext.setCallSite. For API support only.
- def setCheckpointDir(dir: String): Unit
Set the directory under which RDDs are going to be checkpointed.
Set the directory under which RDDs are going to be checkpointed. The directory must be an HDFS path if running on a cluster.
- def setInterruptOnCancel(interruptOnCancel: Boolean): Unit
Set the behavior of job cancellation from jobs started in this thread.
Set the behavior of job cancellation from jobs started in this thread.
- interruptOnCancel
If true, 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.
- Since
3.5.0
- def setJobDescription(value: String): Unit
Set a human readable description of the current job.
Set a human readable description of the current job.
- Since
2.3.0
- def setJobGroup(groupId: String, description: String): Unit
Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared.
Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared.
- See also
setJobGroup(groupId: String, description: String, interruptThread: Boolean)
. This method sets interruptOnCancel to false.
- def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean): Unit
Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared.
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 also use
org.apache.spark.api.java.JavaSparkContext.cancelJobGroup
to cancel all running jobs in this group. For example,// In the main thread: sc.setJobGroup("some_job_to_cancel", "some job description"); rdd.map(...).count(); // In a separate thread: sc.cancelJobGroup("some_job_to_cancel");
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.
- def setLocalProperty(key: String, value: String): Unit
Set a local property that affects jobs submitted from this thread, and all child threads, such as the Spark fair scheduler pool.
Set a local property that affects jobs submitted from this thread, and all child threads, such as the Spark fair scheduler pool.
These properties are inherited by child threads spawned from this thread. This may have unexpected consequences when working with thread pools. The standard java implementation of thread pools have worker threads spawn other worker threads. As a result, local properties may propagate unpredictably.
- def setLogLevel(logLevel: String): Unit
Control our logLevel.
Control our logLevel. This overrides any user-defined log settings.
- logLevel
The desired log level as a string. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN
- def sparkUser: String
- def startTime: Long
- def statusTracker: JavaSparkStatusTracker
- def stop(): Unit
Shut down the SparkContext.
- final def synchronized[T0](arg0: => T0): T0
- Definition Classes
- AnyRef
- def textFile(path: String, minPartitions: Int): JavaRDD[String]
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.
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.
- def textFile(path: String): JavaRDD[String]
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.
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.
- def toString(): String
- Definition Classes
- AnyRef → Any
- def union(rdds: JavaDoubleRDD*): JavaDoubleRDD
Build the union of JavaDoubleRDDs.
Build the union of JavaDoubleRDDs.
- Annotations
- @varargs()
- def union[K, V](rdds: JavaPairRDD[K, V]*): JavaPairRDD[K, V]
Build the union of JavaPairRDDs.
Build the union of JavaPairRDDs.
- Annotations
- @varargs()
- def union[T](rdds: JavaRDD[T]*): JavaRDD[T]
Build the union of JavaRDDs.
Build the union of JavaRDDs.
- Annotations
- @varargs()
- def version: String
The version of Spark on which this application is running.
- final def wait(arg0: Long, arg1: Int): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException])
- final def wait(arg0: Long): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException]) @native()
- final def wait(): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException])
- def wholeTextFiles(path: String): JavaPairRDD[String, String]
Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI.
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.
- See also
wholeTextFiles(path: String, minPartitions: Int)
.
- def wholeTextFiles(path: String, minPartitions: Int): JavaPairRDD[String, String]
Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI.
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.
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
JavaPairRDD<String, String> rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")
then
rdd
contains(a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content)
- minPartitions
A suggestion value of the minimal splitting number for input data.
- Note
Small files are preferred, large file is also allowable, but may cause bad performance.
Deprecated Value Members
- def finalize(): Unit
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.Throwable]) @Deprecated
- Deprecated
(Since version 9)