Thanks Samrat, this makes sense to me and addresses connector coupling
concerns.
On Sat, Sep 16, 2023 at 4:30 PM Samrat Deb wrote:
> Hi ,
>
> I've made updates to the FLIP[1] by incorporating relevant changes to avoid
> using the Flink connector JDBC. This decision was based on the following
> r
Hi ,
I've made updates to the FLIP[1] by incorporating relevant changes to avoid
using the Flink connector JDBC. This decision was based on the following
reasons:
AWS Redshift utilizes its specialized JDBC driver[2]. Given that their JDBC
driver may undergo evolutions over time, using the Flink c
Hello Martijn,
I apologize for the delay in responding.
Regarding your question about integrating Redshift directly into the JDBC
connector, we are planning to offer two modes: JDBC and UNLOAD. Through our
internal benchmarking, we have observed good performance in the UNLOAD
flow. Additionally,
Hi Samrat,
I'm still having doubts about the dependency on the JDBC connector. When a
user specifies 'read mode', it will use the JDBC connector under the hood.
Why not integrate Redshift then directly in the JDBC connector itself? It
removes the need for a dependency on the JDBC driver, especiall
Hi Leonard,
> Do we have to rely on the latest version of JDBC Connector here?
No, there's no need for us to depend on the latest version of the JDBC
Connector. Redshift has its dedicated JDBC driver [1], which includes
custom modifications tailored to Redshift's specific implementation needs.
Th
Hello Danny,
I wanted to express my gratitude for your valuable feedback and insightful
suggestions.
I will be revising the FLIP to incorporate all of your queries and review
suggestions. Additionally, I plan to provide a Proof of Concept (POC) for
the connector by the end of this week. This POC
Hello Leonard,
> Do we have to rely on the latest version of JDBC Connector here? I
understand that as long as the version of flink minor is the same as the
JDBC Connector, Could you collect the APIs that Redshift generally needs to
use?
I agree we do not necessarily need to rely on the latest pa
Thanks Samrat and Danny for driving this FLIP.
>> an effective approach is to utilize the latest version of
>> flink-connector-jdbc
> as a Maven dependency
>
> When we have stable source/sink APIs and the connector versions are
> decoupled from Flink this makes sense. But right now this would m
Hey Samrat,
Thanks for driving this.
> new FlinkRedshiftSink(prop)
One design nit for "new FlinkRedshiftSink(prop)" I would prefer to use
first class fields such as database/table-name/url/etc, it makes the API
cleaner and easier to use.
> an effective approach is to utilize the latest version
Hi Martijn,
I fully comprehend your apprehensions regarding AWS's reliance on JDBC and
the resultant impact on the release cycle. However, an effective approach
is to utilize the latest version of flink-connector-jdbc as a Maven
dependency. In this arrangement, flink-connector-aws can be associated
Hi Samrat,
The dependency between JDBC and AWS worries me a lot: we're already seeing
that coupling causes a lot of issues down the line. Why can't we decouple
these?
Best regards,
Martijn
On Tue, Jul 4, 2023 at 3:35 PM Samrat Deb wrote:
> Hi Leonard,
>
> Sorry for the late reply.
>
> > 1 Reu
Hi Leonard,
Sorry for the late reply.
> 1 Reusing the capabilities of JDBC and Filesystem in the Redshift
connector generally makes sense to me. However, since all of them are
managed in different repositories and depend on Flink dependency, could you
explain how you establish the versioning, rel
Thanks Samrat for driving this FLIP.
Since the community has already built a set of basic components for the
connector, I only have three comments.
1 Reusing the capabilities of JDBC and Filesystem in the Redshift connector
generally makes sense to me. However, since all of them are managed in
Hi Martijn,
Thank you for sharing your thoughts on the matter.
I understand that you don't have a strong opinion on whether to support
exactly-once processing from the beginning or at a later stage.
For initial implementation I will go ahead with at-least-once semantics.
> The only consideration
Hi Samrat,
I have no strong opinion on whether to support exactly-once from the start
or potentially later. The only consideration that I could think of is that
if you start with at-least-once, you could consider using the ASync API,
but I don't think the ASync API yet supports exactly-once.
Than
Hi Samrat,
The FLIP looks good, thanks!
Best regards,
Jing
On Tue, Jun 6, 2023 at 8:16 PM Samrat Deb wrote:
> Hi Jing,
>
> > I would suggest adding that information into the
> FLIP.
>
> Updated now, please review the new version of flip whenever time.
>
> > +1 Looking forward to your PR :-)
Hi Jing,
> I would suggest adding that information into the
FLIP.
Updated now, please review the new version of flip whenever time.
> +1 Looking forward to your PR :-)
I will request for your review once m ready with PR :-)
Bests,
Samrat
On Tue, Jun 6, 2023 at 11:43 PM Samrat Deb wrote:
> H
Hi Martijn,
> If I understand this correctly, the Redshift sink
would not be able to support exactly-once, is that correct?
As I delve deeper into the study of Redshift's capabilities, I have
discovered that it does support "merge into" operations [1] and some
merge into examples [2].
This opens
Hi Samrat,
Thanks for the feedback. I would suggest adding that information into the
FLIP.
+1 Looking forward to your PR :-)
Best regards,
Jing
On Sat, Jun 3, 2023 at 9:19 PM Samrat Deb wrote:
> Hi Jing Ge,
>
> >>> Do you already have any prototype? I'd like to join the reviews.
> The prototy
Hi Samrat,
Thanks for the FLIP. If I understand this correctly, the Redshift sink
would not be able to support exactly-once, is that correct?
Best regards,
Martijn
On Sat, Jun 3, 2023 at 9:18 PM Samrat Deb wrote:
> Hi Jing Ge,
>
> >>> Do you already have any prototype? I'd like to join the re
Hi Jing Ge,
>>> Do you already have any prototype? I'd like to join the reviews.
The prototype is in progress. I will raise the dedicated PR for review soon
also notify in this thread as well .
>>> Will the Redshift connector provide additional features
beyond the mediator/wrapper of the jdbc con
Hi Ahmed,
>>> please let me know If you need any collaboration regarding integration
with
AWS connectors credential providers or regarding FLIP-171 I would be more
than happy to assist.
Sure, I will reach out incase of any hands required.
On Fri, Jun 2, 2023 at 6:12 PM Jing Ge wrote:
> Hi Sa
Hi Samrat,
Excited to see your proposal. Supporting data warehouses is one of the
major tracks for Flink. Thanks for driving it! Happy to see that we reached
consensus to prioritize the Sink over Source in the previous discussion. Do
you already have any prototype? I'd like to join the reviews.
J
Hi Samrat
Thanks for putting up this FLIP. I agree regarding the importance of the
use case.
please let me know If you need any collaboration regarding integration with
AWS connectors credential providers or regarding FLIP-171 I would be more
than happy to assist.
I also like Leonard's proposal fo
Hi Liu Ron,
> 1. Regarding the `read.mode` and `write.mode`, you say here provides two
modes, respectively, jdbc and `unload or copy`, What is the default value
for `read.mode` and `write.mode?
I have made an effort to make the configuration options `read.mode` and
`write.mode` mandatory for the
Hi, Samrat
Thanks for driving this FLIP. It looks like supporting
flink-connector-redshift is very useful to Flink. I have two question:
1. Regarding the `read.mode` and `write.mode`, you say here provides two
modes, respectively, jdbc and `unload or copy`, What is the default value
for `read.mod
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-307%3A++Flink+Connector+Redshift
[note] Missed the trailing link for previous mail
On Tue, May 30, 2023 at 2:43 PM Samrat Deb wrote:
> Hi Leonard,
>
> > and I’m glad to help review the design as well as the code review.
> Thank you so
Hi Leonard,
> and I’m glad to help review the design as well as the code review.
Thank you so much. It would be really great and helpful to bring
flink-connector-redshift for flink users :) .
I have divided the implementation in 3 phases in the `Scope` Section[1].
1st phase is to
- Integrate
Thanks @Samrat for bringing this discussion.
It makes sense to me to introduce AWS Redshift connector for Apache Flink, and
I’m glad to help review the design as well as the code review.
About the implementation phases, How about prioritizing support for the
Datastream Sink API and TableSink AP
Hi Yaroslav Tkachenko,
Thanks for pointing that out. I have added it in the limitation section.
For now its ok to put it in the limitation section as per initial
implementation.
Bests,
Samrat
On Mon, May 29, 2023 at 9:50 PM Yaroslav Tkachenko
wrote:
> Hi Samrat,
>
> Regarding the sink, will it
Hi Samrat,
Regarding the sink, will it only support append-only tables (no changelog
mode support)? It looks like it's the case, so IMO, it should be mentioned
in the limitations section.
On Sun, May 28, 2023 at 9:52 PM Samrat Deb wrote:
> Hello all ,
>
> Context:
> Amazon Redshift [1] is a ful
Hello all ,
Context:
Amazon Redshift [1] is a fully managed, petabyte-scale data warehouse
service in the cloud. It allows analyzing data without all of the
configurations of a provisioned data warehouse. Resources are automatically
provisioned and data warehouse capacity is intelligently scaled t
32 matches
Mail list logo