On 21 May 2019, at 22:06, 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?

I’m not very familiar with Jedis, but, at the first glance, it seems that we 
can use List<byte[]> BinaryJedis.mget(final byte[]... keys) [1] in this case.
Also, probably, it can be achieved by using "Response<byte[]> 
PipelineBase.get(byte[])” (we already use Pipeline for writing). In any case, 
Redis uses C char type under the hood and it converts key/value to Java String 
in Jedis client [2].

> 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.

I think it could be a useful feature for end user but it should be tricky one 
since, as you mentioned before, there is no order guarantee and the final 
result for the same key is undetermined. 

> Cons:
> - We have to encode and serialize one extra field (the operation) for each 
> element not terrible but there is a cost there.

Well, I don’t think it should be very costy and hardly affect a performance, so 
it’s not a big deal imo. 

> 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.

Yes but the final result of several operations, applied on the same key, 
depends on the order of these operations. Despite the fact that Jedis supports 
Transcations [3], I’m not sure it can help here. So, I guess it can be applied 
to limited set of Redis commands. Any thoughts on this?

> 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.

Some Redis commands are applied not only over key/value but they may include 
additional information like expiration time, score, ranges and etc. So, it 
would be worth to have container, like RedisRecord or whatever name, for these 
purposes and it can extended in the future.

[1] 
http://xetorthio.github.io/jedis/redis/clients/jedis/BinaryJedis.html#mget-byte:A…-
 
<http://xetorthio.github.io/jedis/redis/clients/jedis/BinaryJedis.html#mget-byte:A%E2%80%A6->
[2] 
https://github.com/xetorthio/jedis/wiki/AdvancedUsage#a-note-about-string-and-binary---what-is-native
 
<https://github.com/xetorthio/jedis/wiki/AdvancedUsage#a-note-about-string-and-binary---what-is-native>
[3] https://github.com/xetorthio/jedis/wiki/AdvancedUsage#transactions 
<https://github.com/xetorthio/jedis/wiki/AdvancedUsage#transactions>



> 
> 
> On Tue, May 21, 2019 at 4:55 PM Alexey Romanenko <aromanenko....@gmail.com 
> <mailto: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 
> <https://issues.apache.org/jira/browse/BEAM-6063>
> 
>> On 21 May 2019, at 07:12, trs...@gmail.com <mailto: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