iliax 1 år sedan
förälder
incheckning
81999d9369

+ 12 - 6
kafka-ui-api/src/main/java/com/provectus/kafka/ui/emitter/OffsetsInfo.java

@@ -8,12 +8,13 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 import lombok.Getter;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.TopicPartition;
 
 
 @Slf4j
 @Slf4j
 @Getter
 @Getter
-public class OffsetsInfo {
+class OffsetsInfo {
 
 
   private final Consumer<?, ?> consumer;
   private final Consumer<?, ?> consumer;
 
 
@@ -23,7 +24,7 @@ public class OffsetsInfo {
   private final Set<TopicPartition> nonEmptyPartitions = new HashSet<>();
   private final Set<TopicPartition> nonEmptyPartitions = new HashSet<>();
   private final Set<TopicPartition> emptyPartitions = new HashSet<>();
   private final Set<TopicPartition> emptyPartitions = new HashSet<>();
 
 
-  public OffsetsInfo(Consumer<?, ?> consumer, String topic) {
+  OffsetsInfo(Consumer<?, ?> consumer, String topic) {
     this(consumer,
     this(consumer,
         consumer.partitionsFor(topic).stream()
         consumer.partitionsFor(topic).stream()
             .map(pi -> new TopicPartition(topic, pi.partition()))
             .map(pi -> new TopicPartition(topic, pi.partition()))
@@ -31,8 +32,7 @@ public class OffsetsInfo {
     );
     );
   }
   }
 
 
-  public OffsetsInfo(Consumer<?, ?> consumer,
-                     Collection<TopicPartition> targetPartitions) {
+  OffsetsInfo(Consumer<?, ?> consumer, Collection<TopicPartition> targetPartitions) {
     this.consumer = consumer;
     this.consumer = consumer;
     this.beginOffsets = consumer.beginningOffsets(targetPartitions);
     this.beginOffsets = consumer.beginningOffsets(targetPartitions);
     this.endOffsets = consumer.endOffsets(targetPartitions);
     this.endOffsets = consumer.endOffsets(targetPartitions);
@@ -46,8 +46,8 @@ public class OffsetsInfo {
     });
     });
   }
   }
 
 
-  public boolean assignedPartitionsFullyPolled() {
-    for (var tp: consumer.assignment()) {
+  boolean assignedPartitionsFullyPolled() {
+    for (var tp : consumer.assignment()) {
       Preconditions.checkArgument(endOffsets.containsKey(tp));
       Preconditions.checkArgument(endOffsets.containsKey(tp));
       if (endOffsets.get(tp) > consumer.position(tp)) {
       if (endOffsets.get(tp) > consumer.position(tp)) {
         return false;
         return false;
@@ -56,4 +56,10 @@ public class OffsetsInfo {
     return true;
     return true;
   }
   }
 
 
+  long summaryOffsetsRange() {
+    MutableLong cnt = new MutableLong();
+    nonEmptyPartitions.forEach(tp -> cnt.add(endOffsets.get(tp) - beginOffsets.get(tp)));
+    return cnt.getValue();
+  }
+
 }
 }

+ 6 - 1
kafka-ui-api/src/main/java/com/provectus/kafka/ui/emitter/SeekOperations.java

@@ -52,7 +52,12 @@ public class SeekOperations {
     return offsetsInfo.assignedPartitionsFullyPolled();
     return offsetsInfo.assignedPartitionsFullyPolled();
   }
   }
 
 
-  //TODO: document
+  // sum of (end - start) offsets for all partitions
+  public long summaryOffsetsRange() {
+    return offsetsInfo.summaryOffsetsRange();
+  }
+
+  // sum of differences between initial consumer seek and current consumer position (across all partitions)
   public long offsetsProcessedFromSeek() {
   public long offsetsProcessedFromSeek() {
     MutableLong count = new MutableLong();
     MutableLong count = new MutableLong();
     offsetsForSeek.forEach((tp, initialOffset) -> count.add(consumer.position(tp) - initialOffset));
     offsetsForSeek.forEach((tp, initialOffset) -> count.add(consumer.position(tp) - initialOffset));

+ 5 - 7
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/analyze/AnalysisTasksStore.java

@@ -92,14 +92,12 @@ class AnalysisTasksStore {
         .result(completedState);
         .result(completedState);
   }
   }
 
 
-  @Value
   @Builder(toBuilder = true)
   @Builder(toBuilder = true)
-  private static class RunningAnalysis {
-    Instant startedAt;
-    double completenessPercent;
-    long msgsScanned;
-    long bytesScanned;
-    Closeable task;
+  private record RunningAnalysis(Instant startedAt,
+                                 double completenessPercent,
+                                 long msgsScanned,
+                                 long bytesScanned,
+                                 Closeable task) {
 
 
     TopicAnalysisProgressDTO toDto() {
     TopicAnalysisProgressDTO toDto() {
       return new TopicAnalysisProgressDTO()
       return new TopicAnalysisProgressDTO()

+ 11 - 23
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/analyze/TopicAnalysisService.java

@@ -47,26 +47,16 @@ public class TopicAnalysisService {
 
 
   public Mono<Void> analyze(KafkaCluster cluster, String topicName) {
   public Mono<Void> analyze(KafkaCluster cluster, String topicName) {
     return topicsService.getTopicDetails(cluster, topicName)
     return topicsService.getTopicDetails(cluster, topicName)
-        .doOnNext(topic ->
-            startAnalysis(
-                cluster,
-                topicName,
-                topic.getPartitions().values()
-                    .stream()
-                    .mapToLong(p -> p.getOffsetMax() - p.getOffsetMin())
-                    .sum()
-            )
-        ).then();
+        .doOnNext(topic -> startAnalysis(cluster, topicName))
+        .then();
   }
   }
 
 
-  private synchronized void startAnalysis(KafkaCluster cluster,
-                                          String topic,
-                                          long approxNumberOfMsgs) {
+  private synchronized void startAnalysis(KafkaCluster cluster, String topic) {
     var topicId = new TopicIdentity(cluster, topic);
     var topicId = new TopicIdentity(cluster, topic);
     if (analysisTasksStore.isAnalysisInProgress(topicId)) {
     if (analysisTasksStore.isAnalysisInProgress(topicId)) {
       throw new TopicAnalysisException("Topic is already analyzing");
       throw new TopicAnalysisException("Topic is already analyzing");
     }
     }
-    var task = new AnalysisTask(cluster, topicId, approxNumberOfMsgs);
+    var task = new AnalysisTask(cluster, topicId);
     analysisTasksStore.registerNewTask(topicId, task);
     analysisTasksStore.registerNewTask(topicId, task);
     SCHEDULER.schedule(task);
     SCHEDULER.schedule(task);
   }
   }
@@ -84,16 +74,14 @@ public class TopicAnalysisService {
     private final Instant startedAt = Instant.now();
     private final Instant startedAt = Instant.now();
 
 
     private final TopicIdentity topicId;
     private final TopicIdentity topicId;
-    private final long approxNumberOfMsgs;
 
 
     private final TopicAnalysisStats totalStats = new TopicAnalysisStats();
     private final TopicAnalysisStats totalStats = new TopicAnalysisStats();
     private final Map<Integer, TopicAnalysisStats> partitionStats = new HashMap<>();
     private final Map<Integer, TopicAnalysisStats> partitionStats = new HashMap<>();
 
 
     private final EnhancedConsumer consumer;
     private final EnhancedConsumer consumer;
 
 
-    AnalysisTask(KafkaCluster cluster, TopicIdentity topicId, long approxNumberOfMsgs) {
+    AnalysisTask(KafkaCluster cluster, TopicIdentity topicId) {
       this.topicId = topicId;
       this.topicId = topicId;
-      this.approxNumberOfMsgs = approxNumberOfMsgs;
       this.consumer = consumerGroupService.createConsumer(
       this.consumer = consumerGroupService.createConsumer(
           cluster,
           cluster,
           // to improve polling throughput
           // to improve polling throughput
@@ -114,9 +102,10 @@ public class TopicAnalysisService {
       try {
       try {
         log.info("Starting {} topic analysis", topicId);
         log.info("Starting {} topic analysis", topicId);
         consumer.partitionsFor(topicId.topicName)
         consumer.partitionsFor(topicId.topicName)
-            .forEach(i -> partitionStats.put(i.partition(), new TopicAnalysisStats()));
+            .forEach(tp -> partitionStats.put(tp.partition(), new TopicAnalysisStats()));
 
 
         var seekOperations = SeekOperations.create(consumer, new ConsumerPosition(BEGINNING, topicId.topicName, null));
         var seekOperations = SeekOperations.create(consumer, new ConsumerPosition(BEGINNING, topicId.topicName, null));
+        long summaryOffsetsRange = seekOperations.summaryOffsetsRange();
         seekOperations.assignAndSeekNonEmptyPartitions();
         seekOperations.assignAndSeekNonEmptyPartitions();
 
 
         while (!seekOperations.assignedPartitionsFullyPolled()) {
         while (!seekOperations.assignedPartitionsFullyPolled()) {
@@ -125,7 +114,7 @@ public class TopicAnalysisService {
             totalStats.apply(r);
             totalStats.apply(r);
             partitionStats.get(r.partition()).apply(r);
             partitionStats.get(r.partition()).apply(r);
           });
           });
-          updateProgress(seekOperations);
+          updateProgress(seekOperations.offsetsProcessedFromSeek(), summaryOffsetsRange);
         }
         }
         analysisTasksStore.setAnalysisResult(topicId, startedAt, totalStats, partitionStats);
         analysisTasksStore.setAnalysisResult(topicId, startedAt, totalStats, partitionStats);
         log.info("{} topic analysis finished", topicId);
         log.info("{} topic analysis finished", topicId);
@@ -141,14 +130,13 @@ public class TopicAnalysisService {
       }
       }
     }
     }
 
 
-    private void updateProgress(SeekOperations seekOperations) {
-      long processedOffsets = seekOperations.offsetsProcessedFromSeek();
-      if (processedOffsets > 0 && approxNumberOfMsgs != 0) {
+    private void updateProgress(long processedOffsets, long summaryOffsetsRange) {
+      if (processedOffsets > 0 && summaryOffsetsRange != 0) {
         analysisTasksStore.updateProgress(
         analysisTasksStore.updateProgress(
             topicId,
             topicId,
             totalStats.totalMsgs,
             totalStats.totalMsgs,
             totalStats.keysSize.sum + totalStats.valuesSize.sum,
             totalStats.keysSize.sum + totalStats.valuesSize.sum,
-            Math.min(100.0, (((double) processedOffsets) / approxNumberOfMsgs) * 100)
+            Math.min(100.0, (((double) processedOffsets) / summaryOffsetsRange) * 100)
         );
         );
       }
       }
     }
     }