Skip to content

Commit 8720b69

Browse files
authored
Merge pull request #167 from fe2s/split-implicits
separate core(rdd) and streaming implicits to separate packages, so t…
2 parents 7a1a658 + 1ca81b2 commit 8720b69

File tree

6 files changed

+70
-55
lines changed

6 files changed

+70
-55
lines changed

doc/getting-started.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,7 @@ df.write
9191
### Create Stream
9292

9393
```scala
94-
import com.redislabs.provider.redis._
94+
import com.redislabs.provider.redis.streaming._
9595

9696
val ssc = new StreamingContext(sc, Seconds(1))
9797
val redisStream = ssc.createRedisStream(Array("foo", "bar"),

doc/streaming.md

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@ Spark-Redis supports streaming data from Stream and List data structures:
1010
To stream data from [Redis Stream](https://redis.io/topics/streams-intro) use `createRedisXStream` method (added in Spark-Redis 2.3.1):
1111

1212
```scala
13-
import com.redislabs.provider.redis._
13+
import com.redislabs.provider.redis.streaming._
1414
import com.redislabs.provider.redis.streaming.{ConsumerConfig, StreamItem}
1515
import org.apache.spark.sql.SparkSession
1616
import org.apache.spark.streaming.dstream.InputDStream
@@ -115,7 +115,7 @@ Use the following to get a `(listName, value)` stream from `foo` and `bar` list
115115
```scala
116116
import org.apache.spark.streaming.{Seconds, StreamingContext}
117117
import org.apache.spark.storage.StorageLevel
118-
import com.redislabs.provider.redis._
118+
import com.redislabs.provider.redis.streaming._
119119
val ssc = new StreamingContext(sc, Seconds(1))
120120
val redisStream = ssc.createRedisStream(Array("foo", "bar"), storageLevel = StorageLevel.MEMORY_AND_DISK_2)
121121
redisStream.print()
@@ -128,7 +128,7 @@ Use the following to get a `value` stream from `foo` and `bar` list
128128
```scala
129129
import org.apache.spark.streaming.{Seconds, StreamingContext}
130130
import org.apache.spark.storage.StorageLevel
131-
import com.redislabs.provider.redis._
131+
import com.redislabs.provider.redis.streaming._
132132
val ssc = new StreamingContext(sc, Seconds(1))
133133
val redisStream = ssc.createRedisStreamWithoutListname(Array("foo", "bar"), storageLevel = StorageLevel.MEMORY_AND_DISK_2)
134134
redisStream.print()

src/main/scala/com/redislabs/provider/redis/redisFunctions.scala

Lines changed: 3 additions & 50 deletions
Original file line numberDiff line numberDiff line change
@@ -1,13 +1,9 @@
11
package com.redislabs.provider.redis
22

3-
import com.redislabs.provider.redis.streaming.{ConsumerConfig, RedisInputDStream, RedisStreamReceiver, StreamItem}
4-
import org.apache.spark.SparkContext
5-
import org.apache.spark.rdd.RDD
63
import com.redislabs.provider.redis.rdd._
74
import com.redislabs.provider.redis.util.PipelineUtils._
8-
import org.apache.spark.storage.StorageLevel
9-
import org.apache.spark.streaming.StreamingContext
10-
import org.apache.spark.streaming.dstream.InputDStream
5+
import org.apache.spark.SparkContext
6+
import org.apache.spark.rdd.RDD
117

128
/**
139
* RedisContext extends sparkContext's functionality with redis functions
@@ -443,52 +439,9 @@ object RedisContext extends Serializable {
443439
}
444440
}
445441

446-
/**
447-
* RedisStreamingContext extends StreamingContext's functionality with Redis
448-
*
449-
* @param ssc a spark StreamingContext
450-
*/
451-
class RedisStreamingContext(@transient val ssc: StreamingContext) extends Serializable {
452-
/**
453-
* @param keys an Array[String] which consists all the Lists we want to listen to
454-
* @param storageLevel the receiver' storage tragedy of received data, default as MEMORY_AND_DISK_2
455-
* @return a stream of (listname, value)
456-
*/
457-
def createRedisStream(keys: Array[String],
458-
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
459-
(implicit
460-
redisConfig: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
461-
RedisInputDStream[(String, String)] = {
462-
new RedisInputDStream(ssc, keys, storageLevel, redisConfig, classOf[(String, String)])
463-
}
464-
465-
/**
466-
* @param keys an Array[String] which consists all the Lists we want to listen to
467-
* @param storageLevel the receiver' storage tragedy of received data, default as MEMORY_AND_DISK_2
468-
* @return a stream of (value)
469-
*/
470-
def createRedisStreamWithoutListname(keys: Array[String],
471-
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
472-
(implicit
473-
redisConf: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
474-
RedisInputDStream[String] = {
475-
new RedisInputDStream(ssc, keys, storageLevel, redisConf, classOf[String])
476-
}
477-
478-
def createRedisXStream(consumersConfig: Seq[ConsumerConfig],
479-
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
480-
(implicit
481-
redisConfig: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
482-
InputDStream[StreamItem] = {
483-
val readWriteConfig = ReadWriteConfig.fromSparkConf(ssc.sparkContext.getConf)
484-
val receiver = new RedisStreamReceiver(consumersConfig, redisConfig, readWriteConfig, storageLevel)
485-
ssc.receiverStream(receiver)
486-
}
487-
}
488-
489442
trait RedisFunctions {
443+
490444
implicit def toRedisContext(sc: SparkContext): RedisContext = new RedisContext(sc)
491445

492-
implicit def toRedisStreamingContext(ssc: StreamingContext): RedisStreamingContext = new RedisStreamingContext(ssc)
493446
}
494447

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
package com.redislabs.provider.redis
2+
3+
package object streaming extends RedisStreamingFunctions {
4+
5+
}
Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
1+
package com.redislabs.provider.redis.streaming
2+
3+
import com.redislabs.provider.redis.{ReadWriteConfig, RedisConfig}
4+
import org.apache.spark.storage.StorageLevel
5+
import org.apache.spark.streaming.StreamingContext
6+
import org.apache.spark.streaming.dstream.InputDStream
7+
8+
/**
9+
* RedisStreamingContext extends StreamingContext's functionality with Redis
10+
*
11+
* @param ssc a spark StreamingContext
12+
*/
13+
class RedisStreamingContext(@transient val ssc: StreamingContext) extends Serializable {
14+
/**
15+
* @param keys an Array[String] which consists all the Lists we want to listen to
16+
* @param storageLevel the receiver' storage tragedy of received data, default as MEMORY_AND_DISK_2
17+
* @return a stream of (listname, value)
18+
*/
19+
def createRedisStream(keys: Array[String],
20+
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
21+
(implicit
22+
redisConfig: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
23+
RedisInputDStream[(String, String)] = {
24+
new RedisInputDStream(ssc, keys, storageLevel, redisConfig, classOf[(String, String)])
25+
}
26+
27+
/**
28+
* @param keys an Array[String] which consists all the Lists we want to listen to
29+
* @param storageLevel the receiver' storage tragedy of received data, default as MEMORY_AND_DISK_2
30+
* @return a stream of (value)
31+
*/
32+
def createRedisStreamWithoutListname(keys: Array[String],
33+
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
34+
(implicit
35+
redisConf: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
36+
RedisInputDStream[String] = {
37+
new RedisInputDStream(ssc, keys, storageLevel, redisConf, classOf[String])
38+
}
39+
40+
def createRedisXStream(consumersConfig: Seq[ConsumerConfig],
41+
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_2)
42+
(implicit
43+
redisConfig: RedisConfig = RedisConfig.fromSparkConf(ssc.sparkContext.getConf)):
44+
InputDStream[StreamItem] = {
45+
val readWriteConfig = ReadWriteConfig.fromSparkConf(ssc.sparkContext.getConf)
46+
val receiver = new RedisStreamReceiver(consumersConfig, redisConfig, readWriteConfig, storageLevel)
47+
ssc.receiverStream(receiver)
48+
}
49+
}
50+
51+
trait RedisStreamingFunctions {
52+
53+
implicit def toRedisStreamingContext(ssc: StreamingContext): RedisStreamingContext = new RedisStreamingContext(ssc)
54+
55+
}
56+

src/test/scala/com/redislabs/provider/redis/stream/RedisXStreamSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,8 @@ package com.redislabs.provider.redis.stream
33
import com.redislabs.provider.redis.streaming.{ConsumerConfig, Earliest}
44
import com.redislabs.provider.redis.util.ConnectionUtils.withConnection
55
import com.redislabs.provider.redis.util.TestUtils
6-
import com.redislabs.provider.redis.{SparkStreamingRedisSuite, _}
6+
import com.redislabs.provider.redis.SparkStreamingRedisSuite
7+
import com.redislabs.provider.redis.streaming._
78
import org.apache.spark.storage.StorageLevel
89
import org.scalatest.Matchers
910
import org.scalatest.concurrent.Eventually._

0 commit comments

Comments
 (0)