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 <j...@ververica.com.invalid> wrote: > 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. > > Just out of curiosity, speaking of JDBC mode, according to the FLIP, it > should be doable to directly use the jdbc connector with Redshift, if I am > not mistaken. Will the Redshift connector provide additional features > beyond the mediator/wrapper of the jdbc connector? > > Best regards, > Jing > > On Thu, Jun 1, 2023 at 8:22 PM Ahmed Hamdy <hamdy10...@gmail.com> wrote: > > > 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 for starting with DataStreamSink and > > TableSink, It would be great to have some milestones delivered as soon as > > ready. > > best regards > > Ahmed Hamdy > > > > > > On Wed, 31 May 2023 at 11:15, Samrat Deb <decordea...@gmail.com> wrote: > > > > > 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 "flink-connector-redshift" according to > > > FLIP[1]. The rationale behind this decision is to empower users who are > > > familiar with their Redshift setup and have specific expectations for > the > > > sink. By making these configurations mandatory, users can have more > > control > > > and flexibility in configuring the connector to meet their > requirements. > > > > > > However, I am open to receiving feedback on whether it would be > > beneficial > > > to make the configuration options non-mandatory and set default values > > for > > > them. If you believe there are advantages to having default values or > any > > > other suggestions, please share your thoughts. Your feedback is highly > > > appreciated. > > > > > > > 2. For Source, does it both support batch read and streaming read? > > > > > > Redshift currently does not provide native support for streaming reads, > > > although it does support streaming writes[2]. As part of the plan, I > > intend > > > to conduct a proof of concept and benchmarking to explore the > > possibilities > > > of implementing streaming reads using the Flink JDBC connector, as > > Redshift > > > is JDBC compatible. > > > However, it is important to note that, in the initial phase of > > > implementation, the focus will primarily be on supporting batch reads > > > rather than streaming reads. This approach will allow us to deliver a > > > robust and reliable solution for batch processing in phase 2 of the > > > implementation. > > > > > > [1] > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-307%3A++Flink+Connector+Redshift > > > [2] > > > > > > > > > https://docs.aws.amazon.com/redshift/latest/dg/materialized-view-streaming-ingestion.html > > > > > > Bests, > > > Samrat > > > > > > On Wed, May 31, 2023 at 8:03 AM liu ron <ron9....@gmail.com> wrote: > > > > > > > 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.mode` and `write.mode? > > > > 2. For Source, does it both support batch read and streaming read? > > > > > > > > > > > > Best, > > > > Ron > > > > > > > > Samrat Deb <decordea...@gmail.com> 于2023年5月30日周二 17:15写道: > > > > > > > > > [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 <decordea...@gmail.com> > > > > wrote: > > > > > > > > > > > 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 with Flink Sink API (*FLIP-171* > > > > > > < > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-171%3A+Async+Sink > > > > > > > > > > > ) > > > > > > > > > > > > > > > > > > > About the implementation phases, How about prioritizing support > > for > > > > the > > > > > > Datastream Sink API and TableSink API in the first phase? > > > > > > I can completely resonate with you to prioritize support for > > > Datastream > > > > > > Sink API and TableSink API in the first phase. > > > > > > I will update the FLIP[1] as you have suggested. > > > > > > > > > > > > > It seems that the primary use cases for the Redshift connector > > are > > > > > > acting as a sink for processed data by Flink. > > > > > > Yes, majority ask and requirement for Redshift connector is sink > > for > > > > > > processed data by Flink. > > > > > > > > > > > > Bests, > > > > > > Samrat > > > > > > > > > > > > On Tue, May 30, 2023 at 12:35 PM Leonard Xu <xbjt...@gmail.com> > > > wrote: > > > > > > > > > > > >> 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 API in the first phase? It > seems > > > > that > > > > > the > > > > > >> primary use cases for the Redshift connector are acting as a > sink > > > for > > > > > >> processed data by Flink. > > > > > >> > > > > > >> Best, > > > > > >> Leonard > > > > > >> > > > > > >> > > > > > >> > On May 29, 2023, at 12:51 PM, Samrat Deb < > decordea...@gmail.com > > > > > > > > wrote: > > > > > >> > > > > > > >> > 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 > > to > > > > > >> deliver > > > > > >> > fast performance for even the most demanding and unpredictable > > > > > >> workloads. > > > > > >> > Redshift is one of the widely used warehouse solutions in the > > > > current > > > > > >> > market. > > > > > >> > > > > > > >> > Building flink connector redshift will allow flink users to > have > > > > > source > > > > > >> and > > > > > >> > sink directly to redshift. It will help flink to expand the > > scope > > > to > > > > > >> > redshift as a new connector in the ecosystem. > > > > > >> > > > > > > >> > I would like to start a discussion on the FLIP-307: Flink > > > connector > > > > > >> > redshift [2]. > > > > > >> > Looking forward to comments, feedbacks and suggestions from > the > > > > > >> community > > > > > >> > on the proposal. > > > > > >> > > > > > > >> > [1] > > https://docs.aws.amazon.com/redshift/latest/mgmt/welcome.html > > > > > >> > [2] > > > > > >> > > > > > > >> > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-307%3A++Flink+Connector+Redshift > > > > > >> > > > > > > >> > > > > > > >> > > > > > > >> > Bests, > > > > > >> > Samrat > > > > > >> > > > > > >> > > > > > > > > > > > > > > >