arjunashok commented on code in PR #249:
URL: https://github.com/apache/cassandra-sidecar/pull/249#discussion_r2291779233
##########
adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraMetricsOperations.java:
##########
@@ -250,4 +296,181 @@ private List<ClientConnectionEntry>
statsToEntries(Stream<ConnectedClientStats>
stat.authenticationMode,
stat.authenticationMetadata);
}
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public CompactionStatsData compactionStats()
+ {
+ // Get compaction manager and storage service proxies
+ CompactionManagerJmxOperations compactionManager =
jmxClient.proxy(CompactionManagerJmxOperations.class,
COMPACTION_MANAGER_OBJ_NAME);
+ StorageJmxOperations storageService =
jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+ // Get concurrent compactors from StorageService as per specification
+ long concurrentCompactors = storageService.getConcurrentCompactors();
+
+ // Get pending tasks grouped by keyspace and table
+ Map<String, Map<String, Integer>> pendingTasks =
getPendingCompactionTasksByTable();
+ long totalPendingTasks = pendingTasks.values().stream()
+ .mapToLong(tableMap ->
tableMap.values().stream().mapToInt(Integer::intValue).sum())
+ .sum();
+
+ // Get compaction metrics from JMX counters and meters
+ long completedCompactionsCount =
getValueAsLong(getCompactionMetric(CompactionStatsMetrics.TOTAL_COMPACTIONS_COMPLETED));
+ long dataCompactedBytes =
getValueAsLong(getCompactionMetric(CompactionStatsMetrics.BYTES_COMPACTED));
+ long abortedCompactionsCount =
getValueAsLong(getCompactionMetric(CompactionStatsMetrics.COMPACTIONS_ABORTED));
+ long reducedCompactionsCount =
getValueAsLong(getCompactionMetric(CompactionStatsMetrics.COMPACTIONS_REDUCED));
+ long sstablesDroppedFromCompactionCount =
getValueAsLong(getCompactionMetric(CompactionStatsMetrics.SSTABLES_DROPPED_FROM_COMPACTION));
+
+ // Get completed compactions rate with proper time conversions
+ CompletedCompactionsRateData completedCompactionsRate =
getCompletedCompactionsRate();
+
+ // Get active compactions with all required fields
+ List<ActiveCompactionEntryData> activeCompactions =
getActiveCompactions(compactionManager.getCompactions());
+ long activeCompactionsCount = activeCompactions.size();
+
+ // Calculate remaining time in seconds based on throughput and
remaining bytes
+ long activeCompactionsRemainingTime =
calculateRemainingTimeSeconds(activeCompactions, storageService);
+
+ return CompactionStatsData.builder()
+ .concurrentCompactors(concurrentCompactors)
+ .pendingTasks(pendingTasks)
+ .totalPendingTasks(totalPendingTasks)
+ .completedCompactions(completedCompactionsCount)
+ .dataCompacted(dataCompactedBytes)
+ .abortedCompactions(abortedCompactionsCount)
+ .reducedCompactions(reducedCompactionsCount)
+ .sstablesDroppedFromCompaction(sstablesDroppedFromCompactionCount)
+ .completedCompactionsRate(completedCompactionsRate)
+ .activeCompactions(activeCompactions)
+ .activeCompactionsCount(activeCompactionsCount)
+ .activeCompactionsRemainingTime(activeCompactionsRemainingTime)
+ .build();
+ }
+
+ private Map<String, Map<String, Integer>>
getPendingCompactionTasksByTable()
+ {
+ // Get pending tasks by table name from Gauge metric
+ Object value =
getCompactionMetric(CompactionStatsMetrics.PENDING_TASKS_BY_TABLE_NAME);
+ return parsePendingTasksMap(value);
+ }
+
+ private Map<String, Map<String, Integer>> parsePendingTasksMap(Object
value)
+ {
+ Map<?, ?> rawMap = safeCast(value, Map.class, "pending tasks");
+ Map<String, Map<String, Integer>> result = new HashMap<>();
+
+ for (Map.Entry<?, ?> entry : rawMap.entrySet())
+ {
+ String keyspace = safeCast(entry.getKey(), String.class, "keyspace
name");
+ Map<?, ?> rawTableMap = safeCast(entry.getValue(), Map.class,
"table data");
+ Map<String, Integer> tableMap = new HashMap<>();
+
+ for (Map.Entry<?, ?> tableEntry : rawTableMap.entrySet())
+ {
+ String tableName = safeCast(tableEntry.getKey(), String.class,
"table name");
+ Number taskCount = safeCast(tableEntry.getValue(),
Number.class, "task count");
+ tableMap.put(tableName, taskCount.intValue());
+ }
+
+ result.put(keyspace, tableMap);
+ }
+
+ return result;
+ }
+
+ private Object getCompactionMetric(final CompactionStatsMetrics metric)
+ {
+ String metricObjectType = String.format(METRICS_OBJ_TYPE_COMPACTION,
metric.metricName());
+ return queryMetric(metricObjectType, metric.type);
+ }
+
+ private CompletedCompactionsRateData getCompletedCompactionsRate()
+ {
+ // Get rates from meter metric for TotalCompactionsCompleted
+ String metricObjectType = String.format(METRICS_OBJ_TYPE_COMPACTION,
"TotalCompactionsCompleted");
+ MeterMetricsJmxOperations metricsProxy =
jmxClient.proxy(MeterMetricsJmxOperations.class, metricObjectType);
+
+ // Convert per-second rates to the specification:
+ // meanRate: compactions per hour
+ // fifteenMinuteRate: compactions per minute for last 15 minutes
+ double meanRateValuePerHour = metricsProxy.getMeanRate() * 3600; //
Convert per second to per hour
+ double fifteenMinuteRateValuePerMinute =
metricsProxy.getFifteenMinuteRate() * 60; // Convert per second to per minute
+
+ String meanRate = String.format("%.2f/hour", meanRateValuePerHour);
+ String fifteenMinuteRate = String.format("%.2f/minute",
fifteenMinuteRateValuePerMinute);
Review Comment:
Any reason we want to append the units suffix with the values? I'd recommend
we keep these as numeric values with the unit-specific details encoded into the
metric name. eg. `meanHourlyRate`. That way, clients are not expected to parse
the values.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]