Hi Danny Please consider this is not an AWS-specific connector and will not depend on flink-connector-aws-base. Prometheus remote-write specs deem authentication as out-of-scope [1].
Amazon Managed Prometheus uses request signing to authenticate remote-writes. To support AMP but also keep the connector generic, the idea is to define a genetic request signing interface, and provide an implementation for AMP. Also, to keep the connector API generic, we think the best approach is to pass a signer instance to the sink builder. I am not aware of Prometheus remote-write authentication schemes, other than request signing. The actual open questions are, in my opinion: 1. should we add the interface for other authentications. If so, what schemes? 2. should the AMP request signer be part of the connector or distributed as a separate dependency? Regards Lorenzo [1] https://prometheus.io/docs/concepts/remote_write_spec/#out-of-scope On Mon, 18 Sept 2023 at 17:22, Danny Cranmer <dannycran...@apache.org> wrote: > Thanks for the reply Lorenzo. > > > Static credentials are just for the sake of the example. The current > prototype implementation already uses DefaultAWSCredentialsProviderChain > that supports static and dynamic credentials. We can make the credential > provider configurable. > > The point here was that flink-connector-aws-base provides a common way to > define AWS credentials in config. It would be nice to reuse this mechanism > for consistency. I am wondering if we will reuse this approach, and if so > how the dependency hierarchy will look? We had a similar discussion > regarding the redshift connector [1]. The FLIP uses > "AWS_ACCESS_KEY_ID"/"AWS_SECRET_ACCESS_KEY"/etc which look like the > constants in flink-connector-aws [2]? > > Thanks, > Danny > > [1] https://lists.apache.org/thread/fowhltc1n0tn4627ycwhrd5p8ds77lm8 > [2] > > https://github.com/apache/flink-connector-aws/blob/main/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java#L91 > > On Mon, Sep 18, 2023 at 3:50 PM Lorenzo Nicora <lorenzo.nic...@gmail.com> > wrote: > > > Hello > > > > I am also re-sending an old answer I sent on May 24th, that, for some > > reason, did not appear in the thread. > > ---------------------- > > Q1) The fact we are using the remote write feature is not covered beyond > > the code example. Can we add details on this to make it clear? > Additionally > > would this support _any_ Prometheus server or do we need to enable remote > > endpoint feature on the server? > > > > A1) We use the remote-write API. The server must provide a standard > > remote-write endpoint. Remote-write specs do not say anything about > > authentication. At the moment we are planning to support 1/ > unauthenticated > > requests, 2/ AWS signed requests for AMP. The idea is the signer > interface > > allows transformations of the request URL. Request payload cannot be > > modified and must be Protobuf, as by spec. > > > > > > > > Q2) Are there any concerns around Prometheus versioning or is the API > > backwards compatible? Which versions of Prometheus will we be supporting > > > > A2) We are using the only version of Prometheus Remote-Write specs > > available v1.0, defined in Remote-Write spec document [1] published April > > 2023. There was a previous v0.1 draft version of the same specs. We will > > probably also be compatible with the draft version, but I still have to > > check the differences. > > > > > > > > Q3) With regard to the "AmazonPrometheusRequestSigner" the example has > > static creds. Can we integrate with the AWS Util to support all > credential > > providers, static and dynamic? > > > > A3) Static credentials are just for the sake of the example. The current > > prototype implementation already uses DefaultAWSCredentialsProviderChain > > that supports static and dynamic credentials. We can make the credential > > provider configurable. > > > > Lorenzo > > > > [1] https://prometheus.io/docs/concepts/remote_write_spec/ > > [2] > > > > > https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit > > Lorenzo > > > > > > On Sun, 17 Sept 2023 at 09:51, Ahmed Hamdy <hamdy10...@gmail.com> wrote: > > > > > Thanks Lorenzo, > > > Looking forward to the PRs. > > > Best Regards > > > Ahmed Hamdy > > > > > > > > > On Sat, 16 Sept 2023 at 06:27, Lorenzo Nicora < > lorenzo.nic...@gmail.com> > > > wrote: > > > > > > > Hello > > > > > > > > (apologies if this is a duplicate reply) > > > > > > > > I was working with Karthi on this connector, and I have taken over > the > > > > development. > > > > We have a working version we would like to submit to the community. > > > > > > > > The renumbered FLIP-312 is also updated with more details [1]. > > > > Happy to answer any questions. > > > > > > > > Regards > > > > Lorenzo > > > > > > > > [1] > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-312%3A+Prometheus+Sink+Connector > > > > > > > > On Mon, 21 Aug 2023, 13:06 Ahmed Hamdy, <hamdy10...@gmail.com> > wrote: > > > > > > > > > Hello Karthi > > > > > Is this FLIP still in progress? I see the FLIP not updated & > couldn't > > > > find > > > > > open JIRAs. > > > > > I am happy to take over if you are no longer working on this. > > > > > Best Regards > > > > > Ahmed Hamdy > > > > > > > > > > > > > > > On Mon, 22 May 2023 at 14:49, Martijn Visser < > > martijnvis...@apache.org > > > > > > > > > wrote: > > > > > > > > > > > Hi all, > > > > > > > > > > > > > For example, a user might want to read in logs, perform some > > > > > aggregations > > > > > > and publish it into a metrics store for visualisation. This might > > be > > > a > > > > > > great use-case for reducing the cardinality of metrics! > > > > > > > > > > > > I can see that. What I would like to see in the FLIP is a > proposal > > on > > > > the > > > > > > boundaries of the metrics reporter vs the Prometheus sink. I > think > > > it's > > > > > > important that we make clear when to use a metric reporter and > when > > > > not. > > > > > I > > > > > > can imagine that there will be Flink users who think that they > can > > > get > > > > > data > > > > > > from the metric reporter, make aggregrations in Flink and then > > store > > > it > > > > > > using the Prometheus sink. > > > > > > > > > > > > Overall, I think more context must be added to the FLIP, > especially > > > on > > > > > the > > > > > > motivation. > > > > > > > > > > > > Best regards, > > > > > > > > > > > > Martijn > > > > > > > > > > > > On Fri, May 19, 2023 at 4:28 PM Karthi Thyagarajan < > > > > > kar...@karthitect.com> > > > > > > wrote: > > > > > > > > > > > > > Hi Lijie > > > > > > > > > > > > > > Thank you for pointing this out. I've corrected it [1]. Also, > > this > > > > page > > > > > > > [2] still shows 178 and 229 as available, which is why I picked > > it > > > > up. > > > > > > > > > > > > > > Thanks > > > > > > > Karthi > > > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-312%3A+Prometheus+Sink+Connector > > > > > > > [2] > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals > > > > > > > > > > > > > > On May 15, 2023, at 9:37 PM, Lijie Wang < > > wangdachui9...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > Hi Karthi, > > > > > > > > > > > > > > I think you are using a wrong FLIP id, the FLIP-229 has already > > be > > > > > > used[1]. > > > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-229%3A+Introduces+Join+Hint+for+Flink+SQL+Batch+Job > > > > > > > > > > > > > > Best, > > > > > > > Lijie > > > > > > > > > > > > > > Martijn Visser <martijnvis...@apache.org> 于2023年5月16日周二 > 04:44写道: > > > > > > > > > > > > > > Hi Karthi, > > > > > > > > > > > > > > Thanks for the FLIP and opening up the discussion. My main > > question > > > > is: > > > > > > why > > > > > > > should we create a separate connector and not use and/or > improve > > > the > > > > > > > existing integrations with Prometheus? I would like to > understand > > > > more > > > > > so > > > > > > > that it can be added to the motivation of the FLIP. > > > > > > > > > > > > > > Best regards, > > > > > > > > > > > > > > Martijn > > > > > > > > > > > > > > On Mon, May 15, 2023 at 6:03 PM Karthi Thyagarajan < > > > > > > kar...@karthitect.com> > > > > > > > wrote: > > > > > > > > > > > > > > > Hello all, > > > > > > > > > > > > > > > > We would like to start a discussion thread on FLIP-229: > > > Prometheus > > > > > Sink > > > > > > > > Connector [1] where we propose to provide a sink connector > for > > > > > > Prometheus > > > > > > > > [2] based on the Async Sink [3]. Looking forward to comments > > and > > > > > > > feedback. > > > > > > > > Thank you. > > > > > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-229%3A+Prometheus+Sink+Connector > > > > > > > > [2] https://prometheus.io/ > > > > > > > > [3] > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-171%3A+Async+Sink > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >