Packages

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 must stop() the active SparkContext before creating a new one.

Linear Supertypes
Closeable, AutoCloseable, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. JavaSparkContext
  2. Closeable
  3. AutoCloseable
  4. AnyRef
  5. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Instance Constructors

  1. new JavaSparkContext(master: String, appName: String, sparkHome: String, jars: Array[String], environment: Map[String, String])

    master

    Cluster URL to connect to (e.g. mesos://host:port, 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

  2. new JavaSparkContext(master: String, appName: String, sparkHome: String, jars: Array[String])

    master

    Cluster URL to connect to (e.g. mesos://host:port, 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.

  3. new JavaSparkContext(master: String, appName: String, sparkHome: String, jarFile: String)

    master

    Cluster URL to connect to (e.g. mesos://host:port, 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.

  4. new JavaSparkContext(master: String, appName: String, conf: SparkConf)

    master

    Cluster URL to connect to (e.g. mesos://host:port, 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

  5. new JavaSparkContext(master: String, appName: String)

    master

    Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).

    appName

    A name for your application, to display on the cluster web UI

  6. new JavaSparkContext(conf: SparkConf)

    conf

    a org.apache.spark.SparkConf object specifying Spark parameters

  7. new JavaSparkContext()

    Create a JavaSparkContext that loads settings from system properties (for instance, when launching with ./bin/spark-submit).

  8. new JavaSparkContext(sc: SparkContext)

Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. 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, use SparkFiles.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.

  5. 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, use SparkFiles.get(fileName) to find its download location.

    Note

    A path can be added only once. Subsequent additions of the same path are ignored.

  6. 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.

  7. 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

  8. def appName: String
  9. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  10. 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.

  11. 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.

  12. 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

  13. 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.

  14. def cancelAllJobs(): Unit

    Cancel all jobs that have been scheduled or are running.

  15. 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.

  16. 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

  17. def checkpointFile[T](path: String): JavaRDD[T]
    Attributes
    protected
  18. def clearCallSite(): Unit

    Pass-through to SparkContext.setCallSite.

    Pass-through to SparkContext.setCallSite. For API support only.

  19. def clearJobGroup(): Unit

    Clear the current thread's job group ID and its description.

  20. def clearJobTags(): Unit

    Clear the current thread's job tags.

    Clear the current thread's job tags.

    Since

    3.5.0

  21. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native() @IntrinsicCandidate()
  22. def close(): Unit
    Definition Classes
    JavaSparkContext → Closeable → AutoCloseable
  23. def defaultMinPartitions: Integer

    Default min number of partitions for Hadoop RDDs when not given by user

  24. 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).

  25. def emptyRDD[T]: JavaRDD[T]

    Get an RDD that has no partitions or elements.

  26. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  27. def equals(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  28. def getCheckpointDir: Optional[String]
  29. final def getClass(): Class[_]
    Definition Classes
    AnyRef → Any
    Annotations
    @native() @IntrinsicCandidate()
  30. 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.

  31. 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

  32. 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.

  33. 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.

  34. 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.

  35. 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.

  36. 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.

  37. 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.

  38. 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.

  39. 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.

  40. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @native() @IntrinsicCandidate()
  41. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  42. def isLocal: Boolean
  43. def jars: List[String]
  44. def master: String
  45. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  46. 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.

  47. 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.

  48. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native() @IntrinsicCandidate()
  49. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native() @IntrinsicCandidate()
  50. 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.

  51. 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.

  52. def parallelize[T](list: List[T]): JavaRDD[T]

    Distribute a local Scala collection to form an RDD.

  53. def parallelize[T](list: List[T], numSlices: Int): JavaRDD[T]

    Distribute a local Scala collection to form an RDD.

  54. def parallelizeDoubles(list: List[Double]): JavaDoubleRDD

    Distribute a local Scala collection to form an RDD.

  55. def parallelizeDoubles(list: List[Double], numSlices: Int): JavaDoubleRDD

    Distribute a local Scala collection to form an RDD.

  56. def parallelizePairs[K, V](list: List[(K, V)]): JavaPairRDD[K, V]

    Distribute a local Scala collection to form an RDD.

  57. def parallelizePairs[K, V](list: List[(K, V)], numSlices: Int): JavaPairRDD[K, V]

    Distribute a local Scala collection to form an RDD.

  58. 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

  59. def resources: Map[String, ResourceInformation]
  60. val sc: SparkContext
  61. 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.

  62. 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.

  63. def setCallSite(site: String): Unit

    Pass-through to SparkContext.setCallSite.

    Pass-through to SparkContext.setCallSite. For API support only.

  64. 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.

  65. 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

  66. 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

  67. 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.

  68. 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.

  69. 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.

  70. 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

  71. def sparkUser: String
  72. def startTime: Long
  73. def statusTracker: JavaSparkStatusTracker
  74. def stop(): Unit

    Shut down the SparkContext.

  75. final def synchronized[T0](arg0: ⇒ T0): T0
    Definition Classes
    AnyRef
  76. 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.

  77. 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.

  78. def toString(): String
    Definition Classes
    AnyRef → Any
  79. def union(rdds: JavaDoubleRDD*): JavaDoubleRDD

    Build the union of JavaDoubleRDDs.

    Build the union of JavaDoubleRDDs.

    Annotations
    @varargs()
  80. def union[K, V](rdds: JavaPairRDD[K, V]*): JavaPairRDD[K, V]

    Build the union of JavaPairRDDs.

    Build the union of JavaPairRDDs.

    Annotations
    @varargs()
  81. def union[T](rdds: JavaRDD[T]*): JavaRDD[T]

    Build the union of JavaRDDs.

    Build the union of JavaRDDs.

    Annotations
    @varargs()
  82. def version: String

    The version of Spark on which this application is running.

  83. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  84. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()
  85. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  86. 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).

  87. 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

  1. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] ) @Deprecated
    Deprecated

Inherited from Closeable

Inherited from AutoCloseable

Inherited from AnyRef

Inherited from Any

Ungrouped