This is an automated email from the ASF dual-hosted git repository.

domgarguilo pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new 5d58c806ba Improve SummarySerializer (#5946)
5d58c806ba is described below

commit 5d58c806ba80c35e1c37624a8598658b6936d245
Author: Dom G. <[email protected]>
AuthorDate: Tue Oct 14 16:25:08 2025 -0400

    Improve SummarySerializer (#5946)
    
    * Convert SummarySerializer class to a record to simplify
    * Add explicit null checks to protect against NPE
---
 .../accumulo/core/summary/SummaryReader.java       |  6 +--
 .../accumulo/core/summary/SummarySerializer.java   | 56 +++++++---------------
 2 files changed, 20 insertions(+), 42 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java 
b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
index 7888e80b67..d7e588ba91 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
@@ -247,7 +247,7 @@ public class SummaryReader {
     int i = 1;
     for (SummarySerializer summaryStore : stores) {
       out.printf("%sSummary %d of %d generated by : %s\n", indent, i, 
stores.size(),
-          summaryStore.getSummarizerConfiguration());
+          summaryStore.summarizerConfiguration());
       i++;
       summaryStore.print(indent, indent, out);
     }
@@ -297,11 +297,11 @@ public class SummaryReader {
     }
     for (SummarySerializer summaryStore : summaryStores) {
       if (summaryStore.exceededMaxSize()) {
-        initial.add(new 
SummaryCollection.FileSummary(summaryStore.getSummarizerConfiguration()));
+        initial.add(new 
SummaryCollection.FileSummary(summaryStore.summarizerConfiguration()));
       } else {
         Map<String,Long> summary = summaryStore.getSummary(ranges, factory);
         boolean exceeded = summaryStore.exceedsRange(ranges);
-        initial.add(new 
SummaryCollection.FileSummary(summaryStore.getSummarizerConfiguration(),
+        initial.add(new 
SummaryCollection.FileSummary(summaryStore.summarizerConfiguration(),
             summary, exceeded));
       }
     }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java 
b/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
index 1d5b257dd8..d316aeb98f 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarySerializer.java
@@ -46,34 +46,17 @@ import com.google.common.collect.ImmutableMap;
 
 /**
  * This class supports serializing summaries and periodically storing 
summaries. The implementations
- * attempts to generate around 10 summaries that are evenly spaced. This 
allows asking for summaries
+ * attempt to generate around 10 summaries that are evenly spaced. This allows 
asking for summaries
  * for sub-ranges of data in a rfile.
  *
  * <p>
- * At first summaries are created for every 1000 keys values. After 10 
summaries are added, the 10
+ * At first, summaries are created for every 1000 keys values. After 10 
summaries are added, the 10
  * summaries are merged to 5 and summaries are then created for every 2000 key 
values. The code
  * keeps merging summaries and doubling the amount of key values per summary. 
This results in each
  * summary covering about the same number of key values.
  */
-class SummarySerializer {
-
-  private final SummarizerConfiguration sconf;
-  private final LgSummaries[] allSummaries;
-
-  private SummarySerializer(SummarizerConfiguration sconf, LgSummaries[] 
allSummaries) {
-    this.sconf = sconf;
-    this.allSummaries = allSummaries;
-  }
-
-  private SummarySerializer(SummarizerConfiguration sconf) {
-    this.sconf = sconf;
-    // this indicates max size was exceeded
-    this.allSummaries = null;
-  }
-
-  public SummarizerConfiguration getSummarizerConfiguration() {
-    return sconf;
-  }
+record SummarySerializer(SummarizerConfiguration summarizerConfiguration,
+    LgSummaries[] allSummaries) {
 
   public void print(String prefix, String indent, PrintStream out) {
 
@@ -88,16 +71,21 @@ class SummarySerializer {
 
   public Map<String,Long> getSummary(List<RowRange> ranges, SummarizerFactory 
sf) {
 
-    Summarizer kvs = sf.getSummarizer(sconf);
+    Preconditions.checkState(allSummaries != null,
+        "Summaries were not stored because they exceeded the maximum size");
+
+    Summarizer kvs = sf.getSummarizer(summarizerConfiguration);
 
     Map<String,Long> summary = new HashMap<>();
     for (LgSummaries lgs : allSummaries) {
-      lgs.getSummary(ranges, kvs.combiner(sconf), summary);
+      lgs.getSummary(ranges, kvs.combiner(summarizerConfiguration), summary);
     }
     return summary;
   }
 
   public boolean exceedsRange(List<RowRange> ranges) {
+    Preconditions.checkState(allSummaries != null,
+        "Summaries were not stored because they exceeded the maximum size");
     ranges.forEach(SummarySerializer::validateSummaryRange);
     return Arrays.stream(allSummaries).anyMatch(lgs -> 
ranges.stream().anyMatch(lgs::exceedsRange));
   }
@@ -106,8 +94,7 @@ class SummarySerializer {
     return allSummaries == null;
   }
 
-  private static class SummaryStoreImpl
-      implements 
org.apache.accumulo.core.client.summary.Summarizer.StatisticConsumer {
+  private static class SummaryStoreImpl implements 
Summarizer.StatisticConsumer {
 
     HashMap<String,Long> summaries;
 
@@ -122,7 +109,7 @@ class SummarySerializer {
     private final SummarizerConfiguration conf;
     private Collector collector;
 
-    private final int maxSummaries = 10;
+    private static final int MAX_SUMMARIES = 10;
 
     private int cutoff = 1000;
     private int count = 0;
@@ -190,7 +177,7 @@ class SummarySerializer {
       Preconditions.checkState(!finished);
       summaries.add(new SummaryInfo(row, summary, count));
 
-      if (summaries.size() % 2 == 0 && summaries.size() > maxSummaries) {
+      if (summaries.size() % 2 == 0 && summaries.size() > MAX_SUMMARIES) {
         summaries = merge(summaries.size());
         cutoff *= 2;
       }
@@ -391,7 +378,8 @@ class SummarySerializer {
       throws IOException {
     boolean exceededMaxSize = in.readBoolean();
     if (exceededMaxSize) {
-      return new SummarySerializer(sconf);
+      // null indicates max size was exceeded
+      return new SummarySerializer(sconf, null);
     } else {
       WritableUtils.readVInt(in);
       // load symbol table
@@ -411,17 +399,7 @@ class SummarySerializer {
     }
   }
 
-  private static class LgSummaries {
-
-    private final Text firstRow;
-    private final SummaryInfo[] summaries;
-    private final String lgroupName;
-
-    LgSummaries(Text firstRow, SummaryInfo[] summaries, String lgroupName) {
-      this.firstRow = firstRow;
-      this.summaries = summaries;
-      this.lgroupName = lgroupName;
-    }
+  private record LgSummaries(Text firstRow, SummaryInfo[] summaries, String 
lgroupName) {
 
     boolean exceedsRange(RowRange range) {
 

Reply via email to