I have updated the FLIP [1] (please double-check) and will add my +1 now to
the voting.
Best,
Arvid
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-477+Amazon+SQS+Source+Connector
On Wed, Sep 25, 2024 at 2:21 PM Saurabh Singh
wrote:
> Hi Flink Devs,
>
> Thanks a lot for the valuabl
Hi Flink Devs,
Thanks a lot for the valuable feedback from Arvid, Danny and David on the
design of this SQS Connector.
We had an in-depth discussion on Slack regarding the semantics and stages
of implementation. Below is the *conclusion* of our conversation.
- Currently, FLIP will concentrate
https://cwiki.apache.org/confluence/display/FLINK/FLIP-477+Amazon+SQS+Source+Connector
I'm a Principal Engineer at AWS and work directly with the AWS SQS team. I can
provide all kinds of details about the SQS API and SQS semantics in general to
help you validate the design for exactly once proce
Hi Saurabh,
thank you very much for taking the time to explain SQS features to me. It's
a cool concept that I haven't seen before.
While your presented approach already works for Flink EOS, I still need to
poke a bit because I think the usability may be improved.
Some background information on h
Hi Arvid,
Thanks a lot for your review.
1. Exactly once:
*When using an SQS source, we leverage two key properties of SQS:1. Message
Retention: This property ensures that messages are retained in the queue
for a specified period, allowing consumers enough time to process them.
Post these me
Hi Saurabh,
thank you for addressing the points. I still have concerns around EOS and a
few remarks.
1. Exactly once:
The checkpoint subsuming contract typically only applies to sinks, so it's
unusual to be used in sources. Let's quickly look at an example so we are
all on the same page.
Assum
Sorry for being late to the party. I saw your call to vote and looked at
the FLIP.
First, most of the design is looking really good and it will be good to
have another connector integrated into the AWS ecosystem. A couple of
questions/remarks:
1) Since we only have 1 split, we should also limit th
Hello Saurabh,
Thanks for contributing this, I have seen multiple custom implementations
of SQS sources, so a community supported version will be a great addition.
// .setFailOnError(false)
A general callout that this is not ideal, I know we use it elsewhere but we
should consider some pluggable
Hi Abhisagar and Saurabh
I have created the FLIP page and assigned it FLIP-477[1]. Feel free to
resume with the next steps.
1-
https://cwiki.apache.org/confluence/display/FLINK/FLIP-+477+Amazon+SQS+Source+Connector
Best Regards
Ahmed Hamdy
On Tue, 20 Aug 2024 at 06:05, Abhisagar Khatri
wrote:
Hi Flink Devs,
Thanks for all the feedback flink devs.
Following the procedure outlined on the Flink Improvement Proposal
Confluence page [1], we kindly ask the PMC/Committers to transfer the
content from the Amazon SQS Source Connector Google Doc [2] and assign a
FLIP Number for us, which we wil
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 source
reader class because it allows us to override the notifyCheckpoi
Hi Saurabh
Thanks for addressing, I see the FLIP is in much better state.
Could we specify where we queue messages for deletion, In my opinion the
record 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.
Hi Ahmed,
Thank you very much for the detailed, valuable review. Please find our
responses below:
- In the FLIP you mention the split is going to be 1 sqs Queue, does
this mean we would support reading from multiple queues? This is also not
clear in the implementation of `addSplitsBack`
Hi Saurabh
I think this is going to be a valuable addition which is needed.
I have a couple of comments
- In the FLIP you mention the split is going to be 1 sqs Queue, does this
mean we would support reading from multiple queues? The builder example
seems to support a single queue
SqsSource.builder
Hi Li Wang,
Thanks for the review and appreciate your feedback.
I completely understand your concern and agree with it. Our goal is to
provide users of connectors with a consistent and coherent ecosystem, free
of any issues.
To ensure that, we are closely monitoring/reviewing the SQS Sink
implemen
Hi Saurabh,
Thanks for the FLIP. Given the ongoing effort to implement the SQS sink
connector (
https://cwiki.apache.org/confluence/display/FLINK/FLIP-438%3A+Amazon+SQS+Sink+Connector),
it is important to consider how the SQS source connector supports this
development, ensuring a unified framework
Hi Samrat,
Thanks for the review and feedback.
We have evaluated all the three points. Please find the answers below:
1. AWS has announced JSON protocol support in SQS [1]. Can you shed some
light on how different protocols will be supported?
- We will utilize the AWS client library to connect w
Hi Saurabh,
Thank you for sharing the FLIP for the SQS source connector. An SQS source
connector will be a great addition to the Flink ecosystem, as there is a
growing demand for SQS source/sink integration.
I have a few queries:
1. AWS has announced JSON protocol support in SQS [1]. Can you she
Hi Yanquan,
Yes, you are right about SQS - It is a MessageQueue.
https://aws.amazon.com/sqs/features/
Thanks for reviewing the FLIP and valuable suggestions about adding
observability.
We have reviewed the above-mentioned Standardize Connector Metric FLIP [1]
and added a section about it in the
Hi Dhingra, thanks for driving this. I am not very familiar with SQS, but
this should be some kind of message queue. So could we add metric of
currentFetchEventTimeLag
and currentEmitEventTimeLag from FLIP-33[1] in SQS source? and I want to
know what metrics do we provide in SQS source.
[1]
http
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
21 matches
Mail list logo