Skip to content

Commit

Permalink
[Feature][Core] Support record metrics in flink engine
Browse files Browse the repository at this point in the history
  • Loading branch information
TyrantLucifer committed Dec 20, 2023
1 parent d41af8a commit e98d4cd
Show file tree
Hide file tree
Showing 15 changed files with 767 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@

package org.apache.seatunnel.common.utils;

import java.time.Instant;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
import java.util.HashMap;
import java.util.Map;
Expand Down Expand Up @@ -49,10 +51,24 @@ public static LocalDateTime parse(String dateTime, Formatter formatter) {
return LocalDateTime.parse(dateTime, FORMATTER_MAP.get(formatter));
}

public static LocalDateTime parse(long timestamp) {
return parse(timestamp, ZoneId.systemDefault());
}

public static LocalDateTime parse(long timestamp, ZoneId zoneId) {
Instant instant = Instant.ofEpochMilli(timestamp);
return LocalDateTime.ofInstant(instant, zoneId);
}

public static String toString(LocalDateTime dateTime, Formatter formatter) {
return dateTime.format(FORMATTER_MAP.get(formatter));
}

public static String toString(long timestamp, Formatter formatter) {
Instant instant = Instant.ofEpochMilli(timestamp);
return toString(LocalDateTime.ofInstant(instant, ZoneId.systemDefault()), formatter);
}

public enum Formatter {
YYYY_MM_DD_HH_MM_SS("yyyy-MM-dd HH:mm:ss"),
YYYY_MM_DD_HH_MM_SS_SPOT("yyyy.MM.dd HH:mm:ss"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,13 @@
import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment;
import org.apache.seatunnel.core.starter.execution.TaskExecution;
import org.apache.seatunnel.core.starter.flink.FlinkStarter;
import org.apache.seatunnel.translation.flink.metric.FlinkJobMetricsSummary;

import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.RuntimeExecutionMode;

import lombok.extern.slf4j.Slf4j;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.net.MalformedURLException;
Expand All @@ -51,8 +54,10 @@
import java.util.stream.Stream;

/** Used to execute a SeaTunnelTask. */
@Slf4j
public class FlinkExecution implements TaskExecution {

private static final Logger LOGGER = LoggerFactory.getLogger(FlinkExecution.class);

private final FlinkRuntimeEnvironment flinkRuntimeEnvironment;
private final PluginExecuteProcessor<DataStreamTableInfo, FlinkRuntimeEnvironment>
sourcePluginExecuteProcessor;
Expand Down Expand Up @@ -109,20 +114,32 @@ public void execute() throws TaskExecuteException {
dataStreams = sourcePluginExecuteProcessor.execute(dataStreams);
dataStreams = transformPluginExecuteProcessor.execute(dataStreams);
sinkPluginExecuteProcessor.execute(dataStreams);
log.info(
LOGGER.info(
"Flink Execution Plan: {}",
flinkRuntimeEnvironment.getStreamExecutionEnvironment().getExecutionPlan());
log.info("Flink job name: {}", flinkRuntimeEnvironment.getJobName());
LOGGER.info("Flink job name: {}", flinkRuntimeEnvironment.getJobName());
if (!flinkRuntimeEnvironment.isStreaming()) {
flinkRuntimeEnvironment
.getStreamExecutionEnvironment()
.setRuntimeMode(RuntimeExecutionMode.BATCH);
log.info("Flink job Mode: {}", JobMode.BATCH);
LOGGER.info("Flink job Mode: {}", JobMode.BATCH);
}
try {
flinkRuntimeEnvironment
.getStreamExecutionEnvironment()
.execute(flinkRuntimeEnvironment.getJobName());
final long jobStartTime = System.currentTimeMillis();
JobExecutionResult jobResult =
flinkRuntimeEnvironment
.getStreamExecutionEnvironment()
.execute(flinkRuntimeEnvironment.getJobName());
final long jobEndTime = System.currentTimeMillis();

final FlinkJobMetricsSummary jobMetricsSummary =
FlinkJobMetricsSummary.builder()
.jobExecutionResult(jobResult)
.jobStartTime(jobStartTime)
.jobEndTime(jobEndTime)
.build();

LOGGER.info("Job finished, execution result: \n{}", jobMetricsSummary);
} catch (Exception e) {
throw new TaskExecuteException("Execute Flink job error", e);
}
Expand Down Expand Up @@ -170,9 +187,9 @@ private Config injectJarsToConfig(Config config, String path, List<URL> jars) {
for (URL jarUrl : jars) {
if (new File(jarUrl.getFile()).exists()) {
validJars.add(jarUrl);
log.info("Inject jar to config: {}", jarUrl);
LOGGER.info("Inject jar to config: {}", jarUrl);
} else {
log.warn("Remove invalid jar when inject jars into config: {}", jarUrl);
LOGGER.warn("Remove invalid jar when inject jars into config: {}", jarUrl);
}
}

Expand Down
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;
}
}
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.");
}
}
}
}
}
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);
}
}
}
Loading

0 comments on commit e98d4cd

Please sign in to comment.