-
Notifications
You must be signed in to change notification settings - Fork 57
[monitoring] Impl metrics #631
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
6fc7e1a
e3e6b72
8a8f2ed
db3326b
ac017c0
1521e64
69d2b66
b9e11f1
ead37d6
9539a50
8dbfabd
3b8bd42
e8be6d2
6faac11
9d19bdf
cc30055
f3993bf
f99f1b5
a440ace
7568a41
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -7,11 +7,11 @@ | |
| import com.clickhouse.kafka.connect.sink.dlq.ErrorReporter; | ||
| import com.clickhouse.kafka.connect.sink.kafka.RangeContainer; | ||
| import com.clickhouse.kafka.connect.sink.kafka.RangeState; | ||
| import com.clickhouse.kafka.connect.sink.state.State; | ||
| import com.clickhouse.kafka.connect.sink.state.StateProvider; | ||
| import com.clickhouse.kafka.connect.sink.state.StateRecord; | ||
| import com.clickhouse.kafka.connect.util.QueryIdentifier; | ||
| import com.clickhouse.kafka.connect.util.Utils; | ||
| import com.clickhouse.kafka.connect.util.jmx.SinkTaskStatistics; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -28,15 +28,17 @@ public class Processing { | |
| private StateProvider stateProvider = null; | ||
| private DBWriter dbWriter = null; | ||
| private ClickHouseSinkConfig clickHouseSinkConfig; | ||
|
|
||
| private SinkTaskStatistics statistics; | ||
|
|
||
| private ErrorReporter errorReporter = null; | ||
|
|
||
| public Processing(StateProvider stateProvider, DBWriter dbWriter, ErrorReporter errorReporter, ClickHouseSinkConfig clickHouseSinkConfig) { | ||
| public Processing(StateProvider stateProvider, DBWriter dbWriter, ErrorReporter errorReporter, | ||
| ClickHouseSinkConfig clickHouseSinkConfig, SinkTaskStatistics statistics) { | ||
| this.stateProvider = stateProvider; | ||
| this.dbWriter = dbWriter; | ||
| this.errorReporter = errorReporter; | ||
| this.clickHouseSinkConfig = clickHouseSinkConfig; | ||
| this.statistics = statistics; | ||
| } | ||
| /** | ||
| * the logic is only for topic partition scoop | ||
|
|
@@ -49,14 +51,23 @@ private void doInsert(List<Record> records, RangeContainer rangeContainer) { | |
| LOGGER.trace("doInsert - No records to insert."); | ||
| return; | ||
| } | ||
| QueryIdentifier queryId = new QueryIdentifier(records.get(0).getRecordOffsetContainer().getTopic(), records.get(0).getRecordOffsetContainer().getPartition(), | ||
|
|
||
| final Record firstRecord = records.get(0); | ||
| long eventReceiveLag = 0; | ||
| if (firstRecord.getSinkRecord().timestamp() != null) { | ||
| eventReceiveLag = System.currentTimeMillis() - firstRecord.getSinkRecord().timestamp(); | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we know where the timestamp comes from? Record metadata, or when created, just want to validate since we can have clock skew in some siuations
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will find out. |
||
| } | ||
| final String topic = firstRecord.getRecordOffsetContainer().getTopic(); | ||
| QueryIdentifier queryId = new QueryIdentifier(topic, records.get(0).getRecordOffsetContainer().getPartition(), | ||
| rangeContainer.getMinOffset(), rangeContainer.getMaxOffset(), | ||
| UUID.randomUUID().toString()); | ||
|
|
||
| try { | ||
| LOGGER.debug("doInsert - Records: [{}] - {}", records.size(), queryId); | ||
| dbWriter.doInsert(records, queryId, errorReporter); | ||
| statistics.recordTopicStats(records.size(), queryId.getTopic(), eventReceiveLag); | ||
| } catch (Exception e) { | ||
| statistics.recordTopicStatsOnFailure(records.size(), queryId.getTopic(), eventReceiveLag); | ||
| throw new RuntimeException(queryId.toString(), e);//This way the queryId will propagate | ||
| } | ||
| } | ||
|
|
@@ -66,12 +77,21 @@ private void doInsert(List<Record> records) { | |
| LOGGER.trace("doInsert - No records to insert."); | ||
| return; | ||
| } | ||
| QueryIdentifier queryId = new QueryIdentifier(records.get(0).getRecordOffsetContainer().getTopic(), UUID.randomUUID().toString()); | ||
|
|
||
| final Record firstRecord = records.get(0); | ||
| long eventReceiveLag = 0; | ||
| if (firstRecord.getSinkRecord().timestamp() != null) { | ||
| eventReceiveLag = System.currentTimeMillis() - firstRecord.getSinkRecord().timestamp(); | ||
| } | ||
| final String topic = firstRecord.getRecordOffsetContainer().getTopic(); | ||
| QueryIdentifier queryId = new QueryIdentifier(topic, UUID.randomUUID().toString()); | ||
|
|
||
| try { | ||
| LOGGER.info("doInsert - Records: [{}] - {}", records.size(), queryId); | ||
| dbWriter.doInsert(records, queryId, errorReporter); | ||
| statistics.recordTopicStats(records.size(), topic, eventReceiveLag); | ||
| } catch (Exception e) { | ||
| statistics.recordTopicStatsOnFailure(records.size(), topic, eventReceiveLag); | ||
| throw new RuntimeException(queryId.toString(), e);//This way the queryId will propagate | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,34 +1,106 @@ | ||
| package com.clickhouse.kafka.connect.util.jmx; | ||
|
|
||
| import com.clickhouse.kafka.connect.sink.Version; | ||
|
|
||
| import java.util.Deque; | ||
| import java.util.Map; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentLinkedDeque; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
|
|
||
| public class SinkTaskStatistics implements SinkTaskStatisticsMBean { | ||
| private volatile long receivedRecords; | ||
| private volatile long recordProcessingTime; | ||
| private volatile long taskProcessingTime; | ||
| private final AtomicLong receivedRecords; | ||
| private final AtomicLong recordProcessingTime; | ||
| private final AtomicLong taskProcessingTime; | ||
| private final AtomicLong insertedRecords; | ||
| private final Map<String, TopicStatistics> topicStatistics; | ||
| private final int taskId; | ||
|
|
||
| private final Deque<String> topicMBeans; | ||
|
|
||
| public SinkTaskStatistics(int taskId) { | ||
| this.taskId = taskId; | ||
| this.topicMBeans = new ConcurrentLinkedDeque<>(); | ||
| this.receivedRecords = new AtomicLong(0); | ||
| this.recordProcessingTime = new AtomicLong(0); | ||
| this.taskProcessingTime = new AtomicLong(0); | ||
| this.insertedRecords = new AtomicLong(0); | ||
| this.topicStatistics = new ConcurrentHashMap<>(); | ||
| } | ||
|
|
||
| public void registerMBean() { | ||
| MBeanServerUtils.registerMBean(this, getMBeanNAme(taskId)); | ||
| } | ||
|
|
||
| public void unregisterMBean() { | ||
| MBeanServerUtils.unregisterMBean(getMBeanNAme(taskId)); | ||
| for (String topicMBean : topicMBeans) { | ||
| MBeanServerUtils.unregisterMBean(topicMBean); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public long getReceivedRecords() { | ||
| return receivedRecords; | ||
| return receivedRecords.get(); | ||
| } | ||
|
|
||
| @Override | ||
| public long getRecordProcessingTime() { | ||
| return recordProcessingTime; | ||
| return recordProcessingTime.get(); | ||
| } | ||
|
|
||
| @Override | ||
| public long getTaskProcessingTime() { | ||
| return taskProcessingTime; | ||
| return taskProcessingTime.get(); | ||
| } | ||
|
|
||
| @Override | ||
| public long getInsertedRecords() { | ||
| return insertedRecords.get(); | ||
| } | ||
|
|
||
| public void receivedRecords(final int n ) { | ||
| this.receivedRecords += n; | ||
| public void receivedRecords(final int n) { | ||
| this.receivedRecords.addAndGet(n); | ||
| } | ||
|
|
||
| public void recordProcessingTime(ExecutionTimer timer) { | ||
| this.recordProcessingTime += timer.nanosElapsed(); | ||
| this.recordProcessingTime.addAndGet(timer.nanosElapsed()); | ||
| } | ||
|
|
||
| public void taskProcessingTime(ExecutionTimer timer) { | ||
| this.taskProcessingTime += timer.nanosElapsed(); | ||
| this.taskProcessingTime.addAndGet(timer.nanosElapsed()); | ||
| } | ||
|
|
||
| public void recordTopicStats(int n, String table, long eventReceiveLag) { | ||
chernser marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| insertedRecords.addAndGet(n); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).recordsInserted(n); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).batchInserted(1); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).eventReceiveLag(eventReceiveLag); | ||
| } | ||
|
|
||
| public void recordTopicStatsOnFailure(int n, String table, long eventReceiveLag) { | ||
| insertedRecords.addAndGet(n); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).recordsFailed(n); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).batchesFailed(1); | ||
| topicStatistics.computeIfAbsent(table, this::createTopicStatistics).eventReceiveLag(eventReceiveLag); | ||
| } | ||
|
|
||
| private TopicStatistics createTopicStatistics(String topic) { | ||
| TopicStatistics topicStatistics = new TopicStatistics(); | ||
| String topicMBeanName = getTopicMBeanName(taskId, topic); | ||
| topicMBeans.add(topicMBeanName); | ||
| return MBeanServerUtils.registerMBean(topicStatistics, topicMBeanName); | ||
| } | ||
|
|
||
| public static String getMBeanNAme(int taskId) { | ||
chernser marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| return String.format("com.clickhouse:type=ClickHouseKafkaConnector,name=SinkTask%d,version=%s", taskId, Version.ARTIFACT_VERSION); | ||
| } | ||
|
|
||
| public static String getTopicMBeanName(int taskId, String topic) { | ||
| return String.format("com.clickhouse:type=ClickHouseKafkaConnector,name=SinkTask%d,version=%s,topic=%s", taskId, Version.ARTIFACT_VERSION, topic); | ||
| } | ||
|
|
||
| public void insertTime(long t, String topic) { | ||
| topicStatistics.computeIfAbsent(topic, this::createTopicStatistics).insertTime(t); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.