Skip to content

Issue 265: Remove filterKeysByType step when getting data from redis #266

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 8 commits into from
Sep 11, 2020
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
86 changes: 69 additions & 17 deletions src/main/scala/com/redislabs/provider/redis/rdd/RedisRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -7,11 +7,13 @@ 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.exceptions.JedisDataException
import redis.clients.jedis.{Jedis, ScanParams}
import redis.clients.jedis.util.JedisClusterCRC16

import scala.collection.JavaConversions._
import scala.reflect.{ClassTag, classTag}
import scala.util.{Failure, Success, Try}


class RedisKVRDD(prev: RDD[String],
Expand All @@ -38,11 +40,22 @@ class RedisKVRDD(prev: RDD[String],
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) =>

val response = mapWithPipeline(conn, nodeKeys) { (pipeline, key) =>
pipeline.get(key)
}
val res = stringKeys.zip(response).iterator.asInstanceOf[Iterator[(String, String)]]

val res = nodeKeys.zip(response)
.flatMap{
// Silently filter out this exception, when the value doesn't match the expected type "String"
case (_, e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
// Throw any other JedisDataException we encounter
case (_, e: JedisDataException) => throw e
case (k, v: String) => Some(k, v)
// Default case, that shouldn't really happen
case _ => None
}
.iterator
conn.close()
res
}
Expand All @@ -51,8 +64,13 @@ class RedisKVRDD(prev: RDD[String],
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).iterator
val res = nodeKeys.flatMap{k =>
Try(conn.hgetAll(k).toMap) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.flatten.iterator
conn.close()
res
}
Expand Down Expand Up @@ -80,8 +98,14 @@ class RedisListRDD(prev: RDD[String],
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).iterator
val res: Iterator[String] = nodeKeys.flatMap{k =>
Try(conn.smembers(k).toSet) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.flatten
.iterator
conn.close()
res
}
Expand All @@ -90,8 +114,13 @@ class RedisListRDD(prev: RDD[String],
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)).iterator
val res = nodeKeys.flatMap{ k =>
Try(conn.lrange(k, 0, -1)) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) =>Some(value)
}
}.flatten.iterator
conn.close()
res
}
Expand Down Expand Up @@ -133,13 +162,24 @@ class RedisZSetRDD[T: ClassTag](prev: RDD[String],
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)).
map(tup => (tup.getElement, tup.getScore)).iterator
nodeKeys.flatMap{k =>
Try(conn.zrangeWithScores(k, startPos, endPos)) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.flatten
.map(tup => (tup.getElement, tup.getScore)).iterator
} else if (classTag[T] == classTag[String]) {
zsetKeys.flatMap(k => conn.zrange(k, startPos, endPos)).iterator
nodeKeys.flatMap{k =>
Try(conn.zrange(k, startPos, endPos)) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.flatten.iterator
} else {
throw new scala.Exception("Unknown RedisZSetRDD type")
}
Expand All @@ -155,13 +195,25 @@ class RedisZSetRDD[T: ClassTag](prev: RDD[String],
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)).
map(tup => (tup.getElement, tup.getScore)).iterator
nodeKeys.flatMap{k =>
Try(conn.zrangeByScoreWithScores(k, startScore, endScore)) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.
flatten
.map(tup => (tup.getElement, tup.getScore)).iterator
} else if (classTag[T] == classTag[String]) {
zsetKeys.flatMap(k => conn.zrangeByScore(k, startScore, endScore)).iterator
nodeKeys.flatMap{ k =>
Try(conn.zrangeByScore(k, startScore, endScore)) match {
case Failure(e: JedisDataException) if Option(e.getMessage).getOrElse("").contains("WRONGTYPE") => None
case Failure(e) => throw e
case Success(value) => Some(value)
}
}.flatten.iterator
} else {
throw new scala.Exception("Unknown RedisZSetRDD type")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,11 @@ trait RedisRddSuite extends SparkRedisSuite with Keys with Matchers {
test("RedisKVRDD") {
val redisKVRDD = sc.fromRedisKV("*")
val kvContents = redisKVRDD.sortByKey().collect
val wrongTypeKeysRes = List(hashKey, zSetKey, listKey, setKey).map(sc.fromRedisKV(_).collect)
val wcnts = contentWords.map((_, 1)).groupBy(_._1).
map(x => (x._1, x._2.map(_._2).sum.toString)).toArray.sortBy(_._1)
kvContents shouldBe wcnts
all(wrongTypeKeysRes) should have size 0
}

test("RedisZsetRDD") {
Expand All @@ -77,6 +79,7 @@ trait RedisRddSuite extends SparkRedisSuite with Keys with Matchers {
val redisZRangeByScore = sc.fromRedisZRangeByScore(zSetKey, 3, 9)
val zrangeByScore = redisZRangeByScore.collect.sorted

val wrongTypeKeysRes = List(hashKey, setKey, listKey, contentWords(0)).map(sc.fromRedisZSetWithScore(_).collect)
val wcnts = contentWords.map((_, 1)).groupBy(_._1).
map(x => (x._1, x._2.map(_._2).sum.toDouble))

Expand All @@ -87,28 +90,35 @@ trait RedisRddSuite extends SparkRedisSuite with Keys with Matchers {
zrangeByScoreWithScore should be(wcnts.toArray.filter(x => x._2 >= 3 && x._2 <= 9)
.sortBy(x => (x._2, x._1)))
zrangeByScore should be(wcnts.toArray.filter(x => x._2 >= 3 && x._2 <= 9).map(_._1).sorted)
all(wrongTypeKeysRes) should have length 0
}

test("RedisHashRDD") {
val redisHashRDD = sc.fromRedisHash(hashKey)
val hashContents = redisHashRDD.sortByKey().collect
val wcnts = contentWords.map((_, 1)).groupBy(_._1).
map(x => (x._1, x._2.map(_._2).sum.toString)).toArray.sortBy(_._1)
val wrongTypeKeysRes = List(zSetKey, setKey, listKey, contentWords(0)).map(sc.fromRedisHash(_).collect)
hashContents should be(wcnts)
all(wrongTypeKeysRes) should have length 0
}

test("RedisListRDD") {
val redisListRDD = sc.fromRedisList(listKey)
val listContents = redisListRDD.sortBy(x => x).collect
val ws = contentWords.sorted
val wrongTypeKeysRes = List(zSetKey, setKey, hashKey, contentWords(0)).map(sc.fromRedisList(_).collect)
listContents should be(ws)
all(wrongTypeKeysRes) should have length 0
}

test("RedisSetRDD") {
val redisSetRDD = sc.fromRedisSet(setKey)
val setContents = redisSetRDD.sortBy(x => x).collect
val ws = content.split("\\W+").filter(!_.isEmpty).distinct.sorted
val wrongTypeKeysRes = List(zSetKey, listKey, hashKey, contentWords(0)).map(sc.fromRedisSet(_).collect)
setContents should be(ws)
all(wrongTypeKeysRes) should have length 0
}

test("Expire") {
Expand Down