Skip to content

Commit

Permalink
feat(cli): print stats/errors breakdown by host
Browse files Browse the repository at this point in the history
  • Loading branch information
swist committed Mar 26, 2021
1 parent 044a7ef commit 30aeee3
Show file tree
Hide file tree
Showing 11 changed files with 495 additions and 70 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,17 @@

package io.confluent.ksql.cli.console;

import static com.google.common.collect.ImmutableListMultimap.flatteningToImmutableListMultimap;
import static com.google.common.collect.ImmutableListMultimap.toImmutableListMultimap;
import static io.confluent.ksql.util.CmdLineUtil.splitByUnquotedWhitespace;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Functions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Streams;
import io.confluent.ksql.cli.console.CliConfig.OnOff;
import io.confluent.ksql.cli.console.KsqlTerminal.HistoryEntry;
import io.confluent.ksql.cli.console.KsqlTerminal.StatusClosable;
Expand All @@ -44,7 +49,7 @@
import io.confluent.ksql.cli.console.table.builder.TopicDescriptionTableBuilder;
import io.confluent.ksql.cli.console.table.builder.TypeListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.WarningEntityTableBuilder;
import io.confluent.ksql.metrics.MetricCollectors;
import io.confluent.ksql.metrics.TopicSensors.Stat;
import io.confluent.ksql.model.WindowType;
import io.confluent.ksql.query.QueryError;
import io.confluent.ksql.rest.ApiJsonMapper;
Expand All @@ -65,6 +70,7 @@
import io.confluent.ksql.rest.entity.KafkaTopicsListExtended;
import io.confluent.ksql.rest.entity.KsqlEntity;
import io.confluent.ksql.rest.entity.KsqlErrorMessage;
import io.confluent.ksql.rest.entity.KsqlHostInfoEntity;
import io.confluent.ksql.rest.entity.KsqlStatementErrorMessage;
import io.confluent.ksql.rest.entity.KsqlWarning;
import io.confluent.ksql.rest.entity.PropertiesList;
Expand Down Expand Up @@ -106,6 +112,7 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
Expand All @@ -116,6 +123,7 @@
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.lang3.ObjectUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.common.config.ConfigException;
Expand Down Expand Up @@ -637,6 +645,80 @@ private void printQueryError(final QueryDescription query) {
}
}

private void printStatistics(final SourceDescription source) {
final ImmutableListMultimap<KsqlHostInfoEntity, Stat> clusterStats = source
.getClusterStatistics()
.entrySet()
.stream()
.collect(flatteningToImmutableListMultimap(
Entry::getKey,
(e) -> e.getValue().values().stream()
));
final ImmutableListMultimap<KsqlHostInfoEntity, Stat> clusterErrors = source
.getClusterErrorStats()
.entrySet()
.stream()
.collect(flatteningToImmutableListMultimap(
Entry::getKey,
(e) -> e.getValue().values().stream()
));

final ImmutableListMultimap<KsqlHostInfoEntity, Stat> statisticsMap = source.getStatisticsMap()
.values()
.stream()
.collect(toImmutableListMultimap(
Functions.constant(new KsqlHostInfoEntity("", 0)),
Functions.identity()
));
final ImmutableListMultimap<KsqlHostInfoEntity, Stat> errorStatsMap = source.getErrorStatsMap()
.values()
.stream()
.collect(toImmutableListMultimap(
Functions.constant(new KsqlHostInfoEntity("", 0)),
Functions.identity()
));


if (statisticsMap.isEmpty() && errorStatsMap.isEmpty()) {
writer().println(String.format(
"%n%-20s%n%s",
"Local runtime statistics",
"------------------------"
));
writer().println(source.getStatistics());
writer().println(source.getErrorStats());
return;
}
final boolean printLocalOnly = clusterStats.isEmpty() && clusterErrors.isEmpty();
final List<String> headers = ImmutableList.of("Host", "Metric", "Value", "Last Message");
final Stream<Entry<KsqlHostInfoEntity, Stat>> rows = printLocalOnly
? Streams.concat(statisticsMap.entries().stream(), errorStatsMap.entries().stream())
: Streams.concat(clusterStats.entries().stream(), clusterErrors.entries().stream());


writer().println(String.format(
"%n%-20s%n%s",
printLocalOnly ? "Local runtime statistics" : "Runtime statistics by host",
"-------------------------"
));
final Table statsTable = new Table.Builder()
.withColumnHeaders(headers)
.withRows(rows
.sorted(Comparator
.comparing((Map.Entry<KsqlHostInfoEntity, Stat> e) -> e.getKey().toString())
.thenComparing((Map.Entry<KsqlHostInfoEntity, Stat> e) -> e.getValue().name())
)
.map((e) -> {
final KsqlHostInfoEntity host = e.getKey();
final Stat metric = e.getValue();
final String hostCell = host.getHost().equals("") ? "--" : host.toString();
final String formattedValue = String.format("%10.0f", metric.getValue());
return ImmutableList.of(hostCell, metric.name(), formattedValue, metric.timestamp());
}))
.build();
statsTable.print(this);
}

private void printSourceDescription(final SourceDescription source) {
final boolean isTable = source.getType().equalsIgnoreCase("TABLE");

Expand All @@ -660,14 +742,8 @@ private void printSourceDescription(final SourceDescription source) {
printQueries(source.getReadQueries(), source.getType(), "read");

printQueries(source.getWriteQueries(), source.getType(), "write");
printStatistics(source);

writer().println(String.format(
"%n%-20s%n%s",
"Local runtime statistics",
"------------------------"
));
writer().println(MetricCollectors.format(source.getStatisticsMap().values(), "last-message"));
writer().println(MetricCollectors.format(source.getErrorStatsMap().values(), "last-message"));
writer().println(String.format(
"(%s)",
"Statistics of the local KSQL server interaction with the Kafka topic "
Expand Down Expand Up @@ -946,6 +1022,7 @@ private static final class CliCmdExecutor {
private final CliSpecificCommand cmd;
private final List<String> args;


private static CliCmdExecutor of(final CliSpecificCommand cmd, final List<String> lineParts) {
final String[] nameParts = cmd.getName().split("\\s+");
final List<String> argList = lineParts.subList(nameParts.length, lineParts.size()).stream()
Expand Down
Loading

0 comments on commit 30aeee3

Please sign in to comment.