@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicIntege
2727import java .util .UUID .randomUUID
2828
2929import scala .collection .JavaConverters ._
30- import scala .collection .{ Map , Set }
30+ import scala .collection .Map
3131import scala .collection .generic .Growable
3232import scala .collection .mutable .HashMap
3333import scala .reflect .{ClassTag , classTag }
@@ -122,20 +122,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
122122 */
123123 def this () = this (new SparkConf ())
124124
125- /**
126- * :: DeveloperApi ::
127- * Alternative constructor for setting preferred locations where Spark will create executors.
128- *
129- * @param config a [[org.apache.spark.SparkConf ]] object specifying other Spark parameters
130- * @param preferredNodeLocationData not used. Left for backward compatibility.
131- */
132- @ deprecated(" Passing in preferred locations has no effect at all, see SPARK-8949" , " 1.5.0" )
133- @ DeveloperApi
134- def this (config : SparkConf , preferredNodeLocationData : Map [String , Set [SplitInfo ]]) = {
135- this (config)
136- logWarning(" Passing in preferred locations has no effect at all, see SPARK-8949" )
137- }
138-
139125 /**
140126 * Alternative constructor that allows setting common Spark properties directly
141127 *
@@ -155,21 +141,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
155141 * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
156142 * system or HDFS, HTTP, HTTPS, or FTP URLs.
157143 * @param environment Environment variables to set on worker nodes.
158- * @param preferredNodeLocationData not used. Left for backward compatibility.
159144 */
160- @ deprecated(" Passing in preferred locations has no effect at all, see SPARK-10921" , " 1.6.0" )
161145 def this (
162146 master : String ,
163147 appName : String ,
164148 sparkHome : String = null ,
165149 jars : Seq [String ] = Nil ,
166- environment : Map [String , String ] = Map (),
167- preferredNodeLocationData : Map [String , Set [SplitInfo ]] = Map ()) =
150+ environment : Map [String , String ] = Map ()) =
168151 {
169152 this (SparkContext .updatedConf(new SparkConf (), master, appName, sparkHome, jars, environment))
170- if (preferredNodeLocationData.nonEmpty) {
171- logWarning(" Passing in preferred locations has no effect at all, see SPARK-8949" )
172- }
173153 }
174154
175155 // NOTE: The below constructors could be consolidated using default arguments. Due to
@@ -267,8 +247,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
267247 // Generate the random name for a temp folder in external block store.
268248 // Add a timestamp as the suffix here to make it more safe
269249 val externalBlockStoreFolderName = " spark-" + randomUUID.toString()
270- @ deprecated(" Use externalBlockStoreFolderName instead." , " 1.4.0" )
271- val tachyonFolderName = externalBlockStoreFolderName
272250
273251 def isLocal : Boolean = (master == " local" || master.startsWith(" local[" ))
274252
@@ -641,11 +619,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
641619 localProperties.set(props)
642620 }
643621
644- @ deprecated(" Properties no longer need to be explicitly initialized." , " 1.0.0" )
645- def initLocalProperties () {
646- localProperties.set(new Properties ())
647- }
648-
649622 /**
650623 * Set a local property that affects jobs submitted from this thread, such as the
651624 * Spark fair scheduler pool.
@@ -1585,15 +1558,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
15851558 taskScheduler.schedulingMode
15861559 }
15871560
1588- /**
1589- * Clear the job's list of files added by `addFile` so that they do not get downloaded to
1590- * any new nodes.
1591- */
1592- @ deprecated(" adding files no longer creates local copies that need to be deleted" , " 1.0.0" )
1593- def clearFiles () {
1594- addedFiles.clear()
1595- }
1596-
15971561 /**
15981562 * Gets the locality information associated with the partition in a particular rdd
15991563 * @param rdd of interest
@@ -1685,15 +1649,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
16851649 postEnvironmentUpdate()
16861650 }
16871651
1688- /**
1689- * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to
1690- * any new nodes.
1691- */
1692- @ deprecated(" adding jars no longer creates local copies that need to be deleted" , " 1.0.0" )
1693- def clearJars () {
1694- addedJars.clear()
1695- }
1696-
16971652 // Shut down the SparkContext.
16981653 def stop () {
16991654 if (AsynchronousListenerBus .withinListenerThread.value) {
@@ -1864,63 +1819,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
18641819 runJob(rdd, (ctx : TaskContext , it : Iterator [T ]) => cleanedFunc(it), partitions)
18651820 }
18661821
1867-
1868- /**
1869- * Run a function on a given set of partitions in an RDD and pass the results to the given
1870- * handler function. This is the main entry point for all actions in Spark.
1871- *
1872- * The allowLocal flag is deprecated as of Spark 1.5.0+.
1873- */
1874- @ deprecated(" use the version of runJob without the allowLocal parameter" , " 1.5.0" )
1875- def runJob [T , U : ClassTag ](
1876- rdd : RDD [T ],
1877- func : (TaskContext , Iterator [T ]) => U ,
1878- partitions : Seq [Int ],
1879- allowLocal : Boolean ,
1880- resultHandler : (Int , U ) => Unit ): Unit = {
1881- if (allowLocal) {
1882- logWarning(" sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+" )
1883- }
1884- runJob(rdd, func, partitions, resultHandler)
1885- }
1886-
1887- /**
1888- * Run a function on a given set of partitions in an RDD and return the results as an array.
1889- *
1890- * The allowLocal flag is deprecated as of Spark 1.5.0+.
1891- */
1892- @ deprecated(" use the version of runJob without the allowLocal parameter" , " 1.5.0" )
1893- def runJob [T , U : ClassTag ](
1894- rdd : RDD [T ],
1895- func : (TaskContext , Iterator [T ]) => U ,
1896- partitions : Seq [Int ],
1897- allowLocal : Boolean
1898- ): Array [U ] = {
1899- if (allowLocal) {
1900- logWarning(" sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+" )
1901- }
1902- runJob(rdd, func, partitions)
1903- }
1904-
1905- /**
1906- * Run a job on a given set of partitions of an RDD, but take a function of type
1907- * `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
1908- *
1909- * The allowLocal argument is deprecated as of Spark 1.5.0+.
1910- */
1911- @ deprecated(" use the version of runJob without the allowLocal parameter" , " 1.5.0" )
1912- def runJob [T , U : ClassTag ](
1913- rdd : RDD [T ],
1914- func : Iterator [T ] => U ,
1915- partitions : Seq [Int ],
1916- allowLocal : Boolean
1917- ): Array [U ] = {
1918- if (allowLocal) {
1919- logWarning(" sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+" )
1920- }
1921- runJob(rdd, func, partitions)
1922- }
1923-
19241822 /**
19251823 * Run a job on all partitions in an RDD and return the results in an array.
19261824 */
@@ -2094,10 +1992,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
20941992 taskScheduler.defaultParallelism
20951993 }
20961994
2097- /** Default min number of partitions for Hadoop RDDs when not given by user */
2098- @ deprecated(" use defaultMinPartitions" , " 1.0.0" )
2099- def defaultMinSplits : Int = defaultMinPartitions
2100-
21011995 /**
21021996 * Default min number of partitions for Hadoop RDDs when not given by user
21031997 * Notice that we use math.min so the "defaultMinPartitions" cannot be higher than 2.
@@ -2364,113 +2258,6 @@ object SparkContext extends Logging {
23642258 */
23652259 private [spark] val LEGACY_DRIVER_IDENTIFIER = " <driver>"
23662260
2367- // The following deprecated objects have already been copied to `object AccumulatorParam` to
2368- // make the compiler find them automatically. They are duplicate codes only for backward
2369- // compatibility, please update `object AccumulatorParam` accordingly if you plan to modify the
2370- // following ones.
2371-
2372- @ deprecated(" Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
2373- " backward compatibility." , " 1.3.0" )
2374- object DoubleAccumulatorParam extends AccumulatorParam [Double ] {
2375- def addInPlace (t1 : Double , t2 : Double ): Double = t1 + t2
2376- def zero (initialValue : Double ): Double = 0.0
2377- }
2378-
2379- @ deprecated(" Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
2380- " backward compatibility." , " 1.3.0" )
2381- object IntAccumulatorParam extends AccumulatorParam [Int ] {
2382- def addInPlace (t1 : Int , t2 : Int ): Int = t1 + t2
2383- def zero (initialValue : Int ): Int = 0
2384- }
2385-
2386- @ deprecated(" Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
2387- " backward compatibility." , " 1.3.0" )
2388- object LongAccumulatorParam extends AccumulatorParam [Long ] {
2389- def addInPlace (t1 : Long , t2 : Long ): Long = t1 + t2
2390- def zero (initialValue : Long ): Long = 0L
2391- }
2392-
2393- @ deprecated(" Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
2394- " backward compatibility." , " 1.3.0" )
2395- object FloatAccumulatorParam extends AccumulatorParam [Float ] {
2396- def addInPlace (t1 : Float , t2 : Float ): Float = t1 + t2
2397- def zero (initialValue : Float ): Float = 0f
2398- }
2399-
2400- // The following deprecated functions have already been moved to `object RDD` to
2401- // make the compiler find them automatically. They are still kept here for backward compatibility
2402- // and just call the corresponding functions in `object RDD`.
2403-
2404- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2405- " kept here only for backward compatibility." , " 1.3.0" )
2406- def rddToPairRDDFunctions [K , V ](rdd : RDD [(K , V )])
2407- (implicit kt : ClassTag [K ], vt : ClassTag [V ], ord : Ordering [K ] = null ): PairRDDFunctions [K , V ] =
2408- RDD .rddToPairRDDFunctions(rdd)
2409-
2410- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2411- " kept here only for backward compatibility." , " 1.3.0" )
2412- def rddToAsyncRDDActions [T : ClassTag ](rdd : RDD [T ]): AsyncRDDActions [T ] =
2413- RDD .rddToAsyncRDDActions(rdd)
2414-
2415- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2416- " kept here only for backward compatibility." , " 1.3.0" )
2417- def rddToSequenceFileRDDFunctions [K <% Writable : ClassTag , V <% Writable : ClassTag ](
2418- rdd : RDD [(K , V )]): SequenceFileRDDFunctions [K , V ] = {
2419- val kf = implicitly[K => Writable ]
2420- val vf = implicitly[V => Writable ]
2421- // Set the Writable class to null and `SequenceFileRDDFunctions` will use Reflection to get it
2422- implicit val keyWritableFactory = new WritableFactory [K ](_ => null , kf)
2423- implicit val valueWritableFactory = new WritableFactory [V ](_ => null , vf)
2424- RDD .rddToSequenceFileRDDFunctions(rdd)
2425- }
2426-
2427- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2428- " kept here only for backward compatibility." , " 1.3.0" )
2429- def rddToOrderedRDDFunctions [K : Ordering : ClassTag , V : ClassTag ](
2430- rdd : RDD [(K , V )]): OrderedRDDFunctions [K , V , (K , V )] =
2431- RDD .rddToOrderedRDDFunctions(rdd)
2432-
2433- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2434- " kept here only for backward compatibility." , " 1.3.0" )
2435- def doubleRDDToDoubleRDDFunctions (rdd : RDD [Double ]): DoubleRDDFunctions =
2436- RDD .doubleRDDToDoubleRDDFunctions(rdd)
2437-
2438- @ deprecated(" Replaced by implicit functions in the RDD companion object. This is " +
2439- " kept here only for backward compatibility." , " 1.3.0" )
2440- def numericRDDToDoubleRDDFunctions [T ](rdd : RDD [T ])(implicit num : Numeric [T ]): DoubleRDDFunctions =
2441- RDD .numericRDDToDoubleRDDFunctions(rdd)
2442-
2443- // The following deprecated functions have already been moved to `object WritableFactory` to
2444- // make the compiler find them automatically. They are still kept here for backward compatibility.
2445-
2446- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2447- " kept here only for backward compatibility." , " 1.3.0" )
2448- implicit def intToIntWritable (i : Int ): IntWritable = new IntWritable (i)
2449-
2450- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2451- " kept here only for backward compatibility." , " 1.3.0" )
2452- implicit def longToLongWritable (l : Long ): LongWritable = new LongWritable (l)
2453-
2454- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2455- " kept here only for backward compatibility." , " 1.3.0" )
2456- implicit def floatToFloatWritable (f : Float ): FloatWritable = new FloatWritable (f)
2457-
2458- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2459- " kept here only for backward compatibility." , " 1.3.0" )
2460- implicit def doubleToDoubleWritable (d : Double ): DoubleWritable = new DoubleWritable (d)
2461-
2462- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2463- " kept here only for backward compatibility." , " 1.3.0" )
2464- implicit def boolToBoolWritable (b : Boolean ): BooleanWritable = new BooleanWritable (b)
2465-
2466- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2467- " kept here only for backward compatibility." , " 1.3.0" )
2468- implicit def bytesToBytesWritable (aob : Array [Byte ]): BytesWritable = new BytesWritable (aob)
2469-
2470- @ deprecated(" Replaced by implicit functions in the WritableFactory companion object. This is " +
2471- " kept here only for backward compatibility." , " 1.3.0" )
2472- implicit def stringToText (s : String ): Text = new Text (s)
2473-
24742261 private implicit def arrayToArrayWritable [T <% Writable : ClassTag ](arr : Traversable [T ])
24752262 : ArrayWritable = {
24762263 def anyToWritable [U <% Writable ](u : U ): Writable = u
@@ -2479,50 +2266,6 @@ object SparkContext extends Logging {
24792266 arr.map(x => anyToWritable(x)).toArray)
24802267 }
24812268
2482- // The following deprecated functions have already been moved to `object WritableConverter` to
2483- // make the compiler find them automatically. They are still kept here for backward compatibility
2484- // and just call the corresponding functions in `object WritableConverter`.
2485-
2486- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2487- " backward compatibility." , " 1.3.0" )
2488- def intWritableConverter (): WritableConverter [Int ] =
2489- WritableConverter .intWritableConverter()
2490-
2491- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2492- " backward compatibility." , " 1.3.0" )
2493- def longWritableConverter (): WritableConverter [Long ] =
2494- WritableConverter .longWritableConverter()
2495-
2496- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2497- " backward compatibility." , " 1.3.0" )
2498- def doubleWritableConverter (): WritableConverter [Double ] =
2499- WritableConverter .doubleWritableConverter()
2500-
2501- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2502- " backward compatibility." , " 1.3.0" )
2503- def floatWritableConverter (): WritableConverter [Float ] =
2504- WritableConverter .floatWritableConverter()
2505-
2506- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2507- " backward compatibility." , " 1.3.0" )
2508- def booleanWritableConverter (): WritableConverter [Boolean ] =
2509- WritableConverter .booleanWritableConverter()
2510-
2511- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2512- " backward compatibility." , " 1.3.0" )
2513- def bytesWritableConverter (): WritableConverter [Array [Byte ]] =
2514- WritableConverter .bytesWritableConverter()
2515-
2516- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2517- " backward compatibility." , " 1.3.0" )
2518- def stringWritableConverter (): WritableConverter [String ] =
2519- WritableConverter .stringWritableConverter()
2520-
2521- @ deprecated(" Replaced by implicit functions in WritableConverter. This is kept here only for " +
2522- " backward compatibility." , " 1.3.0" )
2523- def writableWritableConverter [T <: Writable ](): WritableConverter [T ] =
2524- WritableConverter .writableWritableConverter()
2525-
25262269 /**
25272270 * Find the JAR from which a given class was loaded, to make it easy for users to pass
25282271 * their JARs to SparkContext.
0 commit comments