Re: RedisIO refactoring

2019-05-27 Thread Ismaël Mejía
> I’m not very familiar with Jedis, but, at the first glance, it seems that we can use List BinaryJedis.mget(final byte[]... keys) [1] in this case. Also, probably, it can be achieved by using "Response PipelineBase.get(byte[])” (we already use Pipeline for writing). In any case, Redis uses C char

Re: RedisIO refactoring

2019-05-22 Thread Alexey Romanenko
On 21 May 2019, at 22:06, Ismaël Mejía wrote: > > After a quick review of the code now I think I understand why it was modeled > as KV in the first place, the library that RedisIO uses > (Jedis) only supports 'mget' operation on Strings, so the first issue would > be to find a way to do the na

Re: RedisIO refactoring

2019-05-21 Thread Varun Dhussa
Hi all, Meta may not be a requirement, but RedisRecord can make more operations cleaner. Modelling Redis Mutations allows a clear way for operations with different payloads (, and >). I feel that Hash-map and sorted set operations can be modeled better with a mutation class. Of-course, this can a

Re: RedisIO refactoring

2019-05-21 Thread Ismaël Mejía
After a quick review of the code now I think I understand why it was modeled as KV in the first place, the library that RedisIO uses (Jedis) only supports 'mget' operation on Strings, so the first issue would be to find a way to do the native byte[] operations, maybe another library? Ideas? About

Re: RedisIO refactoring

2019-05-21 Thread Alexey Romanenko
Varun, thank you for starting this topic. I agree that it would make sense to do some refactoring and introduce something like “RedisRecord” which will be parameterised and contain all additional metadata if needed. In the same time, we can keep current public API based on KV, but move to using

Re: RedisIO refactoring

2019-05-20 Thread trsell
Hi, I created a custom RedisIO for similar reasons, we wanted more operation types, but also, we wanted to be able to set expiry differently per key. https://github.com/gojek/feast/blob/master/ingestion/src/main/java/feast/store/serving/redis/RedisCustomIO.java We ended up with a PTransform trans

Re: RedisIO refactoring

2019-05-20 Thread Ismaël Mejía
Hello Varun, This is an excellent idea because Redis already supports byte arrays as both keys and values. A more generic approach makes total sense. So worth a JIRA / PR. About the compatiblity concerns, RedisIO is tagged as @Experimental which means we can still evolve its API. Currently we are

RedisIO refactoring

2019-05-19 Thread Varun Dhussa
Hi all, The current RedisIO implementation has a limitation that it can only support a PCollection with KV. This imposes limitations while using single key operations (such as INCR/DECR), or operations on complex data structures (such as ZINCRBY). I had recently submitted a patch to support INCRBY