@@ -2,7 +2,7 @@ package com.redislabs.provider.redis.rdd
22
33import java .util
44
5- import com .redislabs .provider .redis .{RedisNode , RedisEndpoint , RedisConfig }
5+ import com .redislabs .provider .redis .{RedisNode , RedisConfig }
66import org .apache .spark .rdd .RDD
77import org .apache .spark ._
88import redis .clients .jedis ._
@@ -33,9 +33,7 @@ class RedisKVRDD(prev: RDD[String],
3333 }
3434 }
3535
36- def getKV (nodes : Array [RedisNode ],
37- keys : Iterator [String ]): Iterator [(String , String )] = {
38-
36+ def getKV (nodes : Array [RedisNode ], keys : Iterator [String ]): Iterator [(String , String )] = {
3937 groupKeysByNode(nodes, keys).flatMap {
4038 x =>
4139 {
@@ -50,8 +48,7 @@ class RedisKVRDD(prev: RDD[String],
5048 }
5149 }.iterator
5250 }
53- def getHASH (nodes : Array [RedisNode ],
54- keys : Iterator [String ]): Iterator [(String , String )] = {
51+ def getHASH (nodes : Array [RedisNode ], keys : Iterator [String ]): Iterator [(String , String )] = {
5552 groupKeysByNode(nodes, keys).flatMap {
5653 x =>
5754 {
@@ -63,8 +60,7 @@ class RedisKVRDD(prev: RDD[String],
6360 }
6461 }.iterator
6562 }
66- def getZSET (nodes : Array [RedisNode ],
67- keys : Iterator [String ]): Iterator [(String , String )] = {
63+ def getZSET (nodes : Array [RedisNode ], keys : Iterator [String ]): Iterator [(String , String )] = {
6864 groupKeysByNode(nodes, keys).flatMap {
6965 x =>
7066 {
@@ -79,9 +75,7 @@ class RedisKVRDD(prev: RDD[String],
7975 }
8076}
8177
82- class RedisListRDD (prev : RDD [String ],
83- val rddType : String )
84- extends RDD [String ](prev) with Keys {
78+ class RedisListRDD (prev : RDD [String ], val rddType : String ) extends RDD [String ](prev) with Keys {
8579
8680 override def getPartitions : Array [Partition ] = prev.partitions
8781
@@ -97,8 +91,7 @@ class RedisListRDD(prev: RDD[String],
9791 }
9892 }
9993
100- def getSET (nodes : Array [RedisNode ],
101- keys : Iterator [String ]): Iterator [String ] = {
94+ def getSET (nodes : Array [RedisNode ], keys : Iterator [String ]): Iterator [String ] = {
10295 groupKeysByNode(nodes, keys).flatMap {
10396 x =>
10497 {
@@ -110,8 +103,7 @@ class RedisListRDD(prev: RDD[String],
110103 }
111104 }.iterator
112105 }
113- def getLIST (nodes : Array [RedisNode ],
114- keys : Iterator [String ]): Iterator [String ] = {
106+ def getLIST (nodes : Array [RedisNode ], keys : Iterator [String ]): Iterator [String ] = {
115107 groupKeysByNode(nodes, keys).flatMap {
116108 x =>
117109 {
@@ -132,17 +124,16 @@ class RedisKeysRDD(sc: SparkContext,
132124 val keys : Array [String ] = null )
133125 extends RDD [String ](sc, Seq .empty) with Logging with Keys {
134126
135-
136127 override protected def getPreferredLocations (split : Partition ): Seq [String ] = {
137128 Seq (split.asInstanceOf [RedisPartition ].redisConfig.currentAddr)
138129 }
139130
140131 /**
141- * hosts(ip:String, port:Int, startSlot:Int, endSlot:Int) are generated by the redis-cluster's
142- * hash tragedy and partitionNum to divied the cluster to partitionNum
143- *
144- * @return hosts
145- */
132+ * hosts(ip:String, port:Int, startSlot:Int, endSlot:Int) are generated by the redis-cluster's
133+ * hash tragedy and partitionNum to divide the cluster to partitionNum
134+ *
135+ * @return hosts
136+ */
146137 private def scaleHostsWithPartitionNum (): Seq [(String , Int , Int , Int )] = {
147138 def split (host : RedisNode , cnt : Int ) = {
148139 val endpoint = host.endpoint
@@ -205,7 +196,7 @@ class RedisKeysRDD(sc: SparkContext,
205196 if (Option (this .keys).isDefined) {
206197 this .keys.iterator
207198 } else {
208- getKeys(nodes, sPos, ePos, keyPattern, partition.redisConfig ).iterator
199+ getKeys(nodes, sPos, ePos, keyPattern).iterator
209200 }
210201
211202 }
@@ -228,9 +219,9 @@ class RedisKeysRDD(sc: SparkContext,
228219
229220trait Keys {
230221 /**
231- * @param key
232- * @return true if the key is a RedisRegex
233- */
222+ * @param key
223+ * @return true if the key is a RedisRegex
224+ */
234225 private def isRedisRegex (key : String ) = {
235226 def judge (key : String , escape : Boolean ): Boolean = {
236227 if (key.length == 0 ) {
@@ -252,10 +243,10 @@ trait Keys {
252243 }
253244
254245 /**
255- * @param jedis
256- * @param params
257- * @return keys of params pattern in jedis
258- */
246+ * @param jedis
247+ * @param params
248+ * @return keys of params pattern in jedis
249+ */
259250 private def scanKeys (jedis : Jedis , params : ScanParams ): util.HashSet [String ] = {
260251 val keys = new util.HashSet [String ]
261252 var cursor = " 0"
@@ -268,17 +259,16 @@ trait Keys {
268259 }
269260
270261 /**
271- * @param nodes list of (IP:String, port:Int, index:Int, range:Int, startSlot:Int, endSlot:Int)
272- * @param sPos start position of slots
273- * @param ePos end position of slots
274- * @param keyPattern
275- * return keys whose slot is in [sPos, ePos]
276- */
262+ * @param nodes list of RedisNode
263+ * @param sPos start position of slots
264+ * @param ePos end position of slots
265+ * @param keyPattern
266+ * return keys whose slot is in [sPos, ePos]
267+ */
277268 def getKeys (nodes : Array [RedisNode ],
278269 sPos : Int ,
279270 ePos : Int ,
280- keyPattern : String ,
281- config : RedisConfig ): util.HashSet [String ] = {
271+ keyPattern : String ): util.HashSet [String ] = {
282272 val keys = new util.HashSet [String ]()
283273 if (isRedisRegex(keyPattern)) {
284274 nodes.foreach(node => {
@@ -299,10 +289,10 @@ trait Keys {
299289 }
300290
301291 /**
302- * @param nodes list of (IP:String, port:Int, index:Int, range:Int, startSlot:Int, endSlot:Int)
303- * @param keys list of keys
304- * return (node: (key1, key2, ...), node2: (key3, key4,...), ...)
305- */
292+ * @param nodes list of RedisNode
293+ * @param keys list of keys
294+ * return (node: (key1, key2, ...), node2: (key3, key4,...), ...)
295+ */
306296 def groupKeysByNode (nodes : Array [RedisNode ], keys : Iterator [String ]):
307297 Array [(RedisNode , Array [String ])] = {
308298 def getNode (key : String ): RedisNode = {
@@ -315,11 +305,11 @@ trait Keys {
315305 }
316306
317307 /**
318- * @param conn
319- * @param keys
320- * keys are guaranteed that they belongs with the server jedis connected to.
321- * Filter all the keys of "t" type.
322- */
308+ * @param conn
309+ * @param keys
310+ * keys are guaranteed that they belongs with the server jedis connected to.
311+ * return keys of "t" type
312+ */
323313 def filterKeysByType (conn : Jedis , keys: Array [String ], t: String ): Array [String ] = {
324314 val pipeline = conn.pipelined
325315 keys.foreach(pipeline.`type`)
0 commit comments