@@ -118,6 +118,56 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
118118 combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner (numPartitions))
119119 }
120120
121+ /**
122+ * Aggregate the values of each key, using given combine functions and a neutral "zero value".
123+ * This function can return a different result type, U, than the type of the values in this RDD,
124+ * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's,
125+ * as in scala.TraversableOnce. The former operation is used for merging values within a
126+ * partition, and the latter is used for merging values between partitions. To avoid memory
127+ * allocation, both of these functions are allowed to modify and return their first argument
128+ * instead of creating a new U.
129+ */
130+ def aggregateByKey [U : ClassTag ](zeroValue : U , partitioner : Partitioner )(seqOp : (U , V ) => U ,
131+ combOp : (U , U ) => U ): RDD [(K , U )] = {
132+ // Serialize the zero value to a byte array so that we can get a new clone of it on each key
133+ val zeroBuffer = SparkEnv .get.closureSerializer.newInstance().serialize(zeroValue)
134+ val zeroArray = new Array [Byte ](zeroBuffer.limit)
135+ zeroBuffer.get(zeroArray)
136+
137+ lazy val cachedSerializer = SparkEnv .get.closureSerializer.newInstance()
138+ def createZero () = cachedSerializer.deserialize[U ](ByteBuffer .wrap(zeroArray))
139+
140+ combineByKey[U ]((v : V ) => seqOp(createZero(), v), seqOp, combOp, partitioner)
141+ }
142+
143+ /**
144+ * Aggregate the values of each key, using given combine functions and a neutral "zero value".
145+ * This function can return a different result type, U, than the type of the values in this RDD,
146+ * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's,
147+ * as in scala.TraversableOnce. The former operation is used for merging values within a
148+ * partition, and the latter is used for merging values between partitions. To avoid memory
149+ * allocation, both of these functions are allowed to modify and return their first argument
150+ * instead of creating a new U.
151+ */
152+ def aggregateByKey [U : ClassTag ](zeroValue : U , numPartitions : Int )(seqOp : (U , V ) => U ,
153+ combOp : (U , U ) => U ): RDD [(K , U )] = {
154+ aggregateByKey(zeroValue, new HashPartitioner (numPartitions))(seqOp, combOp)
155+ }
156+
157+ /**
158+ * Aggregate the values of each key, using given combine functions and a neutral "zero value".
159+ * This function can return a different result type, U, than the type of the values in this RDD,
160+ * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's,
161+ * as in scala.TraversableOnce. The former operation is used for merging values within a
162+ * partition, and the latter is used for merging values between partitions. To avoid memory
163+ * allocation, both of these functions are allowed to modify and return their first argument
164+ * instead of creating a new U.
165+ */
166+ def aggregateByKey [U : ClassTag ](zeroValue : U )(seqOp : (U , V ) => U ,
167+ combOp : (U , U ) => U ): RDD [(K , U )] = {
168+ aggregateByKey(zeroValue, defaultPartitioner(self))(seqOp, combOp)
169+ }
170+
121171 /**
122172 * Merge the values for each key using an associative function and a neutral "zero value" which
123173 * may be added to the result an arbitrary number of times, and must not change the result
0 commit comments