-
Notifications
You must be signed in to change notification settings - Fork 1.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Feature][Core] Support record metrics in flink engine
- Loading branch information
1 parent
d41af8a
commit e98d4cd
Showing
15 changed files
with
767 additions
and
20 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
...ink-13/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkGroupCounter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.seatunnel.translation.flink.metric; | ||
|
||
import org.apache.seatunnel.api.common.metrics.Counter; | ||
import org.apache.seatunnel.api.common.metrics.Unit; | ||
|
||
public class FlinkGroupCounter implements Counter { | ||
|
||
private final String name; | ||
|
||
private final org.apache.flink.metrics.Counter counter; | ||
|
||
public FlinkGroupCounter(String name, org.apache.flink.metrics.Counter counter) { | ||
this.name = name; | ||
this.counter = counter; | ||
} | ||
|
||
@Override | ||
public void inc() { | ||
counter.inc(); | ||
} | ||
|
||
@Override | ||
public void inc(long n) { | ||
counter.inc(n); | ||
} | ||
|
||
@Override | ||
public void dec() { | ||
throw new UnsupportedOperationException("Flink metrics does not support dec operation"); | ||
} | ||
|
||
@Override | ||
public void dec(long n) { | ||
throw new UnsupportedOperationException("Flink metrics does not support dec operation"); | ||
} | ||
|
||
@Override | ||
public void set(long n) { | ||
throw new UnsupportedOperationException("Flink metrics does not support set operation"); | ||
} | ||
|
||
@Override | ||
public long getCount() { | ||
return counter.getCount(); | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return name; | ||
} | ||
|
||
@Override | ||
public Unit unit() { | ||
return Unit.COUNT; | ||
} | ||
} |
120 changes: 120 additions & 0 deletions
120
...nk-13/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkMetricContext.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,120 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.seatunnel.translation.flink.metric; | ||
|
||
import org.apache.seatunnel.api.common.metrics.Counter; | ||
import org.apache.seatunnel.api.common.metrics.Meter; | ||
import org.apache.seatunnel.api.common.metrics.Metric; | ||
import org.apache.seatunnel.api.common.metrics.MetricsContext; | ||
|
||
import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext; | ||
import org.apache.flink.metrics.MeterView; | ||
import org.apache.flink.metrics.MetricGroup; | ||
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.lang.reflect.Field; | ||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
|
||
public class FlinkMetricContext implements MetricsContext { | ||
|
||
private static final Logger LOGGER = LoggerFactory.getLogger(FlinkMetricContext.class); | ||
|
||
private final Map<String, Metric> metrics = new ConcurrentHashMap<>(); | ||
|
||
private MetricGroup metricGroup; | ||
|
||
private StreamingRuntimeContext runtimeContext; | ||
|
||
public FlinkMetricContext(MetricGroup metricGroup) { | ||
this.metricGroup = metricGroup; | ||
} | ||
|
||
public FlinkMetricContext(StreamingRuntimeContext runtimeContext) { | ||
this.runtimeContext = runtimeContext; | ||
} | ||
|
||
@Override | ||
public Counter counter(String name) { | ||
if (metrics.containsKey(name)) { | ||
return (Counter) metrics.get(name); | ||
} | ||
Counter counter = | ||
runtimeContext == null | ||
? new FlinkGroupCounter(name, metricGroup.counter(name)) | ||
: new FlinkCounter(name, runtimeContext.getLongCounter(name)); | ||
return this.counter(name, counter); | ||
} | ||
|
||
@Override | ||
public <C extends Counter> C counter(String name, C counter) { | ||
this.addMetric(name, counter); | ||
return counter; | ||
} | ||
|
||
@Override | ||
public Meter meter(String name) { | ||
if (metrics.containsKey(name)) { | ||
return (Meter) metrics.get(name); | ||
} | ||
|
||
// Why use reflection to obtain metrics group? | ||
// Because the value types returned by flink 1.13 and 1.14 runtimeContext.getMetricGroup() | ||
// are inconsistent | ||
org.apache.flink.metrics.Meter meter; | ||
if (runtimeContext == null) { | ||
meter = metricGroup.meter(name, new MeterView(5)); | ||
} else { | ||
try { | ||
Field field = AbstractRuntimeUDFContext.class.getDeclaredField("metrics"); | ||
field.setAccessible(true); | ||
MetricGroup mg = (MetricGroup) field.get(runtimeContext); | ||
meter = mg.meter(name, new MeterView(5)); | ||
} catch (Exception e) { | ||
throw new IllegalStateException("Initial meter failed", e); | ||
} | ||
} | ||
return this.meter(name, new FlinkMeter(name, meter)); | ||
} | ||
|
||
@Override | ||
public <M extends Meter> M meter(String name, M meter) { | ||
this.addMetric(name, meter); | ||
return meter; | ||
} | ||
|
||
protected void addMetric(String name, Metric metric) { | ||
if (metric == null) { | ||
LOGGER.warn("Ignoring attempted add of a metric due to being null for name {}.", name); | ||
} else { | ||
synchronized (this) { | ||
Metric prior = this.metrics.put(name, metric); | ||
if (prior != null) { | ||
this.metrics.put(name, prior); | ||
LOGGER.warn( | ||
"Name collision: MetricsContext already contains a Metric with the name '" | ||
+ name | ||
+ "'. Metric will not be reported."); | ||
} | ||
} | ||
} | ||
} | ||
} |
74 changes: 74 additions & 0 deletions
74
...-13/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterContext.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.seatunnel.translation.flink.sink; | ||
|
||
import org.apache.seatunnel.api.common.metrics.MetricsContext; | ||
import org.apache.seatunnel.api.sink.SinkWriter; | ||
import org.apache.seatunnel.translation.flink.metric.FlinkMetricContext; | ||
|
||
import org.apache.flink.api.connector.sink.Sink.InitContext; | ||
import org.apache.flink.metrics.MetricGroup; | ||
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.lang.reflect.Field; | ||
|
||
public class FlinkSinkWriterContext implements SinkWriter.Context { | ||
|
||
private static final Logger LOGGER = LoggerFactory.getLogger(FlinkMetricContext.class); | ||
|
||
private final InitContext writerContext; | ||
|
||
public FlinkSinkWriterContext(InitContext writerContext) { | ||
this.writerContext = writerContext; | ||
} | ||
|
||
@Override | ||
public int getIndexOfSubtask() { | ||
return writerContext.getSubtaskId(); | ||
} | ||
|
||
@Override | ||
public MetricsContext getMetricsContext() { | ||
// In flink 1.14, it has contained runtimeContext in InitContext, so first step to detect if | ||
// it is existed | ||
try { | ||
Field field = writerContext.getClass().getDeclaredField("runtimeContext"); | ||
field.setAccessible(true); | ||
StreamingRuntimeContext runtimeContext = | ||
(StreamingRuntimeContext) field.get(writerContext); | ||
return new FlinkMetricContext(runtimeContext); | ||
} catch (Exception e) { | ||
LOGGER.info( | ||
"Flink version is not 1.14.x, will initial MetricsContext using metricGroup"); | ||
} | ||
// Why use reflection to obtain metrics group? | ||
// Because the value types returned by flink 1.13 and 1.14 InitContext.getMetricGroup() | ||
// are inconsistent | ||
try { | ||
Field field = writerContext.getClass().getDeclaredField("metricGroup"); | ||
field.setAccessible(true); | ||
MetricGroup metricGroup = (MetricGroup) field.get(writerContext); | ||
return new FlinkMetricContext(metricGroup); | ||
} catch (Exception e) { | ||
throw new IllegalStateException("Initial sink metrics failed", e); | ||
} | ||
} | ||
} |
Oops, something went wrong.