Skip to content

Commit 8ce645d

Browse files
rxinJoshRosen
authored andcommitted
[SPARK-12615] Remove some deprecated APIs in RDD/SparkContext
I looked at each case individually and it looks like they can all be removed. The only one that I had to think twice was toArray (I even thought about un-deprecating it, until I realized it was a problem in Java to have toArray returning java.util.List). Author: Reynold Xin <[email protected]> Closes #10569 from rxin/SPARK-12615.
1 parent 7676833 commit 8ce645d

File tree

22 files changed

+64
-643
lines changed

22 files changed

+64
-643
lines changed

core/src/main/scala/org/apache/spark/Aggregator.scala

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -34,10 +34,6 @@ case class Aggregator[K, V, C] (
3434
mergeValue: (C, V) => C,
3535
mergeCombiners: (C, C) => C) {
3636

37-
@deprecated("use combineValuesByKey with TaskContext argument", "0.9.0")
38-
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] =
39-
combineValuesByKey(iter, null)
40-
4137
def combineValuesByKey(
4238
iter: Iterator[_ <: Product2[K, V]],
4339
context: TaskContext): Iterator[(K, C)] = {
@@ -47,10 +43,6 @@ case class Aggregator[K, V, C] (
4743
combiners.iterator
4844
}
4945

50-
@deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0")
51-
def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]]) : Iterator[(K, C)] =
52-
combineCombinersByKey(iter, null)
53-
5446
def combineCombinersByKey(
5547
iter: Iterator[_ <: Product2[K, C]],
5648
context: TaskContext): Iterator[(K, C)] = {

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 2 additions & 259 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicIntege
2727
import java.util.UUID.randomUUID
2828

2929
import scala.collection.JavaConverters._
30-
import scala.collection.{Map, Set}
30+
import scala.collection.Map
3131
import scala.collection.generic.Growable
3232
import scala.collection.mutable.HashMap
3333
import 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

Comments
 (0)