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 with an older version of flink-connector-jdbc. It's not obligatory for flink-connector-aws to constantly align with the most current version. This strategy disengages direct dependencies between the two in their respective release cycles. By adopting this approach, flink-connector-aws can maintain its specific dependencies on JDBC and the preferred version paralleling its approach with the AWS SDK's handling of dependencies.
Bests, Samrat On Tue, Jul 4, 2023 at 7:09 PM Martijn Visser <martijnvis...@apache.org> wrote: > 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 <decordea...@gmail.com> wrote: > > > 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, release, and dependency > > management process? > > > > We intend to maintain the current release cycle for the Flink Connector > > AWS. However, we will be incorporating a dependency of Flink Connector > JDBC > > to most recent version, which will synchronize between the flink > connector > > AWS and flink connector JDBC connectors releases for new Flink version > > support. Additionally, the Flink Connector Redshift will exclusively > > utilize the public API, minimizing the occurrence of immediate breaking > > changes. > > > > > 2 Some configuration option names can be improved to match the naming > > style of existing configuration options, for example: > > table -> table-name > > query -> scan.query > > aws-iam-role -> aws.iam-role > > read.mode -> scan.read.mode: similar to scan.startup.mode , and maybe we > > will have lookup.read.mode > > write.mode -> sink.write.mode > > > > Updated the FLIP-307 [1] > > > > > 3 The source of Redshift connector supports JDBC queries, IIUC, we can > > also support the LookupTableSource as well? > > > > we will support `LookupTableSource` in redshift connector source > > > > [1] > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-307%3A++Flink+Connector+Redshift > > > > Bests, > > Samrat > > > > On Wed, Jun 21, 2023 at 4:47 PM Leonard Xu <xbjt...@gmail.com> wrote: > > > > > 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 different repositories and depend on Flink dependency, could > > you > > > explain how you establish the versioning, release, and dependency > > > management process? > > > > > > 2 Some configuration option names can be improved to match the naming > > > style of existing configuration options, for example: > > > table -> table-name > > > query -> scan.query > > > aws-iam-role -> aws.iam-role > > > read.mode -> scan.read.mode: similar to scan.startup.mode , and maybe > we > > > will have lookup.read.mode > > > write.mode -> sink.write.mode > > > > > > 3 The source of Redshift connector supports JDBC queries, IIUC, we can > > > also support the LookupTableSource as well? > > > > > > Best, > > > Leonard > > > > > > > On Jun 21, 2023, at 4:57 PM, Samrat Deb <decordea...@gmail.com> > wrote: > > > > > > > > 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 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. > > > > > > > > Noted. It's a valid consideration to start compatibility with the > Async > > > > API. > > > > > > > > Bests, > > > > Samrat > > > > > > > > > > > > On Mon, Jun 19, 2023 at 5:28 PM Martijn Visser < > > martijnvis...@apache.org > > > > > > > > wrote: > > > > > > > >> 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. > > > >> > > > >> Thanks, > > > >> > > > >> Martijn > > > >> > > > >> On Fri, Jun 9, 2023 at 7:22 PM Jing Ge <j...@ververica.com.invalid> > > > wrote: > > > >> > > > >>> Hi Samrat, > > > >>> > > > >>> The FLIP looks good, thanks! > > > >>> > > > >>> Best regards, > > > >>> Jing > > > >>> > > > >>> > > > >>> On Tue, Jun 6, 2023 at 8:16 PM Samrat Deb <decordea...@gmail.com> > > > 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 :-) > > > >>>> I will request for your review once m ready with PR :-) > > > >>>> > > > >>>> Bests, > > > >>>> Samrat > > > >>>> > > > >>>> On Tue, Jun 6, 2023 at 11:43 PM Samrat Deb <decordea...@gmail.com > > > > > >>> wrote: > > > >>>> > > > >>>>> 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 up the possibility of implementing exactly-once > > semantics > > > >>> with > > > >>>>> the connector. > > > >>>>> However, I believe it would be prudent to start with a more > focused > > > >>> scope > > > >>>>> for the initial phase of implementation and defer the exact-once > > > >>> support > > > >>>>> for subsequent iterations. > > > >>>>> > > > >>>>> Before finalizing the approach, I would greatly appreciate your > > > >>> thoughts > > > >>>>> and suggestions on this matter. > > > >>>>> Should we prioritize the initial implementation without > > exactly-once > > > >>>>> support, or would you advise incorporating it right from the > start? > > > >>>>> Your insights and experiences would be immensely valuable in > making > > > >>> this > > > >>>>> decision. > > > >>>>> > > > >>>>> > > > >>>>> [1] > > > >>>>> > > > >>>> > > > >>> > > > >> > > > > > > https://docs.aws.amazon.com/redshift/latest/dg/t_updating-inserting-using-staging-tables-.html > > > >>>>> [2] > > > >> https://docs.aws.amazon.com/redshift/latest/dg/merge-examples.html > > > >>>>> > > > >>>>> Bests, > > > >>>>> Samrat > > > >>>>> > > > >>>>> On Mon, Jun 5, 2023 at 7:09 PM Jing Ge > <j...@ververica.com.invalid > > > > > > >>>> wrote: > > > >>>>> > > > >>>>>> 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 < > decordea...@gmail.com> > > > >>>> wrote: > > > >>>>>> > > > >>>>>>> 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 connector? > > > >>>>>>> > > > >>>>>>> Here are the additional features that the Flink connector for > AWS > > > >>>>>> Redshift > > > >>>>>>> can provide on top of using JDBC: > > > >>>>>>> > > > >>>>>>> 1. Integration with AWS Redshift Workload Management (WLM): AWS > > > >>>> Redshift > > > >>>>>>> allows you to configure WLM[1] to manage query prioritization > and > > > >>>>>> resource > > > >>>>>>> allocation. The Flink connector for Redshift will be agnostic > to > > > >> the > > > >>>>>>> configured WLM and utilize it for scaling in and out for the > > sink. > > > >>>> This > > > >>>>>>> means that the connector can leverage the WLM capabilities of > > > >>> Redshift > > > >>>>>> to > > > >>>>>>> optimize the execution of queries and allocate resources > > > >> efficiently > > > >>>>>> based > > > >>>>>>> on your defined workload priorities. > > > >>>>>>> > > > >>>>>>> 2. Abstraction of AWS Redshift Quotas and Limits: AWS Redshift > > > >>> imposes > > > >>>>>>> certain quotas and limits[2] on various aspects such as the > > number > > > >>> of > > > >>>>>>> clusters, concurrent connections, queries per second, etc. The > > > >> Flink > > > >>>>>>> connector for Redshift will provide an abstraction layer for > > > >> users, > > > >>>>>>> allowing them to work with Redshift without having to worry > about > > > >>>> these > > > >>>>>>> specific limits. The connector will handle the management of > > > >>>> connections > > > >>>>>>> and queries within the defined quotas and limits, abstracting > > away > > > >>> the > > > >>>>>>> complexity and ensuring compliance with Redshift's > restrictions. > > > >>>>>>> > > > >>>>>>> These features aim to simplify the integration of Flink with > AWS > > > >>>>>> Redshift, > > > >>>>>>> providing optimized resource utilization and transparent > handling > > > >> of > > > >>>>>>> Redshift-specific limitations. > > > >>>>>>> > > > >>>>>>> Bests, > > > >>>>>>> Samrat > > > >>>>>>> > > > >>>>>>> [1] > > > >>>>>>> > > > >>>>>>> > > > >>>>>> > > > >>>> > > > >>> > > > >> > > > > > > https://docs.aws.amazon.com/redshift/latest/dg/cm-c-implementing-workload-management.html > > > >>>>>>> [2] > > > >>>>>>> > > > >>>>>>> > > > >>>>>> > > > >>>> > > > >>> > > > >> > > > > > > https://docs.aws.amazon.com/redshift/latest/mgmt/amazon-redshift-limits.html > > > >>>>>>> > > > >>>>>>> On Sat, Jun 3, 2023 at 11:40 PM Samrat Deb < > > decordea...@gmail.com > > > >>> > > > >>>>>> wrote: > > > >>>>>>> > > > >>>>>>>> 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 > > > >>>>>>>>>>>>>>> > > > >>>>>>>>>>>>>>> > > > >>>>>>>>>>>>> > > > >>>>>>>>>>>> > > > >>>>>>>>>>> > > > >>>>>>>>>> > > > >>>>>>>>> > > > >>>>>>>> > > > >>>>>>> > > > >>>>>> > > > >>>>> > > > >>>> > > > >>> > > > >> > > > > > > > > >