Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 12 additions & 3 deletions core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
*/
def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] =
sample(withReplacement, fraction, Utils.random.nextLong)

/**
* Return a sampled subset of this RDD.
*/
Expand Down Expand Up @@ -195,8 +195,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
*/
@deprecated("Use reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner)", "1.0.0")
def reduceByKey(partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] =
fromRDD(rdd.reduceByKey(partitioner, func))
reduceByKey(func, partitioner)

/**
* Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
*/
def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairRDD[K, V] =
fromRDD(rdd.reduceByKey(func, partitioner))

/**
* Merge the values for each key using an associative reduce function, but return the results
Expand Down Expand Up @@ -374,7 +383,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* parallelism level.
*/
def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = {
fromRDD(reduceByKey(defaultPartitioner(rdd), func))
fromRDD(reduceByKey(func, defaultPartitioner(rdd)))
}

/**
Expand Down
14 changes: 12 additions & 2 deletions core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,17 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
*/
@deprecated("Use reduceByKey(func: (V, V) ⇒ V, partitioner: Partitioner)", "1.0.0")
def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = {
reduceByKey(func, partitioner)
}

/**
* Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
*/
def reduceByKey(func: (V, V) => V, partitioner: Partitioner): RDD[(K, V)] = {
combineByKey[V]((v: V) => v, func, func, partitioner)
}

Expand Down Expand Up @@ -258,7 +268,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
*/
def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = {
reduceByKey(new HashPartitioner(numPartitions), func)
reduceByKey(func, new HashPartitioner(numPartitions))
}

/**
Expand Down Expand Up @@ -359,7 +369,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* parallelism level.
*/
def reduceByKey(func: (V, V) => V): RDD[(K, V)] = {
reduceByKey(defaultPartitioner(self), func)
reduceByKey(func, defaultPartitioner(self))
}

/**
Expand Down
6 changes: 3 additions & 3 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ abstract class RDD[T: ClassTag](
* Return a new RDD containing the distinct elements in this RDD.
*/
def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] =
map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1)
map(x => (x, null.asInstanceOf[T])).reduceByKey((x: T, y: T) => x, numPartitions).map(_._1)

/**
* Return a new RDD containing the distinct elements in this RDD.
Expand Down Expand Up @@ -341,8 +341,8 @@ abstract class RDD[T: ClassTag](
/**
* Return a sampled subset of this RDD.
*/
def sample(withReplacement: Boolean,
fraction: Double,
def sample(withReplacement: Boolean,
fraction: Double,
seed: Long = Utils.random.nextLong): RDD[T] = {
require(fraction >= 0.0, "Invalid fraction value: " + fraction)
if (withReplacement) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging {
// Run the dummy Spark job to ensure that all slaves have registered.
// This avoids all the receivers to be scheduled on the same node.
if (!ssc.sparkContext.isLocal) {
ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey((x: Int, y: Int) => x + y, 20).collect()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line is over 100 chars wide

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rxin will fix this as soon as, a decision is made over whether we want to do this or not.

}

// Distribute the receivers and start them
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import scala.collection.JavaConversions.mapAsScalaMap
private[streaming]
object RawTextHelper {

/**
/**
* Splits lines and counts the words.
*/
def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = {
Expand Down Expand Up @@ -103,7 +103,7 @@ object RawTextHelper {
for(i <- 0 to 1) {
sc.parallelize(1 to 200000, 1000)
.map(_ % 1331).map(_.toString)
.mapPartitions(splitAndCountPartitions).reduceByKey(_ + _, 10)
.mapPartitions(splitAndCountPartitions).reduceByKey((x: Long, y: Long) => x + y, 10)
.count()
}
}
Expand All @@ -114,4 +114,3 @@ object RawTextHelper {

def max(v1: Long, v2: Long) = math.max(v1, v2)
}