Varun, thank you for starting this topic.

I agree that it would make sense to do some refactoring and introduce something 
like “RedisRecord<K, V>” which will be parameterised and contain all additional 
metadata if needed. In the same time, we can keep current public API based on 
KV<String, String>, but move to using RedisRecord internally instead and add 
RedisIO.writeRecords() that will take PCollection<RedisRecord<K, V>> as an 
input.

We already did a similar thing for KafkaIO before when we moved to 
ProducerRecord<K,V> for writing to Kafka [1] and kept KV API as well.

[1] https://issues.apache.org/jira/browse/BEAM-6063 
<https://issues.apache.org/jira/browse/BEAM-6063>

> On 21 May 2019, at 07:12, trs...@gmail.com wrote:
> 
> 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
>  
> <https://github.com/gojek/feast/blob/master/ingestion/src/main/java/feast/store/serving/redis/RedisCustomIO.java>
> 
> We ended up with a PTransform<RedisMutation, PDone> transform, where a 
> RedisMutation cloass contains the key and value in byte[]s. and also has the 
> method and expiry (and "score" used by ZSET). If we're going to break things, 
> could we take that approach? It allows different elements to have different 
> methods and expirys.
> 
> 
> 
> On Mon, May 20, 2019 at 11:30 PM Ismaël Mejía <ieme...@gmail.com 
> <mailto:ieme...@gmail.com>> wrote:
> 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 trying to be gentle and 
> mark future removals or breaking changes as @Deprecated and replacing them 
> after two releases.
> 
> Probably an approach to make the migration path easier on users will be:
> 
> 1. Create the new generic transforms and expose them.
> 
> RedisIO.readGeneric ->
>     ReadGeneric extends PTransform<PBegin,  PCollection<KV<K,V>>>
> 
> RedisIO.writeGeneric ->
>     WriteGeneric extends PTransform<PCollection<KV<K, V>>, Pdone>
> 
> 2. Refactor the old Read/Write to use the ReadGeneric/WriteGeneric 
> implementation and expose the old transforms read()/write() also as 
> readStrings()/writeStrings() too.
> 
> 3. Mark then read()/write() with a deprecation warning so users are aware of 
> the change of types.
> 
> 4. Two releases after, make readGeneric()/writeGeneric() into the new 
> read()/write().
> 
> 5. We can let readStrings/writeStrings as a convenience method, or deprecate 
> it and remove it afterwards too.
> 
> WDYT?
> 
> On Sun, May 19, 2019 at 9:41 AM Varun Dhussa <varundhu...@google.com 
> <mailto:varundhu...@google.com>> wrote:
> Hi all,
> 
> The current RedisIO implementation has a limitation that it can only support 
> a PCollection with KV<String, String>. 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 and DECRBY operations 
> ()https://github.com/apache/beam/pull/8570 
> <https://github.com/apache/beam/pull/8570>, for which I had to add a string 
> to long parse call.
> 
> I feel that this should be redesigned to support all of Redis' operations. I 
> am thinking of providing a type with a format function (similar to other 
> data-stores). However, I am not able to think of a way to make it compatible 
> with existing interface. Do you think that this is a major concern? Would you 
> recommend a different approach for this? I can start working on it once I get 
> your input.
> 
> Thanks and regards
> 
> Varun Dhussa |         Solutions Architect, Google Cloud |     
> varundhu...@google.com <mailto:prakhargau...@google.com> |      

Reply via email to