Skip to content
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

[CONNECTOR] add sink metrics #1269

Merged
merged 1 commit into from
Dec 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import org.astraea.common.metrics.BeanQuery;
import org.astraea.common.metrics.MBeanClient;

public class SourceMetrics {
public class ConnectorMetrics {

public static List<SourceTaskInfo> sourceTaskInfo(MBeanClient client) {
return client
Expand All @@ -37,7 +37,21 @@ public static List<SourceTaskInfo> sourceTaskInfo(MBeanClient client) {
.collect(Collectors.toUnmodifiableList());
}

public static List<SourceTaskError> sourceTaskError(MBeanClient client) {
public static List<SinkTaskInfo> sinkTaskInfo(MBeanClient client) {
return client
.queryBeans(
BeanQuery.builder()
.domainName("kafka.connect")
.property("type", "sink-task-metrics")
.property("connector", "*")
.property("task", "*")
.build())
.stream()
.map(b -> (SinkTaskInfo) () -> b)
.collect(Collectors.toUnmodifiableList());
}

public static List<TaskError> taskError(MBeanClient client) {
return client
.queryBeans(
BeanQuery.builder()
Expand All @@ -47,7 +61,7 @@ public static List<SourceTaskError> sourceTaskError(MBeanClient client) {
.property("task", "*")
.build())
.stream()
.map(b -> (SourceTaskError) () -> b)
.map(b -> (TaskError) () -> b)
.collect(Collectors.toUnmodifiableList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* 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.astraea.common.metrics.connector;

import org.astraea.common.metrics.HasBeanObject;

@FunctionalInterface
public interface SinkTaskInfo extends HasBeanObject {

default String connectorName() {
return beanObject().properties().get("connector");
}

default int taskId() {
return Integer.parseInt(beanObject().properties().get("task"));
}

default String taskType() {
return beanObject().properties().get("type");
}

default double offsetCommitCompletionRate() {
return (double) beanObject().attributes().get("offset-commit-completion-rate");
}

default double offsetCommitCompletionTotal() {
return (double) beanObject().attributes().get("offset-commit-completion-total");
}

default double offsetCommitSeqNo() {
return (double) beanObject().attributes().get("offset-commit-seq-no");
}

default double offsetCommitSkipRate() {
return (double) beanObject().attributes().get("offset-commit-skip-rate");
}

default double offsetCommitSkipTotal() {
return (double) beanObject().attributes().get("offset-commit-skip-total");
}

default double partitionCount() {
return (double) beanObject().attributes().get("partition-count");
}

default double putBatchAvgTimeMs() {
return (double) beanObject().attributes().get("put-batch-avg-time-ms");
}

default double putBatchMaxTimeMs() {
return (double) beanObject().attributes().get("put-batch-max-time-ms");
}

default double sinkRecordActiveCount() {
return (double) beanObject().attributes().get("sink-record-active-count");
}

default double sinkRecordActiveCountAvg() {
return (double) beanObject().attributes().get("sink-record-active-count-avg");
}

default double sinkRecordActiveCountMax() {
return (double) beanObject().attributes().get("sink-record-active-count-max");
}

default double sinkRecordReadRate() {
return (double) beanObject().attributes().get("sink-record-read-rate");
}

default double sinkRecordReadTotal() {
return (double) beanObject().attributes().get("sink-record-read-total");
}

default double sinkRecordSendRate() {
return (double) beanObject().attributes().get("sink-record-send-rate");
}

default double sinkRecordSendTotal() {
return (double) beanObject().attributes().get("sink-record-send-total");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,10 @@ default int taskId() {
return Integer.parseInt(beanObject().properties().get("task"));
}

default String taskType() {
return beanObject().properties().get("type");
}

default double pollBatchAvgTimeMs() {
return (double) beanObject().attributes().get("poll-batch-avg-time-ms");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
import org.astraea.common.metrics.HasBeanObject;

@FunctionalInterface
public interface SourceTaskError extends HasBeanObject {
public interface TaskError extends HasBeanObject {

default String connectorName() {
return beanObject().properties().get("connector");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,16 @@
*/
package org.astraea.connector.perf;

import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.astraea.common.Configuration;
import org.astraea.common.Utils;
import org.astraea.common.connector.ConnectorClient;
import org.astraea.common.consumer.Record;
import org.astraea.common.metrics.MBeanClient;
import org.astraea.common.metrics.connector.ConnectorMetrics;
import org.astraea.it.RequireSingleWorkerCluster;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -98,4 +101,67 @@ void testTask() {
// the frequency is 1s so the elapsed time of executing put should be greater than 500ms
Assertions.assertTrue(System.currentTimeMillis() - now > 500);
}

@Test
void testMetrics() {
var name = Utils.randomString();
var topicName = Utils.randomString();
var client = ConnectorClient.builder().url(workerUrl()).build();
client
.createConnector(
name,
Map.of(
ConnectorClient.CONNECTOR_CLASS_KEY,
PerfSink.class.getName(),
ConnectorClient.TASK_MAX_KEY,
"1",
ConnectorClient.TOPICS_KEY,
topicName))
.toCompletableFuture()
.join();
Utils.sleep(Duration.ofSeconds(3));

var m0 =
ConnectorMetrics.sinkTaskInfo(MBeanClient.local()).stream()
.filter(m -> m.connectorName().equals(name))
.collect(Collectors.toList());
Assertions.assertNotEquals(0, m0.size());
m0.forEach(
m -> {
Assertions.assertNotNull(m.taskType());
// it is hard to check the commit metrics, so we check non-null
Assertions.assertDoesNotThrow(m::offsetCommitSeqNo);
Assertions.assertDoesNotThrow(m::offsetCommitCompletionRate);
Assertions.assertDoesNotThrow(m::offsetCommitCompletionTotal);
Assertions.assertDoesNotThrow(m::offsetCommitSkipTotal);
Assertions.assertDoesNotThrow(m::offsetCommitSkipRate);
Assertions.assertDoesNotThrow(m::partitionCount);
Assertions.assertDoesNotThrow(m::putBatchMaxTimeMs);
Assertions.assertDoesNotThrow(m::putBatchAvgTimeMs);
Assertions.assertDoesNotThrow(m::sinkRecordReadRate);
Assertions.assertDoesNotThrow(m::sinkRecordActiveCount);
Assertions.assertDoesNotThrow(m::sinkRecordActiveCountAvg);
Assertions.assertDoesNotThrow(m::sinkRecordSendRate);
Assertions.assertDoesNotThrow(m::sinkRecordReadTotal);
Assertions.assertDoesNotThrow(m::sinkRecordActiveCountMax);
Assertions.assertDoesNotThrow(m::sinkRecordSendTotal);
});

var m1 =
ConnectorMetrics.taskError(MBeanClient.local()).stream()
.filter(m -> m.connectorName().equals(name))
.collect(Collectors.toList());
Assertions.assertNotEquals(0, m1.size());
m1.forEach(
m -> {
Assertions.assertEquals(0, m.lastErrorTimestamp());
Assertions.assertEquals(0D, m.deadletterqueueProduceFailures());
Assertions.assertEquals(0D, m.deadletterqueueProduceRequests());
Assertions.assertEquals(0D, m.totalErrorsLogged());
Assertions.assertEquals(0D, m.totalRecordErrors());
Assertions.assertEquals(0D, m.totalRetries());
Assertions.assertEquals(0D, m.totalRecordFailures());
Assertions.assertEquals(0D, m.totalRecordsSkipped());
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.astraea.common.admin.Replica;
import org.astraea.common.connector.ConnectorClient;
import org.astraea.common.metrics.MBeanClient;
import org.astraea.common.metrics.connector.SourceMetrics;
import org.astraea.common.metrics.connector.ConnectorMetrics;
import org.astraea.it.RequireSingleWorkerCluster;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -211,12 +211,13 @@ void testMetrics() {
Utils.sleep(Duration.ofSeconds(3));

var m0 =
SourceMetrics.sourceTaskInfo(MBeanClient.local()).stream()
ConnectorMetrics.sourceTaskInfo(MBeanClient.local()).stream()
.filter(m -> m.connectorName().equals(name))
.collect(Collectors.toList());
Assertions.assertNotEquals(0, m0.size());
m0.forEach(
m -> {
Assertions.assertNotNull(m.taskType());
Assertions.assertNotEquals(0D, m.pollBatchAvgTimeMs());
Assertions.assertNotEquals(0D, m.pollBatchMaxTimeMs());
Assertions.assertNotEquals(0D, m.sourceRecordActiveCountMax());
Expand All @@ -229,7 +230,7 @@ void testMetrics() {
});

var m1 =
SourceMetrics.sourceTaskError(MBeanClient.local()).stream()
ConnectorMetrics.taskError(MBeanClient.local()).stream()
.filter(m -> m.connectorName().equals(name))
.collect(Collectors.toList());
Assertions.assertNotEquals(0, m1.size());
Expand Down
29 changes: 14 additions & 15 deletions gui/src/main/java/org/astraea/gui/Context.java
Original file line number Diff line number Diff line change
Expand Up @@ -72,16 +72,24 @@ public void workerJmxPort(int workerJmxPort) {
workerClients = new Clients<>(workerJmxPort);
}

@SuppressWarnings("resource")
public Map<Integer, MBeanClient> addBrokerClients(List<NodeInfo> nodeInfos) {
if (brokerClients == null) return Map.of();
nodeInfos.forEach(n -> brokerClients.add(n.id(), n.host()));
return brokerClients.clients();
nodeInfos.forEach(
n ->
brokerClients.clients.computeIfAbsent(
n.id(), ignored -> MBeanClient.jndi(n.host(), brokerClients.jmxPort)));
return Map.copyOf(brokerClients.clients);
}

@SuppressWarnings("resource")
public Map<String, MBeanClient> addWorkerClients(Set<String> hostnames) {
if (workerClients == null) return Map.of();
hostnames.forEach(n -> workerClients.add(n, n));
return workerClients.clients();
hostnames.forEach(
n ->
workerClients.clients.computeIfAbsent(
n, ignored -> MBeanClient.jndi(n, workerClients.jmxPort)));
return Map.copyOf(workerClients.clients);
}

public Admin admin() {
Expand All @@ -98,12 +106,12 @@ public ConnectorClient connectorClient() {

public Map<Integer, MBeanClient> brokerClients() {
if (brokerClients == null) return Map.of();
return brokerClients.clients();
return Map.copyOf(brokerClients.clients);
}

public Map<String, MBeanClient> workerClients() {
if (workerClients == null) return Map.of();
return workerClients.clients();
return Map.copyOf(workerClients.clients);
}

private static class Clients<T extends Comparable<T>> {
Expand All @@ -113,14 +121,5 @@ private static class Clients<T extends Comparable<T>> {
Clients(int jmxPort) {
this.jmxPort = jmxPort;
}

void add(T identity, String hostname) {
var previous = clients.put(identity, MBeanClient.jndi(hostname, jmxPort));
if (previous != null) previous.close();
}

Map<T, MBeanClient> clients() {
return Map.copyOf(clients);
}
}
}
Loading