Hi Flink Devs,
I'm glad to announce that FLIP-477:Amazon SQS Source Connector[1] has been
accepted.
There are 5 approving votes, 3 of which are Binding and 2 are Non-Binding:
- Arvid Heise (Binding)
- Danny Cranmer (Binding)
- Robert Metzger (Binding)
- Ahmed Hamdy (Non-binding)
- Aleksandr Pil
we might need to reread messages that were
in-flight, and we do not have the receipt IDs. But they may not be visible
now, so besides tuning the visibility period is there a better way to
optimise this?
Danny CranmerDanny Cranmer 1:38 AM
<https://apache-flink.slack.com/archives/C07MRSRNWBU/p17
orget to emit SourceReaderFinishedEvent for
> all idle subtasks). If the pipeline is embarrassing parallel, all
> downstream subtasks will also be closed.
> But there is nothing that you can do except document it to set the
> parallelism of the source to 1. A higher parallelism is only u
Hi Flink Devs,
Gentle Reminder for voting on FLIP-477: Amazon SQS Source Connector [1].
[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-477
+Amazon+SQS+Source+Connector
Regards
Saurabh & Abhi
On Thu, Aug 22, 2024 at 1:17 AM Aleksandr Pilipenko
wrote:
> Thank you for driving this!
>
2 PM Saurabh Singh
wrote:
> Hi Ahmed,
>
> Yes, you're correct. Currently, we're utilizing the "record emitter" to
> send messages into the queue for deletion. However, for the actual deletion
> process, which is dependent on the checkpoints, we've been using the s
emitter is a good place for that where we delete messages that are
> forwarded to the next operator.
> Other than that I don't have further comments.
> Thanks again for the effort.
>
> Best Regards
> Ahmed Hamdy
>
>
> On Wed, 31 Jul 2024 at 10:34, Saurabh Singh
>
ing is not enabled?
>
> I am not 100% convinced we should block FLIP itself on the FLIP-438
> implementation but I echo the fact that there might be some reusable code
> between the 2 submodules we should make use of.
>
> Best Regards
> Ahmed Hamdy
>
>
> On Fri, 26 Jul
rall SQS
> connector codebase. What do you think?
>
> Thanks
>
>
> On Thursday, July 25, 2024, Saurabh Singh
> wrote:
>
> > Hi Samrat,
> >
> > Thanks for the review and feedback.
> > We have evaluated all the three points. Please find the answers below:
t/SQSDeveloperGuide/sqs-queue-types.html
> [3]
>
> https://docs.aws.amazon.com/step-functions/latest/dg/callback-task-sample-sqs.html
>
> Bests,
> Samrat
>
>
> On Fri, 19 Jul 2024 at 9:53 PM, Saurabh Singh
> wrote:
>
> > Hi Fink Devs,
> >
> > Our tea
1] in SQS source? and I want to
> know what metrics do we provide in SQS source.
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics
>
> Saurabh Singh 于2024年7月20日周六 02:43写道:
>
> > Hi Fink Devs,
> >
> > Our te
Hi Fink Devs,
Our team has been working on migrating various data pipelines to Flink to
leverage the benefits of exactly-once processing, checkpointing, and
stateful computing. We have several use cases built around the AWS SQS
Service. For this migration, we have developed an SQS Source Connector
11 matches
Mail list logo