-
Notifications
You must be signed in to change notification settings - Fork 371
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge branch 'master' into fix-issue-132
- Loading branch information
Showing
34 changed files
with
1,166 additions
and
135 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,114 @@ | ||
### Structured Streaming | ||
|
||
Spark-Redis supports [Redis Stream](https://redis.io/topics/streams-intro) data structure as a source for [Structured Streaming](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html): | ||
|
||
The following example reads data from a Redis Stream `sensors` that has two fields `sensor-id` and `temperature`: | ||
|
||
```scala | ||
val spark = SparkSession | ||
.builder | ||
.master("local[*]") | ||
.config("spark.host", "spark.redis.host") | ||
.config("spark.redis.port", "6379") | ||
.getOrCreate() | ||
|
||
val sensors = spark | ||
.readStream | ||
.format("redis") // read from Redis | ||
.option("stream.keys", "sensors") // stream key | ||
.schema(StructType(Array( // stream fields | ||
StructField("sensor-id", StringType), | ||
StructField("temperature", FloatType) | ||
))) | ||
.load() | ||
|
||
val query = sensors | ||
.writeStream | ||
.format("console") | ||
.start() | ||
|
||
query.awaitTermination() | ||
|
||
``` | ||
|
||
You can write the following items to the stream to test it: | ||
|
||
``` | ||
xadd sensors * sensor-id 1 temperature 28.1 | ||
xadd sensors * sensor-id 2 temperature 30.5 | ||
xadd sensors * sensor-id 1 temperature 28.3 | ||
``` | ||
|
||
### Stream Offset | ||
|
||
By default it pulls messages starting from the latest message in the stream. If you need to start from the specific position in the stream, specify the `stream.offsets` parameter as a JSON string. | ||
In the following example we set offset id to be `1548083485360-0`. The group name `redis-source` is a default consumer group that spark-redis automatically creates to read stream. | ||
|
||
```scala | ||
val offsets = """{"offsets":{"sensors":{"groupName":"redis-source","offset":"1548083485360-0"}}}""" | ||
|
||
... | ||
|
||
.option("stream.offsets", offsets) | ||
``` | ||
|
||
If you want to process stream from the beginning, set offset id to `0-0`. | ||
|
||
### Entry id column | ||
|
||
You can access stream entry id by adding a column `_id` to the stream schema: | ||
|
||
``` | ||
val sensors = spark | ||
.readStream | ||
.format("redis") | ||
.option("stream.keys", "sensors") | ||
.schema(StructType(Array( | ||
StructField("_id", StringType), // entry id | ||
StructField("sensor-id", StringType), | ||
StructField("temperature", FloatType) | ||
))) | ||
.load() | ||
``` | ||
|
||
The stream schema: | ||
|
||
|
||
``` | ||
+---------------+---------+-----------+ | ||
| _id|sensor-id|temperature| | ||
+---------------+---------+-----------+ | ||
|1548083485360-0| 1| 28.1| | ||
|1548083486248-0| 2| 30.5| | ||
|1548083486944-0| 1| 28.3| | ||
+---------------+---------+-----------+ | ||
``` | ||
|
||
|
||
### Level of Parallelism | ||
|
||
By default spark-redis creates a consumer group with a single consumer. There are two options how you can increase the level of parallelism. | ||
|
||
The first approach is to create stream from multiple Redis keys. You can specify multiple keys separated by comma, e.g. | ||
`.option("stream.keys", "sensors-eu,sensors-us")`. In this case data from each key will be mapped to a Spark partition. | ||
Please note, the items ordering will be preserved only within a particular Redis key (Spark partition), there is no ordering guarantees for items in different keys. | ||
|
||
With the second approach you can read data from a single Redis key with multiple consumers in parallel, e.g. `option("stream.parallelism", 4)`. | ||
Each consumer will be mapped to a Spark partition. There is no ordering guarantees in this case. | ||
|
||
|
||
|
||
### Other configuration | ||
|
||
The spark-redis automatically creates a consumer group with name `spark-source` if it doesn't exist. You can customize the consumer group name with | ||
`.option("stream.group.name", "my-group")`. Also you can customize the name of consumers in consumer group with `.option("stream.consumer.prefix", "my-consumer")`. | ||
|
||
|
||
Another options you can configure are `stream.read.batch.size` and `stream.read.block`. They define the maximum number of pulled items and time in milliseconds to wait in a `XREADGROUP` call. | ||
The default values are 100 items and 500 ms. | ||
|
||
```scala | ||
.option("stream.read.batch.size", 200) | ||
.option("stream.read.block", 1000) | ||
``` |
1 change: 1 addition & 0 deletions
1
src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
org.apache.spark.sql.redis.stream.RedisStreamProvider |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
29 changes: 29 additions & 0 deletions
29
src/main/scala/com/redislabs/provider/redis/util/CollectionUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
package com.redislabs.provider.redis.util | ||
|
||
import scala.collection.IterableLike | ||
import scala.collection.generic.CanBuildFrom | ||
|
||
/** | ||
* @author The Viet Nguyen | ||
*/ | ||
object CollectionUtils { | ||
|
||
implicit class RichCollection[A, Repr](val xs: IterableLike[A, Repr]) extends AnyVal { | ||
|
||
def distinctBy[B, That](f: A => B)(implicit cbf: CanBuildFrom[Repr, A, That]): That = { | ||
val builder = cbf(xs.repr) | ||
val iterator = xs.iterator | ||
var set = Set[B]() | ||
while (iterator.hasNext) { | ||
val element = iterator.next | ||
val distinctField = f(element) | ||
if (!set(distinctField)) { | ||
set += distinctField | ||
builder += element | ||
} | ||
} | ||
builder.result | ||
} | ||
} | ||
|
||
} |
72 changes: 69 additions & 3 deletions
72
src/main/scala/com/redislabs/provider/redis/util/ConnectionUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,15 +1,81 @@ | ||
package com.redislabs.provider.redis.util | ||
|
||
import java.util.{List => JList} | ||
|
||
import com.redislabs.provider.redis.RedisConfig | ||
import com.redislabs.provider.redis.util.ConnectionUtils.XINFO.{SubCommandGroups, SubCommandStream} | ||
import redis.clients.jedis.Jedis | ||
import redis.clients.jedis.commands.ProtocolCommand | ||
import redis.clients.jedis.util.SafeEncoder | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
/** | ||
* @author The Viet Nguyen | ||
*/ | ||
object ConnectionUtils { | ||
|
||
def withConnection[A](conn: Jedis)(body: Jedis => A): A = { | ||
val res = body(conn) | ||
conn.close() | ||
res | ||
try { | ||
body(conn) | ||
} finally { | ||
conn.close() | ||
} | ||
} | ||
|
||
def withConnection[A](streamKey: String)(body: Jedis => A)(implicit redisConfig: RedisConfig): A = { | ||
withConnection(redisConfig.connectionForKey(streamKey)){ | ||
body | ||
} | ||
} | ||
|
||
implicit class JedisExt(val jedis: Jedis) extends AnyVal { | ||
|
||
//TODO: temporary solution to get latest offset while not supported by Jedis | ||
def xinfo(command: String, args: String*): Map[String, Any] = { | ||
val client = jedis.getClient | ||
val combinedArgs = command +: args | ||
client.sendCommand(XINFO, combinedArgs: _*) | ||
val response = asList(client.getOne).asScala | ||
command match { | ||
case SubCommandStream => | ||
asMap(response) | ||
case SubCommandGroups => | ||
response.map(m => asList(m)).map(_.asScala).map(asMap) | ||
.map(m => String.valueOf(m("name")) -> m).toMap | ||
} | ||
} | ||
|
||
private def asMap(seq: Seq[Any]): Map[String, Any] = { | ||
seq.grouped(2) | ||
.map { group => | ||
val key = asString(group.head) | ||
val value = group(1) match { | ||
case arr: Array[Byte] => asString(arr) | ||
case other: Any => other | ||
} | ||
key -> value | ||
}.toMap | ||
} | ||
|
||
private def asList(any: Any): JList[Any] = | ||
any.asInstanceOf[JList[Any]] | ||
|
||
private def asString(any: Any): String = | ||
new String(any.asInstanceOf[Array[Byte]]) | ||
} | ||
|
||
object XINFO extends ProtocolCommand { | ||
|
||
val SubCommandStream = "STREAM" | ||
val SubCommandGroups = "GROUPS" | ||
|
||
val LastGeneratedId = "last-generated-id" | ||
val LastDeliveredId = "last-delivered-id" | ||
val LastEntry = "last-entry" | ||
val EntryId = "_id" | ||
|
||
override def getRaw: Array[Byte] = SafeEncoder.encode("XINFO") | ||
} | ||
|
||
} |
16 changes: 16 additions & 0 deletions
16
src/main/scala/com/redislabs/provider/redis/util/JsonUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,16 @@ | ||
package com.redislabs.provider.redis.util | ||
|
||
import org.json4s.jackson.Serialization | ||
import org.json4s.{Formats, NoTypeHints} | ||
|
||
/** | ||
* @author The Viet Nguyen | ||
*/ | ||
object JsonUtils { | ||
|
||
private implicit val formats: Formats = Serialization.formats(NoTypeHints) | ||
|
||
def toJson(any: AnyRef): String = { | ||
Serialization.write(any) | ||
} | ||
} |
40 changes: 40 additions & 0 deletions
40
src/main/scala/com/redislabs/provider/redis/util/ParseUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
package com.redislabs.provider.redis.util | ||
|
||
import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Long => JLong, Short => JShort} | ||
|
||
import org.apache.spark.sql.types._ | ||
|
||
/** | ||
* @author The Viet Nguyen | ||
*/ | ||
object ParseUtils { | ||
|
||
def parseFields(value: Map[String, String], schema: StructType): Array[Any] = | ||
schema.fields.map { field => | ||
val fieldName = field.name | ||
val fieldValue = value(fieldName) | ||
parseValue(field.dataType, fieldValue) | ||
} | ||
|
||
private def parseValue(dataType: DataType, fieldValueStr: String): Any = { | ||
if (fieldValueStr == null) { | ||
null | ||
} else { | ||
parseNotNullValue(dataType, fieldValueStr) | ||
} | ||
} | ||
|
||
private def parseNotNullValue(dataType: DataType, fieldValueStr: String): Any = | ||
dataType match { | ||
case ByteType => JByte.parseByte(fieldValueStr) | ||
case IntegerType => Integer.parseInt(fieldValueStr) | ||
case LongType => JLong.parseLong(fieldValueStr) | ||
case FloatType => JFloat.parseFloat(fieldValueStr) | ||
case DoubleType => JDouble.parseDouble(fieldValueStr) | ||
case BooleanType => JBoolean.parseBoolean(fieldValueStr) | ||
case ShortType => JShort.parseShort(fieldValueStr) | ||
case DateType => java.sql.Date.valueOf(fieldValueStr) | ||
case TimestampType => java.sql.Timestamp.valueOf(fieldValueStr) | ||
case _ => fieldValueStr | ||
} | ||
} |
28 changes: 28 additions & 0 deletions
28
src/main/scala/com/redislabs/provider/redis/util/StreamUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
package com.redislabs.provider.redis.util | ||
|
||
import org.apache.commons.lang3.StringUtils | ||
import redis.clients.jedis.{EntryID, Jedis} | ||
|
||
/** | ||
* @author The Viet Nguyen | ||
*/ | ||
object StreamUtils extends Logging { | ||
|
||
val EntryIdEarliest = new EntryID(0, 0) | ||
|
||
def createConsumerGroupIfNotExist(conn: Jedis, streamKey: String, groupName: String, | ||
offset: EntryID): Unit = { | ||
try { | ||
conn.xgroupCreate(streamKey, groupName, offset, true) | ||
} catch { | ||
case e: Exception if StringUtils.contains(e.getMessage, "already exists") => | ||
logInfo(s"Consumer group already exists: $groupName") | ||
} | ||
} | ||
|
||
def resetConsumerGroup(conn: Jedis, streamKey: String, groupName: String, | ||
offset: EntryID): Unit = { | ||
logInfo(s"Setting consumer group $groupName id to $offset") | ||
conn.xgroupSetID(streamKey, groupName, offset) | ||
} | ||
} |
Oops, something went wrong.