Skip to content

Commit 0e40e2b

Browse files
ScrapCodespwendell
authored andcommitted
Deprecated and added a few java api methods for corresponding scala api.
PR [402](https://github.com/apache/incubator-spark/pull/402) from incubator repo. Author: Prashant Sharma <prashant.s@imaginea.com> Closes #19 from ScrapCodes/java-api-completeness and squashes the following commits: 11d0c2b [Prashant Sharma] Integer -> java.lang.Integer 737819a [Prashant Sharma] SPARK-1095 add explicit return types to APIs. 3ddc8bb [Prashant Sharma] Deprected *With functions in scala and added a few missing Java APIs
1 parent 84f7ca1 commit 0e40e2b

File tree

5 files changed

+32
-6
lines changed

5 files changed

+32
-6
lines changed

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -240,6 +240,7 @@ class SparkContext(
240240
localProperties.set(props)
241241
}
242242

243+
@deprecated("Properties no longer need to be explicitly initialized.", "1.0.0")
243244
def initLocalProperties() {
244245
localProperties.set(new Properties())
245246
}
@@ -308,7 +309,7 @@ class SparkContext(
308309
private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this)
309310
private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this)
310311

311-
def initDriverMetrics() {
312+
private def initDriverMetrics() {
312313
SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource)
313314
SparkEnv.get.metricsSystem.registerSource(blockManagerSource)
314315
}

core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
126126
def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] =
127127
wrapRDD(rdd.subtract(other, p))
128128

129+
def generator: String = rdd.generator
130+
129131
override def toString = rdd.toString
130132

131133
/** Assign a name to this RDD */

core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
7474
* of the original partition.
7575
*/
7676
def mapPartitionsWithIndex[R: ClassTag](
77-
f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]],
77+
f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]],
7878
preservesPartitioning: Boolean = false): JavaRDD[R] =
7979
new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))),
8080
preservesPartitioning))

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.api.java
1919

20+
import java.util
2021
import java.util.{Map => JMap}
2122

2223
import scala.collection.JavaConversions
@@ -92,6 +93,24 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
9293

9394
private[spark] val env = sc.env
9495

96+
def isLocal: java.lang.Boolean = sc.isLocal
97+
98+
def sparkUser: String = sc.sparkUser
99+
100+
def master: String = sc.master
101+
102+
def appName: String = sc.appName
103+
104+
def jars: util.List[String] = sc.jars
105+
106+
def startTime: java.lang.Long = sc.startTime
107+
108+
/** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
109+
def defaultParallelism: java.lang.Integer = sc.defaultParallelism
110+
111+
/** Default min number of partitions for Hadoop RDDs when not given by user */
112+
def defaultMinSplits: java.lang.Integer = sc.defaultMinSplits
113+
95114
/** Distribute a local Scala collection to form an RDD. */
96115
def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = {
97116
implicit val ctag: ClassTag[T] = fakeClassTag

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -543,7 +543,8 @@ abstract class RDD[T: ClassTag](
543543
* additional parameter is produced by constructA, which is called in each
544544
* partition with the index of that partition.
545545
*/
546-
def mapWith[A: ClassTag, U: ClassTag]
546+
@deprecated("use mapPartitionsWithIndex", "1.0.0")
547+
def mapWith[A, U: ClassTag]
547548
(constructA: Int => A, preservesPartitioning: Boolean = false)
548549
(f: (T, A) => U): RDD[U] = {
549550
mapPartitionsWithIndex((index, iter) => {
@@ -557,7 +558,8 @@ abstract class RDD[T: ClassTag](
557558
* additional parameter is produced by constructA, which is called in each
558559
* partition with the index of that partition.
559560
*/
560-
def flatMapWith[A: ClassTag, U: ClassTag]
561+
@deprecated("use mapPartitionsWithIndex and flatMap", "1.0.0")
562+
def flatMapWith[A, U: ClassTag]
561563
(constructA: Int => A, preservesPartitioning: Boolean = false)
562564
(f: (T, A) => Seq[U]): RDD[U] = {
563565
mapPartitionsWithIndex((index, iter) => {
@@ -571,7 +573,8 @@ abstract class RDD[T: ClassTag](
571573
* This additional parameter is produced by constructA, which is called in each
572574
* partition with the index of that partition.
573575
*/
574-
def foreachWith[A: ClassTag](constructA: Int => A)(f: (T, A) => Unit) {
576+
@deprecated("use mapPartitionsWithIndex and foreach", "1.0.0")
577+
def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit) {
575578
mapPartitionsWithIndex { (index, iter) =>
576579
val a = constructA(index)
577580
iter.map(t => {f(t, a); t})
@@ -583,7 +586,8 @@ abstract class RDD[T: ClassTag](
583586
* additional parameter is produced by constructA, which is called in each
584587
* partition with the index of that partition.
585588
*/
586-
def filterWith[A: ClassTag](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
589+
@deprecated("use mapPartitionsWithIndex and filter", "1.0.0")
590+
def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
587591
mapPartitionsWithIndex((index, iter) => {
588592
val a = constructA(index)
589593
iter.filter(t => p(t, a))

0 commit comments

Comments
 (0)