[FLINK-35245][cdc-connector][tidb] Add metrics for flink-connector-tidb-cdc

pull/2968/head^2
Xie Yi 9 months ago committed by GitHub
parent b6cfbccd36
commit fa6e7ea512
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

@ -23,9 +23,11 @@ import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.cdc.connectors.tidb.metrics.TiDBSourceMetrics;
import org.apache.flink.cdc.connectors.tidb.table.StartupMode; import org.apache.flink.cdc.connectors.tidb.table.StartupMode;
import org.apache.flink.cdc.connectors.tidb.table.utils.TableKeyRangeUtils; import org.apache.flink.cdc.connectors.tidb.table.utils.TableKeyRangeUtils;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
@ -42,6 +44,7 @@ import org.tikv.common.TiConfiguration;
import org.tikv.common.TiSession; import org.tikv.common.TiSession;
import org.tikv.common.key.RowKey; import org.tikv.common.key.RowKey;
import org.tikv.common.meta.TiTableInfo; import org.tikv.common.meta.TiTableInfo;
import org.tikv.common.meta.TiTimestamp;
import org.tikv.kvproto.Cdcpb; import org.tikv.kvproto.Cdcpb;
import org.tikv.kvproto.Coprocessor; import org.tikv.kvproto.Coprocessor;
import org.tikv.kvproto.Kvrpcpb; import org.tikv.kvproto.Kvrpcpb;
@ -91,6 +94,7 @@ public class TiKVRichParallelSourceFunction<T> extends RichParallelSourceFunctio
private transient boolean running = true; private transient boolean running = true;
private transient ExecutorService executorService; private transient ExecutorService executorService;
private transient TiDBSourceMetrics sourceMetrics;
/** offset state. */ /** offset state. */
private transient ListState<Long> offsetState; private transient ListState<Long> offsetState;
@ -146,6 +150,9 @@ public class TiKVRichParallelSourceFunction<T> extends RichParallelSourceFunctio
+ getRuntimeContext().getIndexOfThisSubtask()) + getRuntimeContext().getIndexOfThisSubtask())
.build(); .build();
executorService = Executors.newSingleThreadExecutor(threadFactory); executorService = Executors.newSingleThreadExecutor(threadFactory);
final MetricGroup metricGroup = getRuntimeContext().getMetricGroup();
sourceMetrics = new TiDBSourceMetrics(metricGroup);
sourceMetrics.registerMetrics();
} }
@Override @Override
@ -210,6 +217,7 @@ public class TiKVRichParallelSourceFunction<T> extends RichParallelSourceFunctio
for (final Kvrpcpb.KvPair pair : segment) { for (final Kvrpcpb.KvPair pair : segment) {
if (TableKeyRangeUtils.isRecordKey(pair.getKey().toByteArray())) { if (TableKeyRangeUtils.isRecordKey(pair.getKey().toByteArray())) {
snapshotEventDeserializationSchema.deserialize(pair, outputCollector); snapshotEventDeserializationSchema.deserialize(pair, outputCollector);
reportMetrics(0L, startTs);
} }
} }
@ -231,6 +239,8 @@ public class TiKVRichParallelSourceFunction<T> extends RichParallelSourceFunctio
Cdcpb.Event.Row committedRow = committedEvents.take(); Cdcpb.Event.Row committedRow = committedEvents.take();
changeEventDeserializationSchema.deserialize( changeEventDeserializationSchema.deserialize(
committedRow, outputCollector); committedRow, outputCollector);
// use startTs of row as messageTs, use commitTs of row as fetchTs
reportMetrics(committedRow.getStartTs(), committedRow.getCommitTs());
} catch (Exception e) { } catch (Exception e) {
e.printStackTrace(); e.printStackTrace();
} }
@ -390,4 +400,20 @@ public class TiKVRichParallelSourceFunction<T> extends RichParallelSourceFunctio
// do nothing // do nothing
} }
} }
private void reportMetrics(long messageTs, long fetchTs) {
long now = System.currentTimeMillis();
// record the latest process time
sourceMetrics.recordProcessTime(now);
long messageTimestamp = TiTimestamp.extractPhysical(messageTs);
long fetchTimestamp = TiTimestamp.extractPhysical(fetchTs);
if (messageTimestamp > 0L) {
// report fetch delay
if (fetchTimestamp >= messageTimestamp) {
sourceMetrics.recordFetchDelay(fetchTimestamp - messageTimestamp);
}
// report emit delay
sourceMetrics.recordEmitDelay(now - messageTimestamp);
}
}
} }

@ -0,0 +1,90 @@
/*
* 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.flink.cdc.connectors.tidb.metrics;
import org.apache.flink.cdc.connectors.tidb.TiKVRichParallelSourceFunction;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.MetricGroup;
import static org.apache.flink.runtime.metrics.MetricNames.CURRENT_EMIT_EVENT_TIME_LAG;
import static org.apache.flink.runtime.metrics.MetricNames.CURRENT_FETCH_EVENT_TIME_LAG;
import static org.apache.flink.runtime.metrics.MetricNames.SOURCE_IDLE_TIME;
/** A collection class for handling metrics in {@link TiKVRichParallelSourceFunction}. */
public class TiDBSourceMetrics {
private final MetricGroup metricGroup;
/**
* The last record processing time, which is updated after {@link
* TiKVRichParallelSourceFunction} fetches a batch of data. It's mainly used to report metrics
* sourceIdleTime for sourceIdleTime = System.currentTimeMillis() - processTime.
*/
private long processTime = 0L;
/**
* currentFetchEventTimeLag = FetchTime - messageTimestamp, where the FetchTime is the time the
* record fetched into the source operator.
*/
private long fetchDelay = 0L;
/**
* currentEmitEventTimeLag = EmitTime - messageTimestamp, where the EmitTime is the time the
* record leaves the source operator.
*/
private long emitDelay = 0L;
public TiDBSourceMetrics(MetricGroup metricGroup) {
this.metricGroup = metricGroup;
}
public void registerMetrics() {
metricGroup.gauge(CURRENT_FETCH_EVENT_TIME_LAG, (Gauge<Long>) this::getFetchDelay);
metricGroup.gauge(CURRENT_EMIT_EVENT_TIME_LAG, (Gauge<Long>) this::getEmitDelay);
metricGroup.gauge(SOURCE_IDLE_TIME, (Gauge<Long>) this::getIdleTime);
}
public long getFetchDelay() {
return fetchDelay;
}
public long getEmitDelay() {
return emitDelay;
}
public long getIdleTime() {
// no previous process time at the beginning, return 0 as idle time
if (processTime == 0) {
return 0;
}
return System.currentTimeMillis() - processTime;
}
public void recordProcessTime(long processTime) {
this.processTime = processTime;
}
public void recordFetchDelay(long fetchDelay) {
this.fetchDelay = fetchDelay;
}
public void recordEmitDelay(long emitDelay) {
this.emitDelay = emitDelay;
}
}

@ -0,0 +1,62 @@
/*
* 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.flink.cdc.connectors.tidb.metrics;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.testutils.MetricListener;
import org.junit.Before;
import org.junit.Test;
import java.util.Optional;
import static org.apache.flink.runtime.metrics.MetricNames.CURRENT_EMIT_EVENT_TIME_LAG;
import static org.apache.flink.runtime.metrics.MetricNames.CURRENT_FETCH_EVENT_TIME_LAG;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/** Unit test for {@link TiDBSourceMetrics}. */
public class TiDBSourceMetricsTest {
private MetricListener metricListener;
private TiDBSourceMetrics sourceMetrics;
@Before
public void setUp() {
metricListener = new MetricListener();
sourceMetrics = new TiDBSourceMetrics(metricListener.getMetricGroup());
sourceMetrics.registerMetrics();
}
@Test
public void testFetchEventTimeLagTracking() {
sourceMetrics.recordFetchDelay(5L);
assertGauge(metricListener, CURRENT_FETCH_EVENT_TIME_LAG, 5L);
}
@Test
public void testEmitEventTimeLagTracking() {
sourceMetrics.recordEmitDelay(3L);
assertGauge(metricListener, CURRENT_EMIT_EVENT_TIME_LAG, 3L);
}
private void assertGauge(MetricListener metricListener, String identifier, long expected) {
Optional<Gauge<Object>> gauge = metricListener.getGauge(identifier);
assertTrue(gauge.isPresent());
assertEquals(expected, (long) gauge.get().getValue());
}
}
Loading…
Cancel
Save