-
Notifications
You must be signed in to change notification settings - Fork 370
/
Copy pathRedisRDD.scala
489 lines (441 loc) · 16.8 KB
/
RedisRDD.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
package com.redislabs.provider.redis.rdd
import java.util
import com.redislabs.provider.redis.partitioner._
import com.redislabs.provider.redis.util.PipelineUtils.mapWithPipeline
import com.redislabs.provider.redis.{ReadWriteConfig, RedisConfig, RedisNode}
import org.apache.spark._
import org.apache.spark.rdd.RDD
import redis.clients.jedis.{Jedis, ScanParams}
import redis.clients.jedis.util.JedisClusterCRC16
import scala.collection.JavaConverters._
import scala.reflect.{ClassTag, classTag}
class RedisKVRDD(prev: RDD[String],
val rddType: String,
implicit val readWriteConfig: ReadWriteConfig)
extends RDD[(String, String)](prev) with Keys {
override def getPartitions: Array[Partition] = prev.partitions
override def compute(split: Partition, context: TaskContext): Iterator[(String, String)] = {
val partition: RedisPartition = split.asInstanceOf[RedisPartition]
val sPos = partition.slots._1
val ePos = partition.slots._2
val nodes = partition.redisConfig.getNodesBySlots(sPos, ePos)
val keys = firstParent[String].iterator(split, context)
val auth = partition.redisConfig.getAuth
val db = partition.redisConfig.getDB
rddType match {
case "kv" => getKV(nodes, keys)
case "hash" => getHASH(nodes, keys)
}
}
def getKV(nodes: Array[RedisNode], keys: Iterator[String]): Iterator[(String, String)] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val stringKeys = filterKeysByType(conn, nodeKeys, "string")
val response = mapWithPipeline(conn, stringKeys) { (pipeline, key) =>
pipeline.get(key)
}
val res = stringKeys.zip(response).iterator.asInstanceOf[Iterator[(String, String)]]
conn.close()
res
}
}
def getHASH(nodes: Array[RedisNode], keys: Iterator[String]): Iterator[(String, String)] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val hashKeys = filterKeysByType(conn, nodeKeys, "hash")
val res = hashKeys.flatMap(conn.hgetAll(_).asScala).iterator
conn.close()
res
}
}
}
class RedisListRDD(prev: RDD[String],
val rddType: String,
implicit val readWriteConfig: ReadWriteConfig) extends RDD[String](prev) with Keys {
override def getPartitions: Array[Partition] = prev.partitions
override def compute(split: Partition, context: TaskContext): Iterator[String] = {
val partition: RedisPartition = split.asInstanceOf[RedisPartition]
val sPos = partition.slots._1
val ePos = partition.slots._2
val nodes = partition.redisConfig.getNodesBySlots(sPos, ePos)
val keys = firstParent[String].iterator(split, context)
rddType match {
case "set" => getSET(nodes, keys)
case "list" => getLIST(nodes, keys)
}
}
def getSET(nodes: Array[RedisNode], keys: Iterator[String]): Iterator[String] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val setKeys = filterKeysByType(conn, nodeKeys, "set")
val res = setKeys.flatMap(conn.smembers(_).asScala).iterator
conn.close()
res
}
}
def getLIST(nodes: Array[RedisNode], keys: Iterator[String]): Iterator[String] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val listKeys = filterKeysByType(conn, nodeKeys, "list")
val res = listKeys.flatMap(conn.lrange(_, 0, -1).asScala).iterator
conn.close()
res
}
}
}
case class ZSetContext(startPos: Long,
endPos: Long,
min: Double,
max: Double,
withScore: Boolean,
typ: String)
class RedisZSetRDD[T: ClassTag](prev: RDD[String],
zsetContext: ZSetContext,
rddType: Class[T],
implicit val readWriteConfig: ReadWriteConfig)
extends RDD[T](prev) with Keys {
override def getPartitions: Array[Partition] = prev.partitions
override def compute(split: Partition, context: TaskContext): Iterator[T] = {
val partition: RedisPartition = split.asInstanceOf[RedisPartition]
val sPos = partition.slots._1
val ePos = partition.slots._2
val nodes = partition.redisConfig.getNodesBySlots(sPos, ePos)
val keys = firstParent[String].iterator(split, context)
val auth = partition.redisConfig.getAuth
val db = partition.redisConfig.getDB
zsetContext.typ match {
case "byRange" => getZSetByRange(nodes, keys, zsetContext.startPos, zsetContext.endPos)
case "byScore" => getZSetByScore(nodes, keys, zsetContext.min, zsetContext.max)
}
}
private def getZSetByRange(nodes: Array[RedisNode],
keys: Iterator[String],
startPos: Long,
endPos: Long): Iterator[T] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val zsetKeys = filterKeysByType(conn, nodeKeys, "zset")
val res = {
if (classTag[T] == classTag[(String, Double)]) {
zsetKeys.flatMap(k => conn.zrangeWithScores(k, startPos, endPos).asScala).
map(tup => (tup.getElement, tup.getScore)).iterator
} else if (classTag[T] == classTag[String]) {
zsetKeys.flatMap(k => conn.zrange(k, startPos, endPos).asScala).iterator
} else {
throw new scala.Exception("Unknown RedisZSetRDD type")
}
}
conn.close()
res
}.asInstanceOf[Iterator[T]]
}
private def getZSetByScore(nodes: Array[RedisNode],
keys: Iterator[String],
startScore: Double,
endScore: Double): Iterator[T] = {
groupKeysByNode(nodes, keys).flatMap { case (node, nodeKeys) =>
val conn = node.endpoint.connect()
val zsetKeys = filterKeysByType(conn, nodeKeys, "zset")
val res = {
if (classTag[T] == classTag[(String, Double)]) {
zsetKeys.flatMap(k => conn.zrangeByScoreWithScores(k, startScore, endScore).asScala).
map(tup => (tup.getElement, tup.getScore)).iterator
} else if (classTag[T] == classTag[String]) {
zsetKeys.flatMap(k => conn.zrangeByScore(k, startScore, endScore).asScala).iterator
} else {
throw new scala.Exception("Unknown RedisZSetRDD type")
}
}
conn.close()
res
}.asInstanceOf[Iterator[T]]
}
}
class RedisKeysRDD(sc: SparkContext,
val redisConfig: RedisConfig,
implicit val readWriteConfig: ReadWriteConfig,
val keyPattern: String = "*",
val partitionNum: Int = 3,
val keys: Array[String] = null)
extends RDD[String](sc, Seq.empty) with Keys {
override protected def getPreferredLocations(split: Partition): Seq[String] = {
Seq(split.asInstanceOf[RedisPartition].redisConfig.initialAddr)
}
/**
* hosts(ip:String, port:Int, startSlot:Int, endSlot:Int) are generated by the redis-cluster's
* hash tragedy and partitionNum to divide the cluster to partitionNum
*
* @return hosts
*/
private def scaleHostsWithPartitionNum(): Seq[(String, Int, Int, Int)] = {
def split(host: RedisNode, cnt: Int) = {
val endpoint = host.endpoint
val start = host.startSlot
val end = host.endSlot
val range = (end - start) / cnt
(0 until cnt).map(i => {
(endpoint.host,
endpoint.port,
if (i == 0) start else (start + range * i + 1),
if (i != cnt - 1) (start + range * (i + 1)) else end)
})
}
val hosts = redisConfig.hosts.sortBy(_.startSlot)
if (hosts.size == partitionNum) {
hosts.map(x => (x.endpoint.host, x.endpoint.port, x.startSlot, x.endSlot))
} else if (hosts.size < partitionNum) {
val presExtCnt = partitionNum / hosts.size
val lastExtCnt = if (presExtCnt * hosts.size < partitionNum) (presExtCnt + 1) else presExtCnt
hosts.zipWithIndex.flatMap {
case (host, idx) => {
split(host, if (idx == hosts.size - 1) lastExtCnt else presExtCnt)
}
}
} else {
val presExtCnt = hosts.size / partitionNum
(0 until partitionNum).map {
idx => {
val ip = hosts(idx * presExtCnt).endpoint.host
val port = hosts(idx * presExtCnt).endpoint.port
val start = hosts(idx * presExtCnt).startSlot
val end = hosts(if (idx == partitionNum - 1) {
(hosts.size - 1)
} else {
((idx + 1) * presExtCnt - 1)
}).endSlot
(ip, port, start, end)
}
}
}
}
override protected def getPartitions: Array[Partition] = {
val hosts = scaleHostsWithPartitionNum()
(0 until partitionNum).map(i => {
new RedisPartition(i,
redisConfig,
(hosts(i)._3, hosts(i)._4)).asInstanceOf[Partition]
}).toArray
}
override def compute(split: Partition, context: TaskContext): Iterator[String] = {
val partition: RedisPartition = split.asInstanceOf[RedisPartition]
val sPos = partition.slots._1
val ePos = partition.slots._2
val nodes = partition.redisConfig.getNodesBySlots(sPos, ePos)
if (Option(this.keys).isDefined) {
this.keys.filter(key => {
val slot = JedisClusterCRC16.getSlot(key)
slot >= sPos && slot <= ePos
}).iterator
} else {
logInfo {
val nodesPassMasked = nodes.map(n => n.copy(endpoint = n.endpoint.maskPassword())).mkString
s"Computing partition, get keys partId: ${partition.index}, [$sPos - $ePos] nodes: $nodesPassMasked"
}
getKeys(nodes, sPos, ePos, keyPattern)
}
}
/**
* filter the 'set' type keys and get all the elements of them
*
* @return RedisSetRDD[String]
*/
def getSet(): RDD[String] = {
new RedisListRDD(this, "set", readWriteConfig)
}
/**
* filter the 'list' type keys and get all the elements of them
*
* @return RedisListRDD[String]
*/
def getList(): RDD[String] = {
new RedisListRDD(this, "list", readWriteConfig)
}
/**
* filter the 'plain k/v' type keys and get all the k/v
*
* @return RedisKVRDD[(String, String)]
*/
def getKV(): RDD[(String, String)] = {
new RedisKVRDD(this, "kv", readWriteConfig)
}
/**
* filter the 'hash' type keys and get all the elements of them
*
* @return RedisHashRDD[(String, String)]
*/
def getHash(): RDD[(String, String)] = {
new RedisKVRDD(this, "hash", readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(without scores) of them
*
* @return RedisZSetRDD[String]
*/
def getZSet(): RDD[String] = {
val zsetContext: ZSetContext = new ZSetContext(0, -1, Double.MinValue, Double.MaxValue, false, "byRange")
new RedisZSetRDD(this, zsetContext, classOf[String], readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(with scores) of them
*
* @return RedisZSetRDD[(String, Double)]
*/
def getZSetWithScore(): RDD[(String, Double)] = {
val zsetContext: ZSetContext = new ZSetContext(0, -1, Double.MinValue, Double.MaxValue, true, "byRange")
new RedisZSetRDD(this, zsetContext, classOf[(String, Double)], readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(without scores) of range [startPos, endPos]
*
* @param startPos start position of zsets
* @param endPos end position of zsets
* @return RedisZSetRDD[String]
*/
def getZSetByRange(startPos: Long, endPos: Long): RDD[String] = {
val zsetContext: ZSetContext = new ZSetContext(startPos, endPos, Double.MinValue, Double.MaxValue, false, "byRange")
new RedisZSetRDD(this, zsetContext, classOf[String], readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(with scores) of range [startPos, endPos]
*
* @param startPos start position of zsets
* @param endPos end position of zsets
* @return RedisZSetRDD[(String, Double)]
*/
def getZSetByRangeWithScore(startPos: Long, endPos: Long): RDD[(String, Double)] = {
val zsetContext: ZSetContext = new ZSetContext(startPos, endPos, Double.MinValue, Double.MaxValue, true, "byRange")
new RedisZSetRDD(this, zsetContext, classOf[(String, Double)], readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(without scores) of score range [min, max]
*
* @param min start position of zsets
* @param max end position of zsets
* @return RedisZSetRDD[String]
*/
def getZSetByScore(min: Double, max: Double): RDD[String] = {
val zsetContext: ZSetContext = new ZSetContext(0, -1, min, max, false, "byScore")
new RedisZSetRDD(this, zsetContext, classOf[String], readWriteConfig)
}
/**
* filter the 'zset' type keys and get all the elements(with scores) of score range [min, max]
*
* @param min start position of zsets
* @param max end position of zsets
* @return RedisZSetRDD[(String, Double)]
*/
def getZSetByScoreWithScore(min: Double, max: Double): RDD[(String, Double)] = {
val zsetContext: ZSetContext = new ZSetContext(0, -1, min, max, true, "byScore")
new RedisZSetRDD(this, zsetContext, classOf[(String, Double)], readWriteConfig)
}
}
trait Keys {
/**
* @param key
* @return true if the key is a RedisRegex
*/
private def isRedisRegex(key: String) = {
def judge(key: String, escape: Boolean): Boolean = {
if (key.length == 0) {
false
} else {
escape match {
case true => judge(key.substring(1), false)
case false => key.charAt(0) match {
case '*' => true
case '?' => true
case '[' => true
case '\\' => judge(key.substring(1), true)
case _ => judge(key.substring(1), false)
}
}
}
}
judge(key, false)
}
/**
* Scan keys, the result may contain duplicates
*
* @param jedis
* @param params
* @return keys of params pattern in jedis
*/
private def scanKeys(jedis: Jedis, params: ScanParams): util.List[String] = {
val keys = new util.ArrayList[String]
var cursor = "0"
do {
val scan = jedis.scan(cursor, params)
keys.addAll(scan.getResult)
cursor = scan.getCursor
} while (cursor != "0")
keys
}
/**
* @param nodes list of RedisNode
* @param sPos start position of slots
* @param ePos end position of slots
* @param keyPattern
* @return keys whose slot is in [sPos, ePos]
*/
def getKeys(nodes: Array[RedisNode],
sPos: Int,
ePos: Int,
keyPattern: String)
(implicit readWriteConfig: ReadWriteConfig): Iterator[String] = {
val endpoints = nodes.map(_.endpoint).distinct
if (isRedisRegex(keyPattern)) {
endpoints.iterator.flatMap { endpoint =>
val keys = new util.HashSet[String]()
val conn = endpoint.connect()
val params = new ScanParams().`match`(keyPattern).count(readWriteConfig.scanCount)
keys.addAll(scanKeys(conn, params).asScala.filter { key =>
val slot = JedisClusterCRC16.getSlot(key)
slot >= sPos && slot <= ePos
}.asJava)
conn.close()
keys.iterator().asScala
}
} else {
val slot = JedisClusterCRC16.getSlot(keyPattern)
if (slot >= sPos && slot <= ePos) Iterator(keyPattern) else Iterator()
}
}
/**
* Master node for a key
*
* @param nodes list of all nodes
* @param key key
* @return master node
*/
def getMasterNode(nodes: Array[RedisNode], key: String): RedisNode = {
val slot = JedisClusterCRC16.getSlot(key)
/* Master only */
nodes.filter { node => node.startSlot <= slot && node.endSlot >= slot }.filter(_.idx == 0)(0)
}
/**
* @param nodes list of RedisNode
* @param keys list of keys
* @return (node: (key1, key2, ...), node2: (key3, key4,...), ...)
*/
def groupKeysByNode(nodes: Array[RedisNode], keys: Iterator[String]): Iterator[(RedisNode, Array[String])] = {
keys.map(key => (getMasterNode(nodes, key), key)).toArray.groupBy(_._1).
map(x => (x._1, x._2.map(_._2))).iterator
}
/**
* @param conn
* @param keys
* keys are guaranteed that they belongs with the server jedis connected to.
* @return keys of "t" type
*/
def filterKeysByType(conn: Jedis, keys: Array[String], t: String)
(implicit readWriteConfig: ReadWriteConfig): Array[String] = {
val types = mapWithPipeline(conn, keys) { (pipeline, key) =>
pipeline.`type`(key)
}
keys.zip(types).filter(x => x._2 == t).map(x => x._1)
}
}
/**
* Key utilities to avoid serialization issues.
*/
object Keys extends Keys