Hi Mani. The connector that’s in io/kinesis is based on version 1 of the AWS 
SDK for Java and the one in amazon-web-services2 is based on the version 2 SDK. 
My understanding is that there were some licensing issues preventing it from 
being bundled together with the other AWS IOs initially but those have since 
been resolved. For now the two connectors are functionally equivalent but the 
v1 is deprecated so you should use v2 instead.

> On Sep 3, 2020, at 1:45 AM, Sunny, Mani Kolbe <[email protected]> wrote:
> 
> 
> Hi Jonothan,
>  
> That is good news! I was of the impression, it is not supported or have to 
> enable some flag. By the way, what is the difference between KinesisIO and 
> aws V2. Is aws V2 is a set of classes to support AWS related connectors and 
> KinesisIO is referring back to them?
>  
> I mean for read/write from Kinesis, KinesisIO is the still the way to go?
>  
> Regards,
> Mani
>  
> From: Jonothan Farr <[email protected]> 
> Sent: Thursday, September 3, 2020 1:02 AM
> To: [email protected]
> Subject: Re: KinesisIO - aggregation
>  
> CAUTION: This email originated from outside of D&B. Please do not click links 
> or open attachments unless you recognize the sender and know the content is 
> safe.
>  
> KinesisIO works fine for me with aggegated records. Here's where 
> deaggregate() is called in v1:
> 
> https://github.com/apache/beam/blob/9f4d54a2e60ec45150437c0b050f4c73cca91f36/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java#L170
> 
> And v2:
> 
> https://github.com/apache/beam/blob/9f4d54a2e60ec45150437c0b050f4c73cca91f36/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java#L181
> 
> Were you having problems getting it to work?
>  
> On Wed, Sep 2, 2020 at 8:22 AM Alexey Romanenko <[email protected]> 
> wrote:
> Yes, it’s not supported for now, but, at the first sight, it seems that we 
> just need to call UserRecord.deaggregate() in GetKinesisRecordsResult in case 
> if record is aggregated. 
>  
> 
> On 2 Sep 2020, at 14:30, Sunny, Mani Kolbe <[email protected]> wrote:
>  
> Hi Alexey,
>  
> I am looking for reading Kinesis stream with that has aggregated record. From 
> your reply, I take that it is currently not supported? Could that be solved 
> by adding an uncompression function on the pipeline ?
>  
> Regards,
> Mani
>  
> From: Alexey Romanenko <[email protected]> 
> Sent: Tuesday, September 1, 2020 6:04 PM
> To: [email protected]
> Subject: Re: KinesisIO - aggregation
>  
> CAUTION: This email originated from outside of D&B. Please do not click links 
> or open attachments unless you recognize the sender and know the content is 
> safe.
>  
> Hello Mani, 
>  
> For Write part it should be already supported since KinesisIO uses KPL to 
> write records under the hood. So, it’s just a question of proper 
> configuration [1][2][3]
> For Read part, since it’s based on AWS API, it’s more complicated and we need 
> to add a support for this explicitly.
>  
> [1] 
> https://github.com/apache/beam/blob/afa59c463ca9c5d42a9ab13402a72c8b95240fa5/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java#L657
> [2] https://docs.aws.amazon.com/streams/latest/dev/kinesis-kpl-config.html
> [3] 
> https://github.com/awslabs/amazon-kinesis-producer/blob/master/java/amazon-kinesis-producer-sample/default_config.properties
>  
> 
> On 31 Aug 2020, at 16:12, Sunny, Mani Kolbe <[email protected]> wrote:
>  
> Hello,
>  
> Does Beam have plans to support Kinesis records with aggregation[1]? I see 
> some code [2] in the repo related to that. Is it planned for any near future 
> releases?
>  
> Regards,
> Mani
>  
>  
> [1] 
> https://docs.aws.amazon.com/streams/latest/dev/kinesis-kpl-concepts.html#kinesis-kpl-concepts-aggretation
> [2] 
> https://github.com/apache/beam/blob/22822b8c611f4d8888d3b2039e4f7e7d98fcab39/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java#L181
>  

Reply via email to