@@ -44,7 +44,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
4444
4545import org .apache .mesos .MesosNativeLibrary
4646
47- import org .apache .spark .annotation .{DeveloperApi , Experimental , RDDScope }
47+ import org .apache .spark .annotation .{DeveloperApi , Experimental , RDDScoped }
4848import org .apache .spark .broadcast .Broadcast
4949import org .apache .spark .deploy .{LocalSparkCluster , SparkHadoopUtil }
5050import org .apache .spark .executor .{ExecutorEndpoint , TriggerThreadDump }
@@ -641,7 +641,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
641641 * @note avoid using `parallelize(Seq())` to create an empty `RDD`. Consider `emptyRDD` for an
642642 * RDD with no partitions, or `parallelize(Seq[T]())` for an RDD of `T` with empty partitions.
643643 */
644- @ RDDScope
644+ @ RDDScoped
645645 def parallelize [T : ClassTag ](seq : Seq [T ], numSlices : Int = defaultParallelism): RDD [T ] = {
646646 assertNotStopped()
647647 new ParallelCollectionRDD [T ](this , seq, numSlices, Map [Int , Seq [String ]]())
@@ -651,15 +651,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
651651 *
652652 * This method is identical to `parallelize`.
653653 */
654- @ RDDScope
654+ @ RDDScoped
655655 def makeRDD [T : ClassTag ](seq : Seq [T ], numSlices : Int = defaultParallelism): RDD [T ] = {
656656 parallelize(seq, numSlices)
657657 }
658658
659659 /** Distribute a local Scala collection to form an RDD, with one or more
660660 * location preferences (hostnames of Spark nodes) for each object.
661661 * Create a new partition for each collection item. */
662- @ RDDScope
662+ @ RDDScoped
663663 def makeRDD [T : ClassTag ](seq : Seq [(T , Seq [String ])]): RDD [T ] = {
664664 assertNotStopped()
665665 val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
@@ -670,7 +670,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
670670 * Read a text file from HDFS, a local file system (available on all nodes), or any
671671 * Hadoop-supported file system URI, and return it as an RDD of Strings.
672672 */
673- @ RDDScope
673+ @ RDDScoped
674674 def textFile (path : String , minPartitions : Int = defaultMinPartitions): RDD [String ] = {
675675 assertNotStopped()
676676 hadoopFile(path, classOf [TextInputFormat ], classOf [LongWritable ], classOf [Text ],
@@ -704,7 +704,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
704704 *
705705 * @param minPartitions A suggestion value of the minimal splitting number for input data.
706706 */
707- @ RDDScope
707+ @ RDDScoped
708708 def wholeTextFiles (path : String , minPartitions : Int = defaultMinPartitions):
709709 RDD [(String , String )] = {
710710 assertNotStopped()
@@ -751,7 +751,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
751751 * @note Small files are preferred; very large files may cause bad performance.
752752 */
753753 @ Experimental
754- @ RDDScope
754+ @ RDDScoped
755755 def binaryFiles (path : String , minPartitions : Int = defaultMinPartitions):
756756 RDD [(String , PortableDataStream )] = {
757757 assertNotStopped()
@@ -780,7 +780,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
780780 * @return An RDD of data with values, represented as byte arrays
781781 */
782782 @ Experimental
783- @ RDDScope
783+ @ RDDScoped
784784 def binaryRecords (path : String , recordLength : Int , conf : Configuration = hadoopConfiguration)
785785 : RDD [Array [Byte ]] = {
786786 assertNotStopped()
@@ -818,7 +818,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
818818 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
819819 * copy them using a `map` function.
820820 */
821- @ RDDScope
821+ @ RDDScoped
822822 def hadoopRDD [K , V ](
823823 conf : JobConf ,
824824 inputFormatClass : Class [_ <: InputFormat [K , V ]],
@@ -840,7 +840,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
840840 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
841841 * copy them using a `map` function.
842842 */
843- @ RDDScope
843+ @ RDDScoped
844844 def hadoopFile [K , V ](
845845 path : String ,
846846 inputFormatClass : Class [_ <: InputFormat [K , V ]],
@@ -876,7 +876,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
876876 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
877877 * copy them using a `map` function.
878878 */
879- @ RDDScope
879+ @ RDDScoped
880880 def hadoopFile [K , V , F <: InputFormat [K , V ]]
881881 (path : String , minPartitions : Int )
882882 (implicit km : ClassTag [K ], vm : ClassTag [V ], fm : ClassTag [F ]): RDD [(K , V )] = {
@@ -901,13 +901,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
901901 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
902902 * copy them using a `map` function.
903903 */
904- @ RDDScope
904+ @ RDDScoped
905905 def hadoopFile [K , V , F <: InputFormat [K , V ]](path : String )
906906 (implicit km : ClassTag [K ], vm : ClassTag [V ], fm : ClassTag [F ]): RDD [(K , V )] =
907907 hadoopFile[K , V , F ](path, defaultMinPartitions)
908908
909909 /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
910- @ RDDScope
910+ @ RDDScoped
911911 def newAPIHadoopFile [K , V , F <: NewInputFormat [K , V ]]
912912 (path : String )
913913 (implicit km : ClassTag [K ], vm : ClassTag [V ], fm : ClassTag [F ]): RDD [(K , V )] = {
@@ -928,7 +928,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
928928 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
929929 * copy them using a `map` function.
930930 */
931- @ RDDScope
931+ @ RDDScoped
932932 def newAPIHadoopFile [K , V , F <: NewInputFormat [K , V ]](
933933 path : String ,
934934 fClass : Class [F ],
@@ -962,7 +962,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
962962 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
963963 * copy them using a `map` function.
964964 */
965- @ RDDScope
965+ @ RDDScoped
966966 def newAPIHadoopRDD [K , V , F <: NewInputFormat [K , V ]](
967967 conf : Configuration = hadoopConfiguration,
968968 fClass : Class [F ],
@@ -983,7 +983,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
983983 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
984984 * copy them using a `map` function.
985985 */
986- @ RDDScope
986+ @ RDDScoped
987987 def sequenceFile [K , V ](path : String ,
988988 keyClass : Class [K ],
989989 valueClass : Class [V ],
@@ -1002,7 +1002,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
10021002 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
10031003 * copy them using a `map` function.
10041004 * */
1005- @ RDDScope
1005+ @ RDDScoped
10061006 def sequenceFile [K , V ](path : String , keyClass : Class [K ], valueClass : Class [V ]): RDD [(K , V )] = {
10071007 assertNotStopped()
10081008 sequenceFile(path, keyClass, valueClass, defaultMinPartitions)
@@ -1030,7 +1030,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
10301030 * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
10311031 * copy them using a `map` function.
10321032 */
1033- @ RDDScope
1033+ @ RDDScoped
10341034 def sequenceFile [K , V ]
10351035 (path : String , minPartitions : Int = defaultMinPartitions)
10361036 (implicit km : ClassTag [K ], vm : ClassTag [V ],
@@ -1054,7 +1054,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
10541054 * though the nice thing about it is that there's very little effort required to save arbitrary
10551055 * objects.
10561056 */
1057- @ RDDScope
1057+ @ RDDScoped
10581058 def objectFile [T : ClassTag ](
10591059 path : String ,
10601060 minPartitions : Int = defaultMinPartitions
@@ -1064,15 +1064,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
10641064 .flatMap(x => Utils .deserialize[Array [T ]](x._2.getBytes, Utils .getContextOrSparkClassLoader))
10651065 }
10661066
1067- @ RDDScope
1067+ @ RDDScoped
10681068 protected [spark] def checkpointFile [T : ClassTag ](
10691069 path : String
10701070 ): RDD [T ] = {
10711071 new CheckpointRDD [T ](this , path)
10721072 }
10731073
10741074 /** Build the union of a list of RDDs. */
1075- @ RDDScope
1075+ @ RDDScoped
10761076 def union [T : ClassTag ](rdds : Seq [RDD [T ]]): RDD [T ] = {
10771077 val partitioners = rdds.flatMap(_.partitioner).toSet
10781078 if (partitioners.size == 1 ) {
@@ -1083,7 +1083,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
10831083 }
10841084
10851085 /** Build the union of a list of RDDs passed as variable-length arguments. */
1086- @ RDDScope
1086+ @ RDDScoped
10871087 def union [T : ClassTag ](first : RDD [T ], rest : RDD [T ]* ): RDD [T ] =
10881088 union(Seq (first) ++ rest)
10891089
0 commit comments