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<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> 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>
> 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, 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 <prakhargau...@google.com> |
>>
>

Reply via email to