Skip to content

Commit

Permalink
Merge branch 'master' into fix-issue-132
Browse files Browse the repository at this point in the history
  • Loading branch information
gkorland authored Feb 15, 2019
2 parents 4e27bed + 6aaaba2 commit 68c8f2a
Show file tree
Hide file tree
Showing 34 changed files with 1,166 additions and 135 deletions.
1 change: 1 addition & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ This library is work in progress so the API may change before the official relea
- [RDD](doc/rdd.md)
- [Dataframe](doc/dataframe.md)
- [Streaming](doc/streaming.md)
- [Structured Streaming](doc/structured-streaming.md)
- [Cluster](doc/cluster.md)
- [Java](doc/java.md)
- [Python](doc/python.md)
Expand Down
2 changes: 1 addition & 1 deletion doc/python.md
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ $ ./bin/pyspark --jars <path-to>/spark-redis-<version>-jar-with-dependencies.jar
By default it connects to `localhost:6379` without any password, you can change the connection settings in the following manner:

```bash
$ bin/spark-shell --jars <path-to>/spark-redis-<version>-jar-with-dependencies.jar --conf "spark.redis.host=localhost" --conf "spark.redis.port=6379" --conf "spark.redis.auth=passwd"
$ bin/pyspark --jars <path-to>/spark-redis-<version>-jar-with-dependencies.jar --conf "spark.redis.host=localhost" --conf "spark.redis.port=6379" --conf "spark.redis.auth=passwd"
```


Expand Down
114 changes: 114 additions & 0 deletions doc/structured-streaming.md
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)
```
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
org.apache.spark.sql.redis.stream.RedisStreamProvider
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,8 @@ package com.redislabs.provider.redis.streaming

import java.util.AbstractMap.SimpleEntry

import com.redislabs.provider.redis.util.Logging
import com.redislabs.provider.redis.util.PipelineUtils.foreachWithPipeline
import com.redislabs.provider.redis.util.{Logging, StreamUtils}
import com.redislabs.provider.redis.{ReadWriteConfig, RedisConfig}
import org.apache.curator.utils.ThreadUtils
import org.apache.spark.storage.StorageLevel
Expand Down Expand Up @@ -59,17 +59,12 @@ class RedisStreamReceiver(consumersConfig: Seq[ConsumerConfig],
}

def createConsumerGroupIfNotExist(): Unit = {
try {
val entryId = conf.offset match {
case Earliest => new EntryID(0, 0)
case Latest => EntryID.LAST_ENTRY
case IdOffset(v1, v2) => new EntryID(v1, v2)
}
jedis.xgroupCreate(conf.streamKey, conf.groupName, entryId, true)
} catch {
case e: Exception =>
if (!e.getMessage.contains("already exists")) throw e
val entryId = conf.offset match {
case Earliest => new EntryID(0, 0)
case Latest => EntryID.LAST_ENTRY
case IdOffset(v1, v2) => new EntryID(v1, v2)
}
StreamUtils.createConsumerGroupIfNotExist(jedis, conf.streamKey, conf.groupName, entryId)
}

def receiveUnacknowledged(): Unit = {
Expand Down
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
}
}

}
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 src/main/scala/com/redislabs/provider/redis/util/JsonUtils.scala
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 src/main/scala/com/redislabs/provider/redis/util/ParseUtils.scala
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 src/main/scala/com/redislabs/provider/redis/util/StreamUtils.scala
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)
}
}
Loading

0 comments on commit 68c8f2a

Please sign in to comment.