-
Notifications
You must be signed in to change notification settings - Fork 370
/
Copy pathredisFunctions.scala
660 lines (609 loc) · 26 KB
/
redisFunctions.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
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
package com.redislabs.provider.redis
import com.redislabs.provider.redis.rdd._
import com.redislabs.provider.redis.util.ConnectionUtils.withConnection
import com.redislabs.provider.redis.util.PipelineUtils._
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import scala.collection.JavaConversions.mapAsJavaMap
import scala.collection.JavaConverters.mapAsJavaMapConverter
/**
* RedisContext extends sparkContext's functionality with redis functions
*
* @param sc a spark context
*/
class RedisContext(@transient val sc: SparkContext) extends Serializable {
val IncorrectKeysOrKeyPatternMsg = "KeysOrKeyPattern should be String or Array[String]"
import com.redislabs.provider.redis.RedisContext._
/**
* @param keyPattern a key pattern to match, or a single key
* @param partitionNum number of partitions
* @return RedisKeysRDD of simple Keys stored in redis server
*/
def fromRedisKeyPattern(keyPattern: String = "*",
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RedisKeysRDD = {
new RedisKeysRDD(sc, redisConfig, readWriteConfig, keyPattern, partitionNum, null)
}
/**
* @param keys an array of keys
* @param partitionNum number of partitions
* @return RedisKeysRDD of simple Keys stored in redis server
*/
def fromRedisKeys(keys: Array[String],
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RedisKeysRDD = {
new RedisKeysRDD(sc, redisConfig, readWriteConfig, "", partitionNum, keys)
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisKVRDD of simple Key-Values stored in redis server
*/
def fromRedisKV[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[(String, String)] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getKV()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getKV()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisListRDD of related values stored in redis server
*/
def fromRedisList[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[String] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getList()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getList()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisZSetRDD of Keys in related ZSets stored in redis server
*/
def fromRedisSet[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[String] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getSet()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getSet()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisHashRDD of related Key-Values stored in redis server
*/
def fromRedisHash[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[(String, String)] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getHash()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getHash()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisZSetRDD of Keys in related ZSets stored in redis server
*/
def fromRedisZSet[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[String] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSet()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSet()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param partitionNum number of partitions
* @return RedisZSetRDD of related Key-Scores stored in redis server
*/
def fromRedisZSetWithScore[T](keysOrKeyPattern: T,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[(String, Double)] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSetWithScore()
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSetWithScore()
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param start start position of target zsets
* @param end end position of target zsets
* @param partitionNum number of partitions
* @return RedisZSetRDD of Keys in related ZSets stored in redis server
*/
def fromRedisZRange[T](keysOrKeyPattern: T,
start: Int,
end: Int,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[String] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSetByRange(start, end)
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSetByRange(start, end)
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param start start position of target zsets
* @param end end position of target zsets
* @param partitionNum number of partitions
* @return RedisZSetRDD of related Key-Scores stored in redis server
*/
def fromRedisZRangeWithScore[T](keysOrKeyPattern: T,
start: Int,
end: Int,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[(String, Double)] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSetByRangeWithScore(start, end)
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSetByRangeWithScore(start, end)
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param min min score of target zsets
* @param max max score of target zsets
* @param partitionNum number of partitions
* @return RedisZSetRDD of Keys in related ZSets stored in redis server
*/
def fromRedisZRangeByScore[T](keysOrKeyPattern: T,
min: Double,
max: Double,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[String] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSetByScore(min, max)
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSetByScore(min, max)
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param keysOrKeyPattern an array of keys or a key pattern
* @param min min score of target zsets
* @param max max score of target zsets
* @param partitionNum number of partitions
* @return RedisZSetRDD of related Key-Scores stored in redis server
*/
def fromRedisZRangeByScoreWithScore[T](keysOrKeyPattern: T,
min: Double,
max: Double,
partitionNum: Int = 3)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)):
RDD[(String, Double)] = {
keysOrKeyPattern match {
case keyPattern: String => fromRedisKeyPattern(keyPattern, partitionNum).getZSetByScoreWithScore(min, max)
case keys: Array[String] => fromRedisKeys(keys, partitionNum).getZSetByScoreWithScore(min, max)
case _ => throw new scala.Exception(IncorrectKeysOrKeyPatternMsg)
}
}
/**
* @param kvs Pair RDD of K/V
* @param ttl time to live
*/
def toRedisKV(kvs: RDD[(String, String)], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setKVs(partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param kvs Pair RDD of K/V
* @param hashName target hash's name which hold all the kvs
* @param ttl time to live
*/
def toRedisHASH(kvs: RDD[(String, String)], hashName: String, ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setHash(hashName, partition, ttl, redisConfig, readWriteConfig))
}
/**
* Write RDD of (hash name, hash KVs)
*
* @param kvs RDD of tuples (hash name, Map(hash field name, hash field value))
* @param ttl time to live
*/
def toRedisHASHes(kvs: RDD[(String, Map[String, String])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setHash(partition, ttl, redisConfig, readWriteConfig))
}
/**
* Write RDD of (hash name, hash KVs). Values are represented as byte array.
*
* @param kvs RDD of tuples (hash name, Map(hash field name, hash field value))
* @param ttl time to live
*/
def toRedisByteHASHes(kvs: RDD[(Array[Byte], Map[Array[Byte], Array[Byte]])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setByteHash(partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param kvs Pair RDD of K/V
* @param zsetName target zset's name which hold all the kvs
* @param ttl time to live
*/
def toRedisZSET(kvs: RDD[(String, String)], zsetName: String, ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setZset(zsetName, partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param kvs Write RDD of (zset name, zset member -> score)
* @param ttl time to live
*/
def toRedisZSETs(kvs: RDD[(String, Map[String, String])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
kvs.foreachPartition(partition => setZset(partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param vs RDD of values
* @param setName target set's name which hold all the vs
* @param ttl time to live
*/
def toRedisSET(vs: RDD[String], setName: String, ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
vs.foreachPartition(partition => setSet(setName, partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param vs RDD of values
* @param listName target list's name which hold all the vs
* @param ttl time to live
*/
def toRedisLIST(vs: RDD[String], listName: String, ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
vs.foreachPartition(partition => setList(listName, partition, ttl, redisConfig, readWriteConfig))
}
/**
* Write RDD of (list name, list values) to Redis Lists.
*
* @param rdd RDD of tuples (list name, list values)
* @param ttl time to live
*/
def toRedisLISTs(rdd: RDD[(String, Seq[String])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
rdd.foreachPartition(partition => setList(partition, ttl, redisConfig, readWriteConfig))
}
/**
* Write RDD of binary values to Redis Lists.
*
* @deprecated use toRedisByteLISTs, the method name has changed to make API consistent
* @param rdd RDD of tuples (list name, list values)
* @param ttl time to live
*/
@Deprecated
def toRedisByteLIST(rdd: RDD[(Array[Byte], Seq[Array[Byte]])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
toRedisByteLISTs(rdd, ttl)(redisConfig, readWriteConfig)
}
/**
* Write RDD of binary values to Redis Lists.
*
* @param rdd RDD of tuples (list name, list values)
* @param ttl time to live
*/
def toRedisByteLISTs(rdd: RDD[(Array[Byte], Seq[Array[Byte]])], ttl: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
rdd.foreachPartition(partition => setByteList(partition, ttl, redisConfig, readWriteConfig))
}
/**
* @param vs RDD of values
* @param listName target list's name which hold all the vs
* @param listSize target list's size
* save all the vs to listName(list type) in redis-server
*/
def toRedisFixedLIST(vs: RDD[String],
listName: String,
listSize: Int = 0)
(implicit
redisConfig: RedisConfig = RedisConfig.fromSparkConf(sc.getConf),
readWriteConfig: ReadWriteConfig = ReadWriteConfig.fromSparkConf(sc.getConf)) {
vs.foreachPartition(partition => setFixedList(listName, listSize, partition, redisConfig, readWriteConfig))
}
}
object RedisContext extends Serializable {
/**
* @param arr k/vs which should be saved in the target host
* save all the k/vs to the target host
* @param ttl time to live
*/
def setKVs(arr: Iterator[(String, String)], ttl: Int, redisConfig: RedisConfig, readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
arr.map(kv => (redisConfig.getHost(kv._1), kv)).toArray.groupBy(_._1).
mapValues(a => a.map(p => p._2)).foreach { x =>
val conn = x._1.endpoint.connect()
foreachWithPipeline(conn, x._2) { case (pipeline, (k, v)) =>
if (ttl <= 0) {
pipeline.set(k, v)
}
else {
pipeline.setex(k, ttl.toLong, v)
}
}
conn.close()
}
}
/**
* @param hashName
* @param arr k/vs which should be saved in the target host
* save all the k/vs to hashName(list type) to the target host
* @param ttl time to live
*/
def setHash(hashName: String, arr: Iterator[(String, String)], ttl: Int, redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
val conn = redisConfig.connectionForKey(hashName)
val pipeline = foreachWithPipelineNoLastSync(conn, arr) { case (pipeline, (k, v)) =>
pipeline.hset(hashName, k, v)
}
if (ttl > 0) pipeline.expire(hashName, ttl.toLong)
pipeline.sync()
conn.close()
}
/**
* @param hashes hashName: map of k/vs to be saved in the target host
* @param ttl time to live
*/
def setHash(hashes: Iterator[(String, Map[String, String])],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
hashes
.map { case (key, hashFields) =>
(redisConfig.getHost(key), (key, hashFields))
}
.toArray
.groupBy(_._1)
.foreach { case (node, arr) =>
withConnection(node.endpoint.connect()) { conn =>
foreachWithPipeline(conn, arr) { (pipeline, a) =>
val (key, hashFields) = a._2
pipeline.hmset(key, hashFields)
if (ttl > 0) pipeline.expire(key, ttl.toLong)
}
}
}
}
/**
* @param hashes hashName: map of k/vs to be saved in the target host
* @param ttl time to live
*/
def setByteHash(hashes: Iterator[(Array[Byte], Map[Array[Byte], Array[Byte]])],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
hashes
.map { case (key, hashFields) =>
(redisConfig.getHost(key), (key, hashFields))
}
.grouped(readWriteConfig.rddWriteIteratorGroupingSize)
.foreach { batch =>
batch
.toArray
.groupBy(_._1)
.foreach { case (node, arr) =>
withConnection(node.endpoint.connect()) { conn =>
foreachWithPipeline(conn, arr) { (pipeline, a) =>
val (key, hashFields) = a._2
pipeline.hmset(key, hashFields)
if (ttl > 0) pipeline.expire(key, ttl.toLong)
}
}
}
}
}
/**
* @param zsetName
* @param arr k/vs which should be saved in the target host
* save all the k/vs to zsetName(zset type) to the target host
* @param ttl time to live
*/
def setZset(zsetName: String, arr: Iterator[(String, String)], ttl: Int, redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
val conn = redisConfig.connectionForKey(zsetName)
val pipeline = foreachWithPipelineNoLastSync(conn, arr) { case (pipeline, (k, v)) =>
pipeline.zadd(zsetName, v.toDouble, k)
}
if (ttl > 0) pipeline.expire(zsetName, ttl.toLong)
pipeline.sync()
conn.close()
}
/**
* @param zsets zsetName: map of member -> score to be saved in the target host
* @param ttl time to live
*/
def setZset(zsets: Iterator[(String, Map[String, String])],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
zsets
.map { case (key, memberScores) =>
(redisConfig.getHost(key), (key, memberScores))
}
.toArray
.groupBy(_._1)
.foreach { case (node, arr) =>
withConnection(node.endpoint.connect()) { conn =>
foreachWithPipeline(conn, arr) { (pipeline, a) =>
val (key, memberScores) = a._2
pipeline.zadd(key, memberScores.mapValues((v) => Double.box(v.toDouble)).asJava)
if (ttl > 0) pipeline.expire(key, ttl.toLong)
}
}
}
}
/**
* @param setName
* @param arr values which should be saved in the target host
* save all the values to setName(set type) to the target host
* @param ttl time to live
*/
def setSet(setName: String, arr: Iterator[String], ttl: Int, redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
val conn = redisConfig.connectionForKey(setName)
val pipeline = foreachWithPipelineNoLastSync(conn, arr) { (pipeline, v) =>
pipeline.sadd(setName, v)
}
if (ttl > 0) pipeline.expire(setName, ttl.toLong)
pipeline.sync()
conn.close()
}
/**
* @param listName
* @param arr values which should be saved in the target host
* save all the values to listName(list type) to the target host
* @param ttl time to live
*/
def setList(listName: String,
arr: Iterator[String],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
val conn = redisConfig.connectionForKey(listName)
val pipeline = foreachWithPipelineNoLastSync(conn, arr) { (pipeline, v) =>
pipeline.rpush(listName, v)
}
if (ttl > 0) pipeline.expire(listName, ttl.toLong)
pipeline.sync()
conn.close()
}
def setByteList(keyValues: Iterator[(Array[Byte], Seq[Array[Byte]])],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
keyValues
.map { case (key, listValues) =>
(redisConfig.getHost(key), (key, listValues))
}
.toArray
.groupBy(_._1)
.foreach { case (node, arr) =>
withConnection(node.endpoint.connect()) { conn =>
foreachWithPipeline(conn, arr) { (pipeline, a) =>
val (key, listVals) = a._2
pipeline.rpush(key, listVals: _*)
if (ttl > 0) pipeline.expire(key, ttl.toLong)
}
}
}
}
def setList(keyValues: Iterator[(String, Seq[String])],
ttl: Int,
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
keyValues
.map { case (key, listValues) =>
(redisConfig.getHost(key), (key, listValues))
}
.toArray
.groupBy(_._1)
.foreach { case (node, arr) =>
withConnection(node.endpoint.connect()) { conn =>
foreachWithPipeline(conn, arr) { (pipeline, a) =>
val (key, listVals) = a._2
pipeline.rpush(key, listVals: _*)
if (ttl > 0) pipeline.expire(key, ttl.toLong)
}
}
}
}
/**
* @param key
* @param listSize
* @param arr values which should be saved in the target host
* save all the values to listName(list type) to the target host
*/
def setFixedList(key: String,
listSize: Int,
arr: Iterator[String],
redisConfig: RedisConfig,
readWriteConfig: ReadWriteConfig) {
implicit val rwConf: ReadWriteConfig = readWriteConfig
val conn = redisConfig.connectionForKey(key)
val pipeline = foreachWithPipelineNoLastSync(conn, arr) { (pipeline, v) =>
pipeline.lpush(key, v)
}
if (listSize > 0) {
pipeline.ltrim(key, 0, listSize - 1)
}
pipeline.sync()
conn.close()
}
}
trait RedisFunctions {
implicit def toRedisContext(sc: SparkContext): RedisContext = new RedisContext(sc)
}