Re: [DISCUSS] Migrate or deprecate the Spark Kinesis connector

2024-10-17 Thread Johnson Chen
;> >> A couple months ago, I raised a PR to upgrade the AWS SDK to v2 for the >> Spark Kinesis connector: https://github.com/apache/spark/pull/44211. >> Given that the 4.0 feature freeze is coming, I am following up to check >> whether we still want to have this change in

Re: [DISCUSS] Migrate or deprecate the Spark Kinesis connector

2024-10-16 Thread Jungtaek Lim
at 5:03 PM Johnson Chen wrote: > Hi Spark community, > > A couple months ago, I raised a PR to upgrade the AWS SDK to v2 for the > Spark Kinesis connector: https://github.com/apache/spark/pull/44211. > Given that the 4.0 feature freeze is coming, I am following up to check > whethe

[DISCUSS] Migrate or deprecate the Spark Kinesis connector

2024-10-14 Thread Johnson Chen
Hi Spark community, A couple months ago, I raised a PR to upgrade the AWS SDK to v2 for the Spark Kinesis connector: https://github.com/apache/spark/pull/44211. Given that the 4.0 feature freeze is coming, I am following up to check whether we still want to have this change in the upcoming 4.0

[spark-kinesis] [SPARK-20168] Requesting some attention for a review

2017-11-14 Thread Yash Sharma
Hi Team, Could I please pull some attention towards the pull request on Spark-Kinesis operability. We have iterated over the patch for past few months, and it would be great to have some final review of the patch. I think its very close now. I would love to work on improvements if any. This patch

[Streaming] Requesting more Committers for Spark-Kinesis integration

2017-09-28 Thread Yash Sharma
Hi Fellow Spark developers/ PMC Members, I am a new member of the community and have started my tiny contributions to Spark-Kinesis Integration. I am trying to fill in the gaps in making spark operate with Kinesis as nicely as Kafka. I am writing this mail to highlight an issue with the kinesis

[Spark][Kinesis] Could I get some committer review on the pull request

2017-09-05 Thread Yash Sharma
Hi All, I've been working on a pull request [1] to allow Spark read from a specific timestamp from Kinesis. I have iterated the patch with the help of other contributors and we think that its in a good state now. This patch would save hours of crash recovery time for Spark while reading off Kinesi

[DStream][Kinesis] Requesting review for spark-kinesis retries

2017-04-18 Thread Yash Sharma
Hi Fellow Devs, Please share your thoughts on the pull request that allows spark to have more graceful retries with kinesis streaming. The patch removes simple hard codings in the code and allows user to pass the values in config. This will help users to cope up with kinesis throttling errors and

Spark - Kinesis integration needs improvements

2017-03-30 Thread Yash Sharma
Hello fellow spark devs, hope you are doing fabulous, Dropping a brain dump here about the Spark kinesis integration. I am able to get spark kinesis to work perfectly under ideal conditions, but see a lot of open ends when things are not so ideal. I feel there are lot of open ends and are

Re: Spark + Kinesis

2015-05-09 Thread Vadim Bichutskiy
Thanks Chris! I was just looking to get back to Spark + Kinesis integration. Will be in touch shortly. Vadim ᐧ On Sun, May 10, 2015 at 12:14 AM, Chris Fregly wrote: > hey vadim- > > sorry for the delay. > > if you're interested in trying to get Kinesis working one-on-one,

Re: Spark + Kinesis

2015-05-09 Thread Chris Fregly
n this. thanks! -chris On Tue, Apr 7, 2015 at 6:17 PM, Vadim Bichutskiy wrote: > Hey y'all, > > While I haven't been able to get Spark + Kinesis integration working, I > pivoted to plan B: I now push data to S3 where I set up a DStream to > monitor an S3 bucket with textFi

Re: Spark + Kinesis

2015-04-07 Thread Vadim Bichutskiy
Hey y'all, While I haven't been able to get Spark + Kinesis integration working, I pivoted to plan B: I now push data to S3 where I set up a DStream to monitor an S3 bucket with textFileStream, and that works great. I <3 Spark! Best, Vadim ᐧ On Mon, Apr 6, 2015 at 12:23 PM, Vad

Re: Spark + Kinesis

2015-04-06 Thread Tathagata Das
trois.com/e1t/c/5/f18dQhb0S7lC8dDMPbW2n0x6l2B9nMJW7t5XZs653q_MN8rBNbzRbv22W8r4TLx56dCDWf13Gc8R02?t=https%3A%2F%2Fgithub.com%2Fapache%2Fspark%2Fblob%2Fmaster%2Fextras%2Fkinesis-asl%2Fsrc%2Fmain%2Fscala%2Forg%2Fapache%2Fspark%2Fexamples%2Fstreaming%2FKinesisWordCountASL.scala&si=5533377798602752&pi=8898f7f0-ede6-4e

Re: Spark + Kinesis

2015-04-06 Thread Vadim Bichutskiy
s checkpoint >> interval. Same as batchInterval for this example. */val >> kinesisCheckpointInterval = batchInterval/* Create the same number of >> Kinesis DStreams/Receivers as Kinesis stream's shards */val >> kinesisStreams = (0 until numStreams).map { i => &