DBZ-789 Expose more granular snapshot metrics via JMX

This commit is contained in:
Peter Goransson 2018-07-05 14:22:38 -04:00 committed by Gunnar Morling
parent 033db6659d
commit 448330fd07
3 changed files with 18 additions and 0 deletions

View File

@ -567,6 +567,7 @@ protected void execute() {
long stop = clock.currentTimeInMillis();
logger.info("Step {}: - {} of {} rows scanned from table '{}' after {}",
stepNum, rowNum, rowCountStr, tableId, Strings.duration(stop - start));
metrics.setRowsScanned(tableId.toString(), rowNum);
}
}
@ -575,6 +576,7 @@ protected void execute() {
long stop = clock.currentTimeInMillis();
logger.info("Step {}: - Completed scanning a total of {} rows from table '{}' after {}",
stepNum, rowNum, tableId, Strings.duration(stop - start));
metrics.setRowsScanned(tableId.toString(), rowNum);
}
} catch (InterruptedException e) {
Thread.interrupted();

View File

@ -5,6 +5,8 @@
*/
package io.debezium.connector.mysql;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@ -24,6 +26,8 @@ class SnapshotReaderMetrics extends Metrics implements SnapshotReaderMetricsMXBe
private final AtomicBoolean snapshotAborted = new AtomicBoolean();
private final AtomicLong startTime = new AtomicLong();
private final AtomicLong stopTime = new AtomicLong();
private final ConcurrentMap<String, Long> rowsScanned = new ConcurrentHashMap<String, Long>();
private final MySqlSchema schema;
private final Clock clock;
@ -118,4 +122,13 @@ public void abortSnapshot() {
public String[] getMonitoredTables() {
return schema.monitoredTablesAsStringArray();
}
public void setRowsScanned(String tableId, Long numRows) {
rowsScanned.put(tableId, numRows);
}
@Override
public ConcurrentMap<String, Long> getRowsScanned() {
return rowsScanned;
}
}

View File

@ -5,6 +5,8 @@
*/
package io.debezium.connector.mysql;
import java.util.Map;
/**
* @author Randall Hauch
*/
@ -17,4 +19,5 @@ public interface SnapshotReaderMetricsMXBean extends ReaderMetricsMXBean {
boolean getSnapshotAborted();
boolean getSnapshotCompleted();
long getSnapshotDurationInSeconds();
Map<String, Long> getRowsScanned();
}