Thanks for the FLIP. I agree that there is a real usecase for metrics sink vs metric reporter. The metric reporters in Flink cover metrics about the Flink job, and a sink is used when the metrics are the _data_.
+1 on the FLIP ID, can you please fix that? With regard to this FLIP. 1/ 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 ther server? 2/ Are there any concerns around Prometheus versioning or is the API backwards compatible? Which versions of Prometheus will we be supporting? 3/ 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? Thanks, Danny On Wed, May 17, 2023 at 4:34 PM Teoh, Hong <lian...@amazon.co.uk.invalid> wrote: > Thanks Karthi for creating the FLIP! > > Re: Martijn, > > As I understand it, the motivation for the Prometheus Sink is for users > who want to write metrics to a Prometheus remote_write endpoint as an > output of their job graph, so it would be good to treat it as a first-class > citizen and do it as part of Flink’s data flow. This way, we would benefit > from at least once guarantees, scaling, state management. > > 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 think you might be referring to the metrics of the Flink job itself > (e.g. CPU / memory metrics). For this use case, I would agree that we > should not use this sink but instead use our metric reporters. > > Regards, > Hong > > > > > On 16 May 2023, at 03:37, Lijie Wang <wangdachui9...@gmail.com> wrote: > > > > CAUTION: This email originated from outside of the organization. Do not > click links or open attachments unless you can confirm the sender and know > the content is safe. > > > > > > > > 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 > >>> > >> > >