Re: [Flink kubernetes operator] custom dependent resources?

2025-01-14 Thread Andrew Otto
+1 to FlinkDeployment helm chart approach. Here is Wikimedia Foundation's chart: (it is heavily dependent on other WMF templates, so you probably can't just use it...but maybe!) https://gerrit.wikimedia.org/r/plugins/gitiles/operations/deployment-charts/+/refs/heads/master/charts/flink-app/ > he

Re: Flink CDC blog

2024-12-11 Thread Andrew Otto
Thanks Robin, this is very helpful! FWIW, at the Wikimedia Foundation, we are exploring this too. - T373144 [SPIKE] Learn and document how to use Flink-CDC from MediaWiki MariaDB locally - T370354 [SPIKE] Experiment with approaches for a incremental upd

Re: [Flink CDC] What's the difference between Pipeline connectors and Flink Source connectors?

2024-12-02 Thread Andrew Otto
Hi Robin! IIUC, the difference is: - Pipeline connectors can be used as a fully contained yaml configured CDC pipeline job - Flink CDC sources are Flink Table connectors that can connect

Re: Flink CDC -> Kafka -> Paimon?

2024-10-28 Thread Andrew Otto
nk is already available in FlinkCDC, so we can directly write > data from MariaDB to Paimon to reduce the number of components and links > that need to be maintained, We will also follow up on any issues > encountered in Paimon Pipeline sink. > > [1] https://issues.apache.org/jira/

Flink CDC -> Kafka -> Paimon?

2024-10-25 Thread Andrew Otto
g. Is Kafka Connect+Debezium used by Flink to support debezium-json formatted messages? I tried passing properties.value.converter.schemas.enable: true to the flink-cdc pipeline kafka sink but that did not work (as expected). Thank you! -Andrew Otto Wikimedia Foundation P.S. Context for what we are trying to do is here: T373144 [SPIKE] Learn and document how to use Flink-CDC from MediaWiki MariaDB locally <https://phabricator.wikimedia.org/T373144>

Re: Get types from Row

2024-08-28 Thread Andrew Otto
I'm not sure if this helps with your need to vary the Sink's schema at runtime, but FWIW you can get the 'schema' of the input datastream via DataStream.getType

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-28 Thread Andrew Otto
ttps://nightlies.apache.org/flink/flink-cdc-docs-stable/ >> >> [2] https://issues.apache.org/jira/browse/FLINK-34840 >> >> >> >> Regards, >> >> Xiqian >> >> >> >> >> >> *De : *Andrew Otto >> *Date : *vendred

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-24 Thread Andrew Otto
table list. On Fri, May 24, 2024 at 11:04 AM Andrew Otto wrote: > Indeed, using Flink-CDC to write to Flink Sink Tables, including Iceberg, > is supported. > > What is not is the automatic syncing of entire databases, with schema > evolution and detection of new (and dropped?) tab

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-24 Thread Andrew Otto
t; >> On Fri, May 24, 2024 at 8:36 AM Giannis Polyzos >> wrote: >> >>> I’m curious if there is any reason for choosing Iceberg instead of >>> Paimon (other than - iceberg is more popular). >>> Especially for a use case like CDC that iceberg struggles to s

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-24 Thread Andrew Otto
cially for a use case like CDC that iceberg struggles to support. > > On Fri, 24 May 2024 at 3:22 PM, Andrew Otto wrote: > >> Interesting thank you! >> >> I asked this in the Paimon users group: >> >> How coupled to Paimon catalogs and tables is the cdc part of P

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-24 Thread Andrew Otto
mething like this in a nutshell: >>> - Make sure that on job start the latest Iceberg schema is read from the >>> Iceberg table >>> - Throw a SuppressRestartsException when data arrives with the wrong >>> schema >>> - Use Flink Kubernetes Operator to restart you

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-23 Thread Andrew Otto
- Make sure that on job start the latest Iceberg schema is read from the > Iceberg table > - Throw a SuppressRestartsException when data arrives with the wrong schema > - Use Flink Kubernetes Operator to restart your failed jobs by setting > kubernetes.operator.job.restart.failed > > T

Re: "Self-service ingestion pipelines with evolving schema via Flink and Iceberg" presentation recording from Flink Forward Seattle 2023

2024-05-23 Thread Andrew Otto
Wow, I would LOVE to see this talk. If there is no recording, perhaps there are slides somewhere? On Thu, May 23, 2024 at 11:00 AM Carlos Sanabria Miranda < sanabria.miranda.car...@gmail.com> wrote: > Hi everyone! > > I have found in the Flink Forward website the following presentation: > "Self

Re: Schema Evolution & Json Schemas

2024-02-25 Thread Andrew Otto
> the following code generator Oh, and FWIW we avoid code generation and POJOs, and instead rely on Flink's Row or RowData abstractions. On Sun, Feb 25, 2024 at 10:35 AM Andrew Otto wrote: > Hi! > > I'm not sure if this totally is relevant for you, but we use JSON

Re: Schema Evolution & Json Schemas

2024-02-25 Thread Andrew Otto
ttps://techblog.wikimedia.org/2020/09/10/wikimedias-event-data-platform-or-json-is-ok-too/> ! -Andrew Otto Wikimedia Foundation On Fri, Feb 23, 2024 at 1:58 AM Salva Alcántara wrote: > I'm facing some issues related to schema evolution in combination with the > usage of Json Sch

Re: Handling Schema Variability and Applying Regex Patterns in Flink Job Configuration

2023-11-06 Thread Andrew Otto
> unpredictable file schema(Table API) in the source directory You'll probably have to write some logic that helps predict the schema :) Are there actual schemas for the CSV files somewhere? JSONSchema or something of the like?At Wikimedia we use JSONSchema (not with CSV data, but it could

Re: Enhancing File Processing and Kafka Integration with Flink Jobs

2023-10-28 Thread Andrew Otto
I wonder if you could use this fact to query the committed checkpoints > and move them away after the job is done. > > This is not a robust solution, I would advise against it. > > Best, > Alexander > > On Fri, 27 Oct 2023 at 16:41, Andrew Otto wrote: > >> For moving

Re: Enhancing File Processing and Kafka Integration with Flink Jobs

2023-10-27 Thread Andrew Otto
For moving the files: > It will keep the files as is and remember the name of the file read in checkpointed state to ensure it doesnt read the same file twice. I wonder if you could use this fact to query the committed checkpoints and move them away after the job is done. I think it should even b

Re: Using HybridSource

2023-07-05 Thread Andrew Otto
Hm. I wonder if you could implement a custom Deserializer that wraps both the CSV and Protobuf deserializer, and conditionally chooses which one to use. As long as the final TypeInformation returned by the Source is the same in either case, I think it should work? > Kafka comes from protobuf while

Re: Flink Stream processing with Dynamic Conditional Sinks

2023-06-05 Thread Andrew Otto
more sinks changes this topology. You might be able to get away with dynamically submitting multiple jobs in a single Flink application, but you'd have to restart the application every time you add a new job. I've not tried this though, so hopefully someone smarter can come in and advise

Re: Flink Kubernetes Operator lifecycle state count metrics question

2023-05-22 Thread Andrew Otto
quot;rdf_streaming_updater",name="flink_kubernetes_operator",host="flink_kubernetes_operator_86b888d6b6_gbrt4",namespace="flink_operator",} 1.0 It looks like maybe this metric is not reporting per namespace, but a global count. On Mon, May 22, 2023 at 2:56 PM Andrew O

Re: Flink Kubernetes Operator lifecycle state count metrics question

2023-05-22 Thread Andrew Otto
Oh, FWIW, I do have operator HA enabled with 2 replicas running, but in my examples there, I am curl-ing the leader flink operator pod. On Mon, May 22, 2023 at 2:47 PM Andrew Otto wrote: > Hello! > > I'm doing some grafana+prometheus dashboarding for > flink-kubernetes-o

Flink Kubernetes Operator lifecycle state count metrics question

2023-05-22 Thread Andrew Otto
t{resourcetype="FlinkDeployment",resourcens="stream_enrichment_poc",name="flink_kubernetes_operator",host="flink_kubernetes_operator_86b888d6b6_gbrt4",namespace="flink_operator",} 1.0 Is it possible that flink_k8soperator_namespace_Lifecycle_State_STABLE_Count is being reported as an incrementing counter instead of a guage? Thanks -Andrew Otto Wikimedia Foundation

Re: flink-kubernetes-operator HA k8s RoleBinding for Leases?

2023-05-09 Thread Andrew Otto
role: flink-operator > > > am I missing something?*​* > > > -- > *From:* Gyula Fóra > *Sent:* Tuesday, May 9, 2023 7:43 AM > *To:* Andrew Otto > *Cc:* User > *Subject:* Re: flink-kubernetes-operator HA k8s RoleBinding for Leases? > &

flink-kubernetes-operator HA k8s RoleBinding for Leases?

2023-05-08 Thread Andrew Otto
netes-operator is deployed. I think that for HA, the flink-kubernetes-operator is going to be asking k8s for Leases in its own namespace, right? Is this a bug, or am I doing something wrong? I'd file a JIRA, but I betcha I'm just doing something wrong (unless I'm the first person

Re: Flink Job across Data Centers

2023-04-12 Thread Andrew Otto
Hi, I asked a similar question in this thread , which might have some relevant info. On Wed, Apr 12, 2023 at 7:23 AM Chirag Dewan via user wrote: > Hi, > > Can anyone share any experience on running Flink jobs across data centers?

Re: Re: KafkaSource consumer group

2023-03-31 Thread Andrew Otto
Hi, FWIW, I asked a similar question here: https://lists.apache.org/thread/1f01zo1lqcmhvosptpjlm6k3mgx0sv1m :) On Fri, Mar 31, 2023 at 3:57 AM Roberts, Ben (Senior Developer) via user < user@flink.apache.org> wrote: > Hi Gordon, > > Thanks for the reply! > I think that makes sense. > > The rea

Re: [ANNOUNCE] Flink Table Store Joins Apache Incubator as Apache Paimon(incubating)

2023-03-27 Thread Andrew Otto
Exciting! If this ends up working well, Wikimedia Foundation would love to try it out! On Mon, Mar 27, 2023 at 8:39 AM Matthias Pohl via user < user@flink.apache.org> wrote: > Congratulations and good luck with pushing the project forward. > > On Mon, Mar 27, 2023 at 2:35 PM Jing Ge via user >

Re: Handling JSON Serialization without Kryo

2023-03-27 Thread Andrew Otto
to JSON and Protobuf Schemas <https://lists.apache.org/thread/7gnllmggbqwnoj22dfcbrmngr16dbnxb> Good luck! -Andrew Otto Wikimedia Foundation On Wed, Mar 22, 2023 at 8:07 AM Rion Williams wrote: > Hi Ken, > > I’m going to profile the job today to try and get a better handle on where

Re: Kubernetes operator set container resources and limits

2023-03-13 Thread Andrew Otto
Hi, > return to the same values from jobManager.resource FlinkDeployment manifest parameter I believe this is the correct way; using jobManager.resources and taskManager.resource

Re: Reusing the same OutputTag in multiple ProcessFunctions

2023-02-15 Thread Andrew Otto
gh that the various tag instances involved all have the same >> String id. (That's why the id exists.) >> >> David >> >> On Tue, Feb 14, 2023 at 11:51 AM Andrew Otto wrote: >> > >> > Hi, >> > >> > I'm attempting to imp

Reusing the same OutputTag in multiple ProcessFunctions

2023-02-14 Thread Andrew Otto
instance in multiple ProcessFunctions? 2. If so, is my problem pyflink / python / pickle specific? Thanks! -Andrew Otto Wikimedia Foundation

Re: Pyflink Side Output Question and/or suggested documentation change

2023-02-13 Thread Andrew Otto
Thank you! On Mon, Feb 13, 2023 at 5:55 AM Dian Fu wrote: > Thanks Andrew, I think this is a valid advice. I will update the > documentation~ > > Regards, > Dian > > , > > On Fri, Feb 10, 2023 at 10:08 PM Andrew Otto wrote: > >> Question about side outputs

Pyflink Side Output Question and/or suggested documentation change

2023-02-10 Thread Andrew Otto
erhaps the docs should be updated to say so? -Andrew Otto Wikimedia Foundation

Re: Kafka Sink Kafka Producer metrics?

2023-02-07 Thread Andrew Otto
in your Kafka client >> properties map. This is a JIRA to document the feature: >> https://issues.apache.org/jira/browse/FLINK-30932 >> >> Best, >> Mason >> >> On Mon, Feb 6, 2023 at 11:49 AM Andrew Otto wrote: >> >>> Hi! >>>

Kafka Sink Kafka Producer metrics?

2023-02-06 Thread Andrew Otto
g-1> does not emit KafkaProducer metrics <https://kafka.apache.org/documentation/#producer_monitoring>. Is this correct? If so, why not? Thanks, -Andrew Otto Wikimedia Foundation

Re: Using pyflink from flink distribution

2023-01-31 Thread Andrew Otto
the things necessary needed to run JVM apps in > the PyFlink package and so I think you could just try this way. > > Regards, > Dian > > On Mon, Jan 30, 2023 at 9:58 PM Andrew Otto wrote: > >> Thanks Dian! >> >> > >> Is using pyflink from the flink dis

Re: Using pyflink from flink distribution

2023-01-30 Thread Andrew Otto
ges located under > /usr/local/lib/python3.7/dist-packages/pyflink/lib manually after `pip > install apache-flink`. It will use the JAR packages located under > $FLINK_HOME/lib. > > >> Is using pyflink from the flink distribution tarball (without pip) not > a supported way t

Re: Which flink version is compatible with beam

2023-01-30 Thread Andrew Otto
Hi, I'm not sure about beam, but Flink is not officially compatible with python3.10. https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/python/installation/ > Python version (3.6, 3.7 or 3.8) is required for PyFlink. On Fri, Jan 27, 2023 at 11:50 PM P Singh wrote: > Hi, > > I

Re: Using pyflink from flink distribution

2023-01-26 Thread Andrew Otto
are in opt/python. On Thu, Jan 26, 2023 at 10:10 AM Andrew Otto wrote: > Let me ask a related question: > > We are building our own base Flink docker image. We will be deploying > both JVM and python apps via flink-kubernetes-operator. > > Is there any reason not to install

Re: Using pyflink from flink distribution

2023-01-26 Thread Andrew Otto
Let me ask a related question: We are building our own base Flink docker image. We will be deploying both JVM and python apps via flink-kubernetes-operator. Is there any reason not to install Flink in this image via `pip install apache-flink` and use it for JVM apps? -Andrew Otto Wikimedia

Using pyflink from flink distribution

2023-01-24 Thread Andrew Otto
nk from the flink distribution tarball (without pip) not a supported way to use pyflink? Thanks! -Andrew Otto Wikimedia Foundation

Re: Flink Kubernetes Operator podTemplate and 'app' pod label bug?

2023-01-23 Thread Andrew Otto
etty common app here. WDYT Gyula? >>> >>> On Thu, Jan 19, 2023 at 1:48 PM Gyula Fóra wrote: >>> >>>> Hi! >>>> >>>> The app label itself is used by Flink internally for a different >>>> purpose so it’s overriden. This is completel

Flink Kubernetes Operator podTemplate and 'app' pod label bug?

2023-01-19 Thread Andrew Otto
my chart or installation. I looked in flink-kubernetes-operator code and I didn't find where this was happening either. I am not setting e.g. kubernetes.jobmanager.labels <https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/config/#kubernetes-jobmanager-labels> . Is this expected? Thank you! -Andrew Otto Wikimedia Foundation

Re: What is the flink-kubernetes-operator webhook for?

2022-12-09 Thread Andrew Otto
;> >> webhooks in general are optional components of the k8s operator pattern. >> Mostly used for validation, sometimes for changing custom resources and >> handling multiple versions, etc. It's an optional component in the Flink >> Kubernetes Operator too. >> &

What is the flink-kubernetes-operator webhook for?

2022-12-09 Thread Andrew Otto
kubernetes-operator-docs-main/docs/try-flink-kubernetes-operator/quick-start/#deploying-the-operator>, do we need to install it for production use? FWIW, we will not be using FlinkSessionJob, so perhaps we don't need it if we don't use that? Thanks! -Andrew Otto Wikimedia Foundation

Re: flink-kubernetes-operator: image entrypoint misbehaves due to inability to write

2022-12-01 Thread Andrew Otto
FlinkDeployment spec.flinkConfiguration section > the user should be able to use env vars if this is added. > > Gyula > > On Thu, Dec 1, 2022 at 5:18 PM Andrew Otto wrote: > >> > Andrew please see my previous response, that covers the secrets case. >> > kubernetes.jo

Re: flink-kubernetes-operator: image entrypoint misbehaves due to inability to write

2022-12-01 Thread Andrew Otto
too, so the confs will be all in the same place. On Thu, Dec 1, 2022 at 9:30 AM Gyula Fóra wrote: > Andrew please see my previous response, that covers the secrets case. > > Gyula > > On Thu, Dec 1, 2022 at 2:54 PM Andrew Otto wrote: > >> > several failures to write

Re: flink-kubernetes-operator: image entrypoint misbehaves due to inability to write

2022-12-01 Thread Andrew Otto
> several failures to write into $FLINK_HOME/conf/. I'm working on building Flink and flink-kubernetes-operator images for the Wikimedia Foundation, and I found this strange as well. It makes sense in a docker

Re: Converting ResolvedSchema to JSON and Protobuf Schemas

2022-11-15 Thread Andrew Otto
> meaning that double and integer I meant to write: "meaning that double and bigint ... " :) On Tue, Nov 15, 2022 at 8:54 AM Andrew Otto wrote: > > Also thanks for showing me your pattern with the SchemaConversions and > stuff. Feels pretty clean and worked like a char

Re: Converting ResolvedSchema to JSON and Protobuf Schemas

2022-11-15 Thread Andrew Otto
ess for my application it would in general be better to use Avro or > Protobuf, since they retain a lot more type information when you convert > them back and forth… > Also thanks for showing me your pattern with the SchemaConversions and > stuff. Feels pretty clean and worked like a cha

Re: Converting ResolvedSchema to JSON and Protobuf Schemas

2022-11-09 Thread Andrew Otto
-schema in a schema registry, as I am pushing > the result-data to a Kafka topic. The result-schema is not known at > compile-time, so I need to find a way to compute it at runtime from the > resulting Flink Schema. > > -Theo > > (resent - again sorry, I forgot to add the othe

Re: Converting ResolvedSchema to JSON and Protobuf Schemas

2022-11-09 Thread Andrew Otto
AM Andrew Otto wrote: > Hello! > > I see you are talking about JSONSchema, not just JSON itself. > > We're trying to do a similar thing at Wikimedia and have developed some > tooling around this. > > JsonSchemaFlinkConverter > <https://gerrit.wikimedia.org/r/plu

Re: Converting ResolvedSchema to JSON and Protobuf Schemas

2022-11-09 Thread Andrew Otto
Hello! I see you are talking about JSONSchema, not just JSON itself. We're trying to do a similar thing at Wikimedia and have developed some tooling around this. JsonSchemaFlinkConverter

Re: KafkaSource, consumer assignment, and Kafka ‘stretch’ clusters for multi-DC streaming apps

2022-10-05 Thread Andrew Otto
(Ah, note that I am considering very simple streaming apps here, e.g. event enrichment apps. No windowing or complex state. The only state is the Kafka offsets, which I suppose would also have to be managed from Kafka, not from Flink state.) On Wed, Oct 5, 2022 at 9:54 AM Andrew Otto wrote

KafkaSource, consumer assignment, and Kafka ‘stretch’ clusters for multi-DC streaming apps

2022-10-05 Thread Andrew Otto
ssible to implement a KafkaSource that assigned tasks using Kafka's usual consumer assignment protocol. Hopefully someone more knowledgeable about Sources and TaskSplits, etc. could advise here. Thank you! - Andrew Otto Wikimedia Foundation

Re: Re: [DISCUSS] Contribution of Multi Cluster Kafka Source

2022-06-27 Thread Andrew Otto
This sounds very useful! Another potential use case: - Consuming from multiple kafka clusters in different datacenters/regions. I'm not sure if we would ultimately want to do this, but having it as an option when you need events from multiple kafka clusters to get the full history of changes (in

Re: How to use connectors in PyFlink 1.15.0 when not defined in Python API?

2022-06-24 Thread Andrew Otto
I've had success using the Java in pyflink via pyflink.java_gateway. Something like: from pyflink.java_gateway import get_gateway jvm = get_gateway() # then perhaps something like: FlinkKinesisConsumer = jvm. org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer There also seems to

Re: Flink, JSON, and JSONSchemas

2022-06-18 Thread Andrew Otto
tion converter, and have copy/pasted and modified Flink's deprecated JsonRowDeserializaitonSchema into our code. Thank you for your responses! -Andrew Otto Wikimedia Foundation On Fri, Jun 17, 2022 at 12:33 AM Shengkai Fang wrote: > Hi. > > > *1. Is there some easy

Flink, JSON, and JSONSchemas

2022-06-16 Thread Andrew Otto
At the Wikimedia Foundation, we use JSON and JSONSchemas for our events in Kafka. There are hundreds of these schemas and topics in Kafka. I'd like to provide library level integration between our 'Event Platform' JSON data and Flink. My main goal: *No case classes or POJOs. *The JSONSchemas s

Re: Flink Shaded dependencies and extending Flink APIs

2022-06-16 Thread Andrew Otto
> > dependencies are relocated in the same way. > > > > Another solution is that you can serialize then deserialize the > > "different" ObjectNode to do the conversion but this sacrifices the > > performance. > > > > Hope this could be helpful! >

Flink Shaded dependencies and extending Flink APIs

2022-06-09 Thread Andrew Otto
d into my project and just make it my own, but this feels quite obnoxious. Thank you! -Andrew Otto Wikimedia Foundation

Re: Notify on 0 events in a Tumbling Event Time Window

2022-05-09 Thread Andrew Otto
This sounds similar to a non streaming problem we had at WMF. We ingest all event data from Kafka into HDFS/Hive and partition the Hive tables in hourly directories. If there are no events in a Kafka topic for a given hour, we have no way of knowing if the hour has been ingested successfully. Fo

Re: Migrating Flink apps across cloud with state

2022-05-04 Thread Andrew Otto
Have you tried MirrorMaker 2's consumer offset translation feature? I have not used this myself, but it sounds like what you are looking for! https://issues.apache.org/jira/browse/KAFKA-9076 https://kafka.apache.org/26/javadoc/org/apache/kafka/connect/mirror/Checkpoint.html https://strimzi.io/blog

Re: [DISCUSS] Deprecate/remove Twitter connector

2022-01-31 Thread Andrew Otto
https://golb.hplar.ch/2018/02/Access-Server-Sent-Events-from-Java.html looks like a nice tutorial. On Mon, Jan 31, 2022 at 12:27 PM Andrew Otto wrote: > Any SSE/EventSource Java Client should work. I have not personally used > one. From a quick search, maybe > https://github.com/lau

Re: [DISCUSS] Deprecate/remove Twitter connector

2022-01-31 Thread Andrew Otto
e close to the use cases. >> >> Best regards >> Jing >> >> On Mon, Jan 31, 2022 at 3:07 PM Andrew Otto wrote: >> >>> Shameless plug: Maybe the Wikipedia EventStreams >>> <https://wikitech.wikimedia.org/wiki/Event_Platform/EventStreams> SSE

Re: [DISCUSS] Deprecate/remove Twitter connector

2022-01-31 Thread Andrew Otto
Shameless plug: Maybe the Wikipedia EventStreams SSE API would make for a great connector example in Flink? :D On Mon, Jan 31, 2022 at 5:41 AM Martijn Visser wrote: > Hi all, > > Than

Re: [FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Andrew Otto
that helps a little bit! :) -Andrew Otto On Thu, Jan 13, 2022 at 10:27 AM Martijn Visser wrote: > Hi everyone, > > I'm currently checking out different metadata platforms, such as Amundsen > [1] and Datahub [2]. In short, these types of tools try to address problems > related to

[no subject]

2021-10-12 Thread Andrew Otto
, just trying some things.) Is there something more I have to do to use HiveCatalog with a kerberized Hive Metastore? Should Flink support this out of the box? Thanks! - Andrew Otto SRE, Wikimedia Foundation

Re: Prometheus Reporter Enhancement

2021-05-18 Thread Andrew Otto
Sounds useful! On Tue, May 18, 2021 at 2:02 PM Mason Chen wrote: > Hi all, > > Would people appreciate enhancements to the prometheus reporter to include > extra labels via a configuration, as a contribution to Flink? I can see it > being useful for adding labels that are not job specific, but i