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 (<K>, <K,V> and <K,<K,V>>). I feel that Hash-map and sorted set operations can be modeled better with a mutation class. Of-course, this can also be done with a generic KV and checking the type of V in the function implementing the operation.
Thanks Varun Dhussa | Solutions Architect, Google Cloud | varundhu...@google.com <prakhargau...@google.com> | On Wed, May 22, 2019 at 1:37 AM Ismaël Mejía <ieme...@gmail.com> wrote: > After a quick review of the code now I think I understand why it was > modeled as KV<String, String> 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 modeling RedisMutation, it has its pros/cons: > Pros: > - We can have pipelines where each element can do a different operation > which seems nice for the end user. > Cons: > - We have to encode and serialize one extra field (the operation) for each > element not terrible but there is a cost there. > Maybe we can have both, the previous approach where you can specify one > operation if all elemetns in the PCollection are part of the same operation > or RedisMutation if we want to do different things per element. Does anyone > know if this can have any issue given Redis functionality? Don't forget > that PCollections do not have order guarantees so some results can look > weird to some users. > > Do we really need RedisRecord? I mean do Redis records have metadata? > Because that's the main motivation to have it in Kafka. i would try to have > the simplest interface possible for the end user which for a KV store is > KV<K,V> no? > I mean at least for the public API we can have RedisRecord internally if > it is worth, but looking again at the Jedis API I cannot see any reason to > do this. > > > On Tue, May 21, 2019 at 4:55 PM Alexey Romanenko <aromanenko....@gmail.com> > wrote: > >> 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 >> >> 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 >> >> 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> | >>>> >>> >>