You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2019/06/10 19:42:27 UTC

[lucene-solr] branch branch_8x updated (8b6a0d0 -> 58d3761)

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

ab pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 8b6a0d0  LUCENE-8847: Code Cleanup: Rewrite StringBuilder.append with concatted strings (#707)
     new c932e7f  SOLR-13512: Raw index data analysis tool (extension of COLSTATUS collection command).
     new 58d3761  SOLR-13512: Fix a bug due to a different StoredFieldVisitor API in 8x.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/handler/admin/ColStatus.java   |  24 +-
 .../solr/handler/admin/CollectionsHandler.java     |   4 +
 .../solr/handler/admin/IndexSizeEstimator.java     | 713 +++++++++++++++++++++
 .../handler/admin/SegmentsInfoRequestHandler.java  |  41 +-
 .../solr/handler/admin/IndexSizeEstimatorTest.java | 241 +++++++
 solr/solr-ref-guide/src/collections-api.adoc       | 267 ++++++++
 .../solrj/request/CollectionAdminRequest.java      |  28 +
 8 files changed, 1311 insertions(+), 9 deletions(-)
 create mode 100644 solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
 create mode 100644 solr/core/src/test/org/apache/solr/handler/admin/IndexSizeEstimatorTest.java


[lucene-solr] 02/02: SOLR-13512: Fix a bug due to a different StoredFieldVisitor API in 8x.

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 58d3761bf5365c1dcdb0022c3f2e1ace38f950a4
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Jun 10 21:40:40 2019 +0200

    SOLR-13512: Fix a bug due to a different StoredFieldVisitor API in 8x.
---
 .../org/apache/solr/handler/admin/IndexSizeEstimator.java  | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java b/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
index 5ef02ff..35b8de0 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
@@ -56,7 +56,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.SuppressForbidden;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
@@ -625,13 +624,16 @@ public class IndexSizeEstimator {
     }
 
     /** Process a string field. */
-    public void stringField(FieldInfo fieldInfo, String value) throws IOException {
+    public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException {
       // trim the value if needed
-      int len = value != null ? UnicodeUtil.calcUTF16toUTF8Length(value, 0, value.length()) : 0;
-      if (value.length() > maxLength) {
-        value = value.substring(0, maxLength);
+      int len = value != null ? value.length : 0;
+      if (len > maxLength) {
+        byte[] newValue = new byte[maxLength];
+        System.arraycopy(value, 0, newValue, 0, maxLength);
+        value = newValue;
       }
-      countItem(fieldInfo.name, value, len);
+      String strValue = new String(value, "UTF-8");
+      countItem(fieldInfo.name, strValue, len);
     }
 
     /** Process a int numeric field. */


[lucene-solr] 01/02: SOLR-13512: Raw index data analysis tool (extension of COLSTATUS collection command).

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit c932e7ffd51d8880a37cbfbc68189f5b7045a05e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Jun 10 19:21:22 2019 +0200

    SOLR-13512: Raw index data analysis tool (extension of COLSTATUS collection command).
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/handler/admin/ColStatus.java   |  24 +-
 .../solr/handler/admin/CollectionsHandler.java     |   4 +
 .../solr/handler/admin/IndexSizeEstimator.java     | 711 +++++++++++++++++++++
 .../handler/admin/SegmentsInfoRequestHandler.java  |  41 +-
 .../solr/handler/admin/IndexSizeEstimatorTest.java | 241 +++++++
 solr/solr-ref-guide/src/collections-api.adoc       | 267 ++++++++
 .../solrj/request/CollectionAdminRequest.java      |  28 +
 8 files changed, 1309 insertions(+), 9 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 72851a0..2e6ca77 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -73,6 +73,8 @@ New Features
 
 * SOLR-13434: OpenTracing support for Solr (Cao Manh Dat)
 
+* SOLR-13512: Raw index data analysis tool (extension of COLSTATUS collection command). (ab)
+
 Bug Fixes
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java b/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
index b8e56a9..df022b9 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
@@ -57,9 +57,13 @@ public class ColStatus {
   private final ZkNodeProps props;
   private final SolrClientCache solrClientCache;
 
-  public static final String CORE_INFO_PROP = SegmentsInfoRequestHandler.WITH_CORE_INFO;
-  public static final String FIELD_INFO_PROP = SegmentsInfoRequestHandler.WITH_FIELD_INFO;
-  public static final String SIZE_INFO_PROP = SegmentsInfoRequestHandler.WITH_SIZE_INFO;
+  public static final String CORE_INFO_PROP = SegmentsInfoRequestHandler.CORE_INFO_PARAM;
+  public static final String FIELD_INFO_PROP = SegmentsInfoRequestHandler.FIELD_INFO_PARAM;
+  public static final String SIZE_INFO_PROP = SegmentsInfoRequestHandler.SIZE_INFO_PARAM;
+  public static final String RAW_SIZE_PROP = SegmentsInfoRequestHandler.RAW_SIZE_PARAM;
+  public static final String RAW_SIZE_SUMMARY_PROP = SegmentsInfoRequestHandler.RAW_SIZE_SUMMARY_PARAM;
+  public static final String RAW_SIZE_DETAILS_PROP = SegmentsInfoRequestHandler.RAW_SIZE_DETAILS_PARAM;
+  public static final String RAW_SIZE_SAMPLING_PERCENT_PROP = SegmentsInfoRequestHandler.RAW_SIZE_SAMPLING_PERCENT_PARAM;
   public static final String SEGMENTS_PROP = "segments";
 
   public ColStatus(HttpClient httpClient, ClusterState clusterState, ZkNodeProps props) {
@@ -80,6 +84,14 @@ public class ColStatus {
     boolean withSegments = props.getBool(SEGMENTS_PROP, false);
     boolean withCoreInfo = props.getBool(CORE_INFO_PROP, false);
     boolean withSizeInfo = props.getBool(SIZE_INFO_PROP, false);
+    boolean withRawSizeInfo = props.getBool(RAW_SIZE_PROP, false);
+    boolean withRawSizeSummary = props.getBool(RAW_SIZE_SUMMARY_PROP, false);
+    boolean withRawSizeDetails = props.getBool(RAW_SIZE_DETAILS_PROP, false);
+    Object samplingPercentVal = props.get(RAW_SIZE_SAMPLING_PERCENT_PROP);
+    Float samplingPercent = samplingPercentVal != null ? Float.parseFloat(String.valueOf(samplingPercentVal)) : null;
+    if (withRawSizeSummary || withRawSizeDetails) {
+      withRawSizeInfo = true;
+    }
     if (withFieldInfo || withSizeInfo) {
       withSegments = true;
     }
@@ -159,6 +171,12 @@ public class ColStatus {
           params.add(FIELD_INFO_PROP, "true");
           params.add(CORE_INFO_PROP, String.valueOf(withCoreInfo));
           params.add(SIZE_INFO_PROP, String.valueOf(withSizeInfo));
+          params.add(RAW_SIZE_PROP, String.valueOf(withRawSizeInfo));
+          params.add(RAW_SIZE_SUMMARY_PROP, String.valueOf(withRawSizeSummary));
+          params.add(RAW_SIZE_DETAILS_PROP, String.valueOf(withRawSizeDetails));
+          if (samplingPercent != null) {
+            params.add(RAW_SIZE_SAMPLING_PERCENT_PROP, String.valueOf(samplingPercent));
+          }
           QueryRequest req = new QueryRequest(params);
           NamedList<Object> rsp = client.request(req);
           rsp.remove("responseHeader");
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 07ec42a..2306916 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -530,6 +530,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           ColStatus.CORE_INFO_PROP,
           ColStatus.SEGMENTS_PROP,
           ColStatus.FIELD_INFO_PROP,
+          ColStatus.RAW_SIZE_PROP,
+          ColStatus.RAW_SIZE_SUMMARY_PROP,
+          ColStatus.RAW_SIZE_DETAILS_PROP,
+          ColStatus.RAW_SIZE_SAMPLING_PERCENT_PROP,
           ColStatus.SIZE_INFO_PROP);
       // make sure we can get the name if there's "name" but not "collection"
       if (props.containsKey(CoreAdminParams.NAME) && !props.containsKey(COLLECTION_PROP)) {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java b/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
new file mode 100644
index 0000000..5ef02ff
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/IndexSizeEstimator.java
@@ -0,0 +1,711 @@
+/*
+ * 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.apache.solr.handler.admin;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.CodecReader;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.StandardDirectoryReader;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Estimates the raw size of all uncompressed indexed data by scanning term, docValues and
+ * stored fields data. This utility also provides detailed statistics about term, docValues,
+ * postings and stored fields distributions.
+ */
+public class IndexSizeEstimator {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String TERMS = "terms";
+  public static final String STORED_FIELDS = "storedFields";
+  public static final String NORMS = "norms";
+  public static final String DOC_VALUES = "docValues";
+  public static final String POINTS = "points";
+  public static final String TERM_VECTORS = "termVectors";
+  public static final String SUMMARY = "summary";
+  public static final String DETAILS = "details";
+  public static final String FIELDS_BY_SIZE = "fieldsBySize";
+  public static final String TYPES_BY_SIZE = "typesBySize";
+
+  public static final int DEFAULT_SAMPLING_THRESHOLD = 100_000;
+  public static final float DEFAULT_SAMPLING_PERCENT = 5.0f;
+
+  private final IndexReader reader;
+  private final int topN;
+  private final int maxLength;
+  private final boolean withSummary;
+  private final boolean withDetails;
+  private int samplingThreshold = DEFAULT_SAMPLING_THRESHOLD;
+  private float samplingPercent = DEFAULT_SAMPLING_PERCENT;
+  private int samplingStep = 1;
+
+  public static final class Estimate implements MapWriter {
+    private final Map<String, Long> fieldsBySize;
+    private final Map<String, Long> typesBySize;
+    private final Map<String, Object> summary;
+    private final Map<String, Object> details;
+
+    public Estimate(Map<String, Long> fieldsBySize, Map<String, Long> typesBySize, Map<String, Object> summary, Map<String, Object> details) {
+      Objects.requireNonNull(fieldsBySize);
+      Objects.requireNonNull(typesBySize);
+      this.fieldsBySize = fieldsBySize;
+      this.typesBySize = typesBySize;
+      this.summary = summary;
+      this.details = details;
+    }
+
+    public Map<String, Long> getFieldsBySize() {
+      return fieldsBySize;
+    }
+
+    public Map<String, Long> getTypesBySize() {
+      return typesBySize;
+    }
+
+    public Map<String, String> getHumanReadableFieldsBySize() {
+      LinkedHashMap<String, String> result = new LinkedHashMap<>();
+      fieldsBySize.forEach((field, size) -> result.put(field, RamUsageEstimator.humanReadableUnits(size)));
+      return result;
+    }
+
+    public Map<String, String> getHumanReadableTypesBySize() {
+      LinkedHashMap<String, String> result = new LinkedHashMap<>();
+      typesBySize.forEach((field, size) -> result.put(field, RamUsageEstimator.humanReadableUnits(size)));
+      return result;
+    }
+
+    public Map<String, Object> getSummary() {
+      return summary;
+    }
+
+    public Map<String, Object> getDetails() {
+      return details;
+    }
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      ew.put(FIELDS_BY_SIZE, fieldsBySize);
+      ew.put(TYPES_BY_SIZE, typesBySize);
+      if (summary != null) {
+        ew.put(SUMMARY, summary);
+      }
+      if (details != null) {
+        ew.put(DETAILS, details);
+      }
+    }
+  }
+
+  public IndexSizeEstimator(IndexReader reader, int topN, int maxLength, boolean withSummary, boolean withDetails) {
+    this.reader = reader;
+    this.topN = topN;
+    this.maxLength = maxLength;
+    this.withSummary = withSummary;
+    this.withDetails = withDetails;
+  }
+
+  /**
+   * Set the sampling threshold. If the index has more documents than this threshold
+   * then only some values will be sampled and the totals will be extrapolated.
+   * @param threshold size threshold (number of documents). Default value is {@link #DEFAULT_SAMPLING_THRESHOLD}.
+   *                  Setting this to values &lt;= 0 means no threshold (and no sampling).
+   */
+  public void setSamplingThreshold(int threshold) {
+    if (threshold <= 0) {
+      threshold = Integer.MAX_VALUE;
+    }
+    this.samplingThreshold = threshold;
+  }
+
+  /**
+   * Sampling percent (a number greater than 0 and less or equal to 100). When index size exceeds
+   * the threshold then approximately only this percent of data will be retrieved from the index and the
+   * totals will be extrapolated.
+   * @param percent sample percent. Default value is {@link #DEFAULT_SAMPLING_PERCENT}.
+   * @throws IllegalArgumentException when value is less than or equal to 0.0 or greater than 100.0, or
+   *        the sampling percent is so small that less than 10 documents would be sampled.
+   */
+  public void setSamplingPercent(float percent) throws IllegalArgumentException {
+    if (percent <= 0 || percent > 100) {
+      throw new IllegalArgumentException("samplingPercent must be 0 < percent <= 100");
+    }
+    if (reader.maxDoc() > samplingThreshold) {
+      samplingStep = Math.round(100.0f / samplingPercent);
+      log.info("- number of documents {} larger than {}, sampling percent is {} and sampling step {}", reader.maxDoc(), samplingThreshold, samplingPercent, samplingStep);
+      if (reader.maxDoc() / samplingStep < 10) {
+        throw new IllegalArgumentException("Out of " + reader.maxDoc() + " less than 10 documents would be sampled, which is too unreliable. Increase the samplingPercent.");
+      }
+    }
+    this.samplingPercent = percent;
+  }
+
+  public Estimate estimate() throws Exception {
+    Map<String, Object> details = new LinkedHashMap<>();
+    Map<String, Object> summary = new LinkedHashMap<>();
+    estimateStoredFields(details);
+    estimateTerms(details);
+    estimateNorms(details);
+    estimatePoints(details);
+    estimateTermVectors(details);
+    estimateDocValues(details);
+    estimateSummary(details, summary);
+    if (samplingStep > 1) {
+      details.put("samplingPercent", samplingPercent);
+      details.put("samplingStep", samplingStep);
+    }
+    ItemPriorityQueue fieldSizeQueue = new ItemPriorityQueue(summary.size());
+    summary.forEach((field, perField) -> {
+      long size = ((AtomicLong)((Map<String, Object>)perField).get("totalSize")).get();
+      if (size > 0) {
+        fieldSizeQueue.insertWithOverflow(new Item(field, size));
+      }
+    });
+    Map<String, Long> fieldsBySize = new LinkedHashMap<>();
+    fieldSizeQueue._forEachEntry((k, v) -> fieldsBySize.put((String)k, (Long)v));
+    Map<String, AtomicLong> typeSizes = new HashMap<>();
+    summary.forEach((field, perField) -> {
+      Map<String, Object> perType = (Map<String, Object>)((Map<String, Object>)perField).get("perType");
+      perType.forEach((type, size) -> {
+        if (type.contains("_lengths")) {
+          AtomicLong totalSize = typeSizes.computeIfAbsent(type.replace("_lengths", ""), t -> new AtomicLong());
+          totalSize.addAndGet(((AtomicLong)size).get());
+        }
+      });
+    });
+    ItemPriorityQueue typesSizeQueue = new ItemPriorityQueue(typeSizes.size());
+    typeSizes.forEach((type, size) -> {
+      if (size.get() > 0) {
+        typesSizeQueue.insertWithOverflow(new Item(type, size.get()));
+      }
+    });
+    Map<String, Long> typesBySize = new LinkedHashMap<>();
+    typesSizeQueue._forEachEntry((k, v) -> typesBySize.put((String)k, (Long)v));
+    // sort summary by field size
+    Map<String, Object> newSummary = new LinkedHashMap<>();
+    fieldsBySize.keySet().forEach(k -> newSummary.put(String.valueOf(k), summary.get(k)));
+    // convert everything to maps and primitives
+    convert(newSummary);
+    convert(details);
+    return new Estimate(fieldsBySize, typesBySize, withSummary ? newSummary : null, withDetails ? details : null);
+  }
+
+  private void convert(Map<String, Object> result) {
+    for (Map.Entry<String, Object> entry : result.entrySet()) {
+      Object value = entry.getValue();
+      if (value instanceof ItemPriorityQueue) {
+        ItemPriorityQueue queue = (ItemPriorityQueue)value;
+        Map<String, Object> map = new LinkedHashMap<>();
+        queue.toMap(map);
+        entry.setValue(map);
+      } else if (value instanceof MapWriterSummaryStatistics) {
+        MapWriterSummaryStatistics stats = (MapWriterSummaryStatistics)value;
+        Map<String, Object> map = new LinkedHashMap<>();
+        stats.toMap(map);
+        entry.setValue(map);
+      } else if (value instanceof AtomicLong) {
+        entry.setValue(((AtomicLong)value).longValue());
+      } else if (value instanceof Map) {
+        // recurse
+        convert((Map<String, Object>)value);
+      }
+    }
+  }
+
+  private void estimateSummary(Map<String, Object> details, Map<String, Object> summary) {
+    log.info("- preparing summary...");
+    details.forEach((type, perType) -> {
+      ((Map<String, Object>)perType).forEach((field, perField) -> {
+        Map<String, Object> perFieldSummary = (Map<String, Object>)summary.computeIfAbsent(field, f -> new HashMap<>());
+        ((Map<String, Object>)perField).forEach((k, val) -> {
+          if (val instanceof SummaryStatistics) {
+            SummaryStatistics stats = (SummaryStatistics)val;
+            if (k.startsWith("lengths")) {
+              AtomicLong total = (AtomicLong)perFieldSummary.computeIfAbsent("totalSize", kt -> new AtomicLong());
+              total.addAndGet((long)stats.getSum());
+            }
+            Map<String, Object> perTypeSummary = (Map<String, Object>)perFieldSummary.computeIfAbsent("perType", pt -> new HashMap<>());
+            AtomicLong total = (AtomicLong)perTypeSummary.computeIfAbsent(type + "_" + k, t -> new AtomicLong());
+            total.addAndGet((long)stats.getSum());
+          }
+        });
+      });
+    });
+  }
+
+  private void estimateNorms(Map<String, Object> result) throws IOException {
+    log.info("- estimating norms...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext leafReaderContext : reader.leaves()) {
+      LeafReader leafReader = leafReaderContext.reader();
+      FieldInfos fieldInfos = leafReader.getFieldInfos();
+      for (FieldInfo info : fieldInfos) {
+        NumericDocValues norms = leafReader.getNormValues(info.name);
+        if (norms == null) {
+          continue;
+        }
+        Map<String, Object> perField = stats.computeIfAbsent(info.name, n -> new HashMap<>());
+        SummaryStatistics lengthSummary = (SummaryStatistics)perField.computeIfAbsent("lengths", s -> new MapWriterSummaryStatistics());
+        while (norms.advance(norms.docID() + samplingStep) != DocIdSetIterator.NO_MORE_DOCS) {
+          for (int i = 0; i < samplingStep; i++) {
+            lengthSummary.addValue(8);
+          }
+        }
+      }
+    }
+    result.put(NORMS, stats);
+  }
+
+  private void estimatePoints(Map<String, Object> result) throws IOException {
+    log.info("- estimating points...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext leafReaderContext : reader.leaves()) {
+      LeafReader leafReader = leafReaderContext.reader();
+      FieldInfos fieldInfos = leafReader.getFieldInfos();
+      for (FieldInfo info : fieldInfos) {
+        PointValues values = leafReader.getPointValues(info.name);
+        if (values == null) {
+          continue;
+        }
+        Map<String, Object> perField = stats.computeIfAbsent(info.name, n -> new HashMap<>());
+        SummaryStatistics lengthSummary = (SummaryStatistics)perField.computeIfAbsent("lengths", s -> new MapWriterSummaryStatistics());
+        lengthSummary.addValue(values.size() * values.getBytesPerDimension() * values.getNumIndexDimensions());
+      }
+    }
+    result.put(POINTS, stats);
+  }
+
+  private void estimateTermVectors(Map<String, Object> result) throws IOException {
+    log.info("- estimating term vectors...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext leafReaderContext : reader.leaves()) {
+      LeafReader leafReader = leafReaderContext.reader();
+      Bits liveDocs = leafReader.getLiveDocs();
+      for (int docId = 0; docId < leafReader.maxDoc(); docId += samplingStep) {
+        if (liveDocs != null && !liveDocs.get(docId)) {
+          continue;
+        }
+        Fields termVectors = leafReader.getTermVectors(docId);
+        if (termVectors == null) {
+          continue;
+        }
+        for (String field : termVectors) {
+          Terms terms = termVectors.terms(field);
+          if (terms == null) {
+            continue;
+          }
+          estimateTermStats(field, terms, stats, true);
+        }
+      }
+    }
+    result.put(TERM_VECTORS, stats);
+  }
+
+  private void estimateDocValues(Map<String, Object> result) throws IOException {
+    log.info("- estimating docValues...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      FieldInfos fieldInfos = leafReader.getFieldInfos();
+      for (FieldInfo info : fieldInfos) {
+        // binary
+        countDocValues(stats, info.name, "binary", leafReader.getBinaryDocValues(info.name), values -> {
+          try {
+            BytesRef value = ((BinaryDocValues) values).binaryValue();
+            return value.length;
+          } catch (IOException e) {
+            // ignore
+          }
+          return 0;
+        });
+        // numeric
+        countDocValues(stats, info.name, "numeric", leafReader.getNumericDocValues(info.name), values -> 8);
+        countDocValues(stats, info.name, "sorted", leafReader.getSortedDocValues(info.name), values -> {
+          try {
+            TermsEnum termsEnum = ((SortedDocValues) values).termsEnum();
+            BytesRef term;
+            while ((term = termsEnum.next()) != null) {
+              return term.length;
+            }
+          } catch (IOException e) {
+            // ignore
+          }
+          return 0;
+        });
+        countDocValues(stats, info.name, "sortedNumeric", leafReader.getSortedNumericDocValues(info.name),
+            values -> ((SortedNumericDocValues) values).docValueCount() * 8);
+        countDocValues(stats, info.name, "sortedSet", leafReader.getSortedSetDocValues(info.name), values -> {
+          try {
+            TermsEnum termsEnum = ((SortedSetDocValues) values).termsEnum();
+            BytesRef term;
+            while ((term = termsEnum.next()) != null) {
+              return term.length;
+            }
+          } catch (IOException e) {
+            // ignore
+          }
+          return 0;
+        });
+      }
+    }
+    result.put(DOC_VALUES, stats);
+  }
+
+  private void countDocValues(Map<String, Map<String, Object>> stats, String field, String type, DocIdSetIterator values,
+                              Function<DocIdSetIterator, Integer> valueLength) throws IOException {
+    if (values == null) {
+      return;
+    }
+    Map<String, Object> perField = stats.computeIfAbsent(field, n -> new HashMap<>());
+    SummaryStatistics lengthSummary = (SummaryStatistics)perField.computeIfAbsent("lengths_" + type, s -> new MapWriterSummaryStatistics());
+    while (values.advance(values.docID() + samplingStep) != DocIdSetIterator.NO_MORE_DOCS) {
+      int len = valueLength.apply(values);
+      for (int i = 0; i < samplingStep; i++) {
+        lengthSummary.addValue(len);
+      }
+    }
+  }
+
+  private void estimateTerms(Map<String, Object> result) throws IOException {
+    log.info("- estimating terms...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      FieldInfos fieldInfos = leafReader.getFieldInfos();
+      for (FieldInfo info : fieldInfos) {
+        Terms terms = leafReader.terms(info.name);
+        if (terms == null) {
+          continue;
+        }
+        estimateTermStats(info.name, terms, stats, false);
+      }
+    }
+    result.put(TERMS, stats);
+  }
+
+  private void estimateTermStats(String field, Terms terms, Map<String, Map<String, Object>> stats, boolean isSampling) throws IOException {
+    Map<String, Object> perField = stats.computeIfAbsent(field, n -> new HashMap<>());
+    SummaryStatistics lengthSummary = (SummaryStatistics)perField.computeIfAbsent("lengths_terms", s -> new MapWriterSummaryStatistics());
+    SummaryStatistics docFreqSummary = (SummaryStatistics)perField.computeIfAbsent("docFreqs", s -> new MapWriterSummaryStatistics());
+    SummaryStatistics totalFreqSummary = (SummaryStatistics)perField.computeIfAbsent("lengths_postings", s -> new MapWriterSummaryStatistics());
+    // TODO: add this at some point
+    //SummaryStatistics impactsSummary = (SummaryStatistics)perField.computeIfAbsent("lengths_impacts", s -> new MapWriterSummaryStatistics());
+    SummaryStatistics payloadSummary = null;
+    if (terms.hasPayloads()) {
+      payloadSummary = (SummaryStatistics)perField.computeIfAbsent("lengths_payloads", s -> new MapWriterSummaryStatistics());
+    }
+    ItemPriorityQueue topLen = (ItemPriorityQueue)perField.computeIfAbsent("topLen", s -> new ItemPriorityQueue(topN));
+    ItemPriorityQueue topTotalFreq = (ItemPriorityQueue)perField.computeIfAbsent("topTotalFreq", s -> new ItemPriorityQueue(topN));
+    TermsEnum termsEnum = terms.iterator();
+    BytesRef term;
+    PostingsEnum postings = null;
+    while ((term = termsEnum.next()) != null) {
+      if (isSampling) {
+        for (int i = 0; i < samplingStep; i++) {
+          lengthSummary.addValue(term.length);
+          docFreqSummary.addValue(termsEnum.docFreq());
+          totalFreqSummary.addValue(termsEnum.totalTermFreq());
+        }
+      } else {
+        lengthSummary.addValue(term.length);
+        docFreqSummary.addValue(termsEnum.docFreq());
+        totalFreqSummary.addValue(termsEnum.totalTermFreq());
+      }
+      if (terms.hasPayloads()) {
+        postings = termsEnum.postings(postings, PostingsEnum.ALL);
+        while (postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+          int freq = postings.freq();
+          for (int i = 0; i < freq; i++) {
+            if (postings.nextPosition() < 0) {
+              break;
+            }
+            BytesRef payload = postings.getPayload();
+            if (payload != null) {
+              if (isSampling) {
+                for (int k = 0; k < samplingStep; k++) {
+                  payloadSummary.addValue(payload.length);
+                }
+              } else {
+                payloadSummary.addValue(payload.length);
+              }
+            }
+          }
+        }
+      }
+      String value = term.utf8ToString();
+      if (value.length() > maxLength) {
+        value = value.substring(0, maxLength);
+      }
+      topLen.insertWithOverflow(new Item(value, term.length));
+      topTotalFreq.insertWithOverflow(new Item(value, termsEnum.totalTermFreq()));
+    }
+  }
+
+
+  private void estimateStoredFields(Map<String, Object> result) throws IOException {
+    log.info("- estimating stored fields...");
+    Map<String, Map<String, Object>> stats = new HashMap<>();
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      EstimatingVisitor visitor = new EstimatingVisitor(stats, topN, maxLength, samplingStep);
+      Bits liveDocs = leafReader.getLiveDocs();
+      if (leafReader instanceof CodecReader) {
+        CodecReader codecReader = (CodecReader)leafReader;
+        StoredFieldsReader storedFieldsReader = codecReader.getFieldsReader();
+        // this instance may be faster for a full sequential pass
+        storedFieldsReader = storedFieldsReader.getMergeInstance();
+        for (int docId = 0; docId < leafReader.maxDoc(); docId += samplingStep) {
+          if (liveDocs != null && !liveDocs.get(docId)) {
+            continue;
+          }
+          storedFieldsReader.visitDocument(docId, visitor);
+        }
+        storedFieldsReader.close();
+      } else {
+        for (int docId = 0; docId < leafReader.maxDoc(); docId += samplingStep) {
+          if (liveDocs != null && !liveDocs.get(docId)) {
+            continue;
+          }
+          leafReader.document(docId, visitor);
+        }
+      }
+    }
+    result.put(STORED_FIELDS, stats);
+  }
+
+  public static class Item {
+    Object value;
+    long size;
+
+    public Item(Object value, long size) {
+      this.value = value;
+      this.size = size;
+    }
+
+    public String toString() {
+      return "size=" + size + ", value=" + value;
+    }
+  }
+
+  public static class MapWriterSummaryStatistics extends SummaryStatistics implements MapWriter {
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      ew.put("n", getN());
+      ew.put("min", getMin());
+      ew.put("max", getMax());
+      ew.put("sum", getSum());
+      ew.put("mean", getMean());
+      ew.put("geoMean", getGeometricMean());
+      ew.put("variance", getVariance());
+      ew.put("populationVariance", getPopulationVariance());
+      ew.put("stddev", getStandardDeviation());
+      ew.put("secondMoment", getSecondMoment());
+      ew.put("sumOfSquares", getSumsq());
+      ew.put("sumOfLogs", getSumOfLogs());
+    }
+  }
+
+  public static class ItemPriorityQueue extends PriorityQueue<Item> implements MapWriter {
+
+    public ItemPriorityQueue(int maxSize) {
+      super(maxSize);
+    }
+
+    @Override
+    protected boolean lessThan(Item a, Item b) {
+      return a.size < b.size;
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      Iterator<Item> it = iterator();
+      while (it.hasNext()) {
+        if (sb.length() > 0) {
+          sb.append('\n');
+        }
+        sb.append(it.next());
+      }
+      return sb.toString();
+    }
+
+    // WARNING: destructive! empties the queue
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      Item[] items = new Item[size()];
+      int pos = size() - 1;
+      while (size() > 0) {
+        items[pos] = pop();
+        pos--;
+      }
+      for (Item item : items) {
+        ew.put(String.valueOf(item.value), item.size);
+      }
+    }
+  }
+
+  private static class EstimatingVisitor extends StoredFieldVisitor {
+    final Map<String, Map<String, Object>> stats;
+    final int topN;
+    final int maxLength;
+    final int samplingStep;
+
+    EstimatingVisitor(Map<String, Map<String, Object>> stats, int topN, int maxLength, int samplingStep) {
+      this.stats = stats;
+      this.topN = topN;
+      this.maxLength = maxLength;
+      this.samplingStep = samplingStep;
+    }
+
+    /** Process a binary field.
+     * @param value newly allocated byte array with the binary contents.
+     */
+    public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
+      // trim the value if needed
+      int len = value != null ? value.length : 0;
+      if (len > maxLength) {
+        byte[] newValue = new byte[maxLength];
+        System.arraycopy(value, 0, newValue, 0, maxLength);
+        value = newValue;
+      }
+      String strValue = new BytesRef(value).toString();
+      countItem(fieldInfo.name, strValue, len);
+    }
+
+    /** Process a string field. */
+    public void stringField(FieldInfo fieldInfo, String value) throws IOException {
+      // trim the value if needed
+      int len = value != null ? UnicodeUtil.calcUTF16toUTF8Length(value, 0, value.length()) : 0;
+      if (value.length() > maxLength) {
+        value = value.substring(0, maxLength);
+      }
+      countItem(fieldInfo.name, value, len);
+    }
+
+    /** Process a int numeric field. */
+    public void intField(FieldInfo fieldInfo, int value) throws IOException {
+      countItem(fieldInfo.name, String.valueOf(value), 4);
+    }
+
+    /** Process a long numeric field. */
+    public void longField(FieldInfo fieldInfo, long value) throws IOException {
+      countItem(fieldInfo.name, String.valueOf(value), 8);
+    }
+
+    /** Process a float numeric field. */
+    public void floatField(FieldInfo fieldInfo, float value) throws IOException {
+      countItem(fieldInfo.name, String.valueOf(value), 4);
+    }
+
+    /** Process a double numeric field. */
+    public void doubleField(FieldInfo fieldInfo, double value) throws IOException {
+      countItem(fieldInfo.name, String.valueOf(value), 8);
+    }
+
+    private void countItem(String field, Object value, int size) {
+      Map<String, Object> perField = stats.computeIfAbsent(field, n -> new HashMap<>());
+      SummaryStatistics summary = (SummaryStatistics)perField.computeIfAbsent("lengths", s -> new MapWriterSummaryStatistics());
+      for (int i = 0; i < samplingStep; i++) {
+        summary.addValue(size);
+      }
+      ItemPriorityQueue topNqueue = (ItemPriorityQueue)perField.computeIfAbsent("topLen", s-> new ItemPriorityQueue(topN));
+      topNqueue.insertWithOverflow(new Item(value, size));
+    }
+
+    @Override
+    public Status needsField(FieldInfo fieldInfo) throws IOException {
+      return Status.YES;
+    }
+  }
+
+  @SuppressForbidden(reason = "System.err and System.out required for a command-line utility")
+  public static void main(String[] args) throws Exception {
+    if (args.length == 0) {
+      System.err.println("Usage: " + IndexSizeEstimator.class.getName() + " [-topN NUM] [-maxLen NUM] [-summary] [-details] <indexDir>");
+      System.err.println();
+      System.err.println("\t<indexDir>\tpath to the index (parent path of 'segments_N' file)");
+      System.err.println("\t-topN NUM\tnumber of top largest items to collect");
+      System.err.println("\t-maxLen NUM\ttruncate the largest items to NUM bytes / characters");
+      System.err.println(-1);
+    }
+    String path = null;
+    int topN = 20;
+    int maxLen = 100;
+    boolean details = false;
+    boolean summary = false;
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals("-topN")) {
+        topN = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-maxLen")) {
+        maxLen = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-details")) {
+        details = true;
+      } else if (args[i].equals("-summary")) {
+        summary = true;
+      } else {
+        path = args[i];
+      }
+    }
+    if (path == null) {
+      System.err.println("ERROR: <indexDir> argument is required.");
+      System.exit(-2);
+    }
+    Directory dir = FSDirectory.open(Paths.get(path));
+    DirectoryReader reader = StandardDirectoryReader.open(dir);
+    IndexSizeEstimator stats = new IndexSizeEstimator(reader, topN, maxLen, summary, details);
+    System.out.println(Utils.toJSONString(stats.estimate()));
+    System.exit(0);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
index 2c0764e..a7b044e 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
@@ -74,9 +74,13 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class SegmentsInfoRequestHandler extends RequestHandlerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String WITH_FIELD_INFO = "fieldInfo";
-  public static final String WITH_CORE_INFO = "coreInfo";
-  public static final String WITH_SIZE_INFO = "sizeInfo";
+  public static final String FIELD_INFO_PARAM = "fieldInfo";
+  public static final String CORE_INFO_PARAM = "coreInfo";
+  public static final String SIZE_INFO_PARAM = "sizeInfo";
+  public static final String RAW_SIZE_PARAM = "rawSize";
+  public static final String RAW_SIZE_SUMMARY_PARAM = "rawSizeSummary";
+  public static final String RAW_SIZE_DETAILS_PARAM = "rawSizeDetails";
+  public static final String RAW_SIZE_SAMPLING_PERCENT_PARAM = "rawSizeSamplingPercent";
 
   private static final List<String> FI_LEGEND;
 
@@ -106,9 +110,15 @@ public class SegmentsInfoRequestHandler extends RequestHandlerBase {
 
   private void getSegmentsInfo(SolrQueryRequest req, SolrQueryResponse rsp)
       throws Exception {
-    boolean withFieldInfo = req.getParams().getBool(WITH_FIELD_INFO, false);
-    boolean withCoreInfo = req.getParams().getBool(WITH_CORE_INFO, false);
-    boolean withSizeInfo = req.getParams().getBool(WITH_SIZE_INFO, false);
+    boolean withFieldInfo = req.getParams().getBool(FIELD_INFO_PARAM, false);
+    boolean withCoreInfo = req.getParams().getBool(CORE_INFO_PARAM, false);
+    boolean withSizeInfo = req.getParams().getBool(SIZE_INFO_PARAM, false);
+    boolean withRawSizeInfo = req.getParams().getBool(RAW_SIZE_PARAM, false);
+    boolean withRawSizeSummary = req.getParams().getBool(RAW_SIZE_SUMMARY_PARAM, false);
+    boolean withRawSizeDetails = req.getParams().getBool(RAW_SIZE_DETAILS_PARAM, false);
+    if (withRawSizeSummary || withRawSizeDetails) {
+      withRawSizeInfo  = true;
+    }
     SolrIndexSearcher searcher = req.getSearcher();
 
     SegmentInfos infos =
@@ -187,6 +197,25 @@ public class SegmentsInfoRequestHandler extends RequestHandlerBase {
       rsp.add("fieldInfoLegend", FI_LEGEND);
     }
     rsp.add("segments", segmentInfos);
+    if (withRawSizeInfo) {
+      IndexSizeEstimator estimator = new IndexSizeEstimator(searcher.getRawReader(), 20, 100, withRawSizeSummary, withRawSizeDetails);
+      Object samplingPercentVal = req.getParams().get(RAW_SIZE_SAMPLING_PERCENT_PARAM);
+      if (samplingPercentVal != null) {
+        estimator.setSamplingPercent(Float.parseFloat(String.valueOf(samplingPercentVal)));
+      }
+      IndexSizeEstimator.Estimate estimate = estimator.estimate();
+      SimpleOrderedMap<Object> estimateMap = new SimpleOrderedMap<>();
+      // make the units more user-friendly
+      estimateMap.add(IndexSizeEstimator.FIELDS_BY_SIZE, estimate.getHumanReadableFieldsBySize());
+      estimateMap.add(IndexSizeEstimator.TYPES_BY_SIZE, estimate.getHumanReadableTypesBySize());
+      if (estimate.getSummary() != null) {
+        estimateMap.add(IndexSizeEstimator.SUMMARY, estimate.getSummary());
+      }
+      if (estimate.getDetails() != null) {
+        estimateMap.add(IndexSizeEstimator.DETAILS, estimate.getDetails());
+      }
+      rsp.add("rawSize", estimateMap);
+    }
   }
 
   private SimpleOrderedMap<Object> getSegmentInfo(
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/IndexSizeEstimatorTest.java b/solr/core/src/test/org/apache/solr/handler/admin/IndexSizeEstimatorTest.java
new file mode 100644
index 0000000..16cf270
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/IndexSizeEstimatorTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.apache.solr.handler.admin;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.TimeOut;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class IndexSizeEstimatorTest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static CloudSolrClient solrClient;
+  private static String collection = IndexSizeEstimator.class.getSimpleName() + "_collection";
+  private static int NUM_DOCS = 2000;
+  private static Set<String> fields;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    // create predictable field names
+    System.setProperty("solr.tests.numeric.dv", "true");
+    System.setProperty("solr.tests.numeric.points", "true");
+    System.setProperty("solr.tests.numeric.points.dv", "true");
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-dynamic"))
+        .configure();
+    solrClient = cluster.getSolrClient();
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .setMaxShardsPerNode(2).process(solrClient);
+    cluster.waitForActiveCollection(collection, 2, 4);
+    SolrInputDocument lastDoc = addDocs(collection, NUM_DOCS);
+    HashSet<String> docFields = new HashSet<>(lastDoc.keySet());
+    docFields.add("_version_");
+    docFields.add("_root_");
+    docFields.add("point_0___double");
+    docFields.add("point_1___double");
+    fields = docFields;
+  }
+
+  @AfterClass
+  public static void releaseClient() throws Exception {
+    solrClient = null;
+  }
+
+  @Test
+  public void testEstimator() throws Exception {
+    JettySolrRunner jetty = cluster.getRandomJetty(random());
+    String randomCoreName = jetty.getCoreContainer().getAllCoreNames().iterator().next();
+    SolrCore core = jetty.getCoreContainer().getCore(randomCoreName);
+    RefCounted<SolrIndexSearcher> searcherRef = core.getSearcher();
+    try {
+      SolrIndexSearcher searcher = searcherRef.get();
+      // limit the max length
+      IndexSizeEstimator estimator = new IndexSizeEstimator(searcher.getRawReader(), 20, 50, true, true);
+      IndexSizeEstimator.Estimate estimate = estimator.estimate();
+      Map<String, Long> fieldsBySize = estimate.getFieldsBySize();
+      assertFalse("empty fieldsBySize", fieldsBySize.isEmpty());
+      assertEquals(fieldsBySize.toString(), fields.size(), fieldsBySize.size());
+      fieldsBySize.forEach((k, v) -> assertTrue("unexpected size of " + k + ": " + v, v > 0));
+      Map<String, Long> typesBySize = estimate.getTypesBySize();
+      assertFalse("empty typesBySize", typesBySize.isEmpty());
+      assertTrue("expected at least 8 types: " + typesBySize.toString(), typesBySize.size() >= 8);
+      typesBySize.forEach((k, v) -> assertTrue("unexpected size of " + k + ": " + v, v > 0));
+      Map<String, Object> summary = estimate.getSummary();
+      assertNotNull("summary", summary);
+      assertFalse("empty summary", summary.isEmpty());
+      assertEquals(summary.keySet().toString(), fields.size(), summary.keySet().size());
+      Map<String, Object> details = estimate.getDetails();
+      assertNotNull("details", details);
+      assertFalse("empty details", details.isEmpty());
+      // by type
+      assertEquals(details.keySet().toString(), 6, details.keySet().size());
+
+      // check sampling
+      estimator.setSamplingThreshold(searcher.getRawReader().maxDoc() / 2);
+      IndexSizeEstimator.Estimate sampledEstimate = estimator.estimate();
+      Map<String, Long> sampledFieldsBySize = sampledEstimate.getFieldsBySize();
+      assertFalse("empty fieldsBySize", sampledFieldsBySize.isEmpty());
+      // verify that the sampled values are within 50% of the original values
+      fieldsBySize.forEach((field, size) -> {
+        Long sampledSize = sampledFieldsBySize.get(field);
+        assertNotNull("sampled size for " + field + " is missing in " + sampledFieldsBySize, sampledSize);
+        double delta = (double) size * 0.5;
+        assertEquals("sampled size of " + field + " is wildly off", (double)size, (double)sampledSize, delta);
+      });
+    } finally {
+      searcherRef.decref();
+      core.close();
+    }
+  }
+
+  @Test
+  public void testIntegration() throws Exception {
+    CollectionAdminResponse rsp = CollectionAdminRequest.collectionStatus(collection)
+        .setWithRawSizeInfo(true)
+        .setWithRawSizeSummary(true)
+        .setWithRawSizeDetails(true)
+        .process(solrClient);
+    CollectionAdminResponse sampledRsp = CollectionAdminRequest.collectionStatus(collection)
+        .setWithRawSizeInfo(true)
+        .setWithRawSizeSummary(true)
+        .setWithRawSizeDetails(true)
+        .setRawSizeSamplingPercent(5)
+        .process(solrClient);
+    assertEquals(0, rsp.getStatus());
+    assertEquals(0, sampledRsp.getStatus());
+    for (int i : Arrays.asList(1, 2)) {
+      NamedList<Object> segInfos = (NamedList<Object>) rsp.getResponse().findRecursive(collection, "shards", "shard" + i, "leader", "segInfos");
+      NamedList<Object> rawSize = (NamedList<Object>)segInfos.get("rawSize");
+      assertNotNull("rawSize missing", rawSize);
+      Map<String, Object> rawSizeMap = rawSize.asMap(10);
+      Map<String, Object> fieldsBySize = (Map<String, Object>)rawSizeMap.get(IndexSizeEstimator.FIELDS_BY_SIZE);
+      assertNotNull("fieldsBySize missing", fieldsBySize);
+      assertEquals(fieldsBySize.toString(), fields.size(), fieldsBySize.size());
+      fields.forEach(field -> assertNotNull("missing field " + field, fieldsBySize.get(field)));
+      Map<String, Object> typesBySize = (Map<String, Object>)rawSizeMap.get(IndexSizeEstimator.TYPES_BY_SIZE);
+      assertNotNull("typesBySize missing", typesBySize);
+      assertTrue("expected at least 8 types: " + typesBySize.toString(), typesBySize.size() >= 8);
+      Map<String, Object> summary = (Map<String, Object>)rawSizeMap.get(IndexSizeEstimator.SUMMARY);
+      assertNotNull("summary missing", summary);
+      assertEquals(summary.toString(), fields.size(), summary.size());
+      fields.forEach(field -> assertNotNull("missing field " + field, summary.get(field)));
+      Map<String, Object> details = (Map<String, Object>)rawSizeMap.get(IndexSizeEstimator.DETAILS);
+      assertNotNull("details missing", summary);
+      assertEquals(details.keySet().toString(), 6, details.size());
+
+      // compare with sampled
+      NamedList<Object> sampledRawSize = (NamedList<Object>) rsp.getResponse().findRecursive(collection, "shards", "shard" + i, "leader", "segInfos", "rawSize");
+      assertNotNull("sampled rawSize missing", sampledRawSize);
+      Map<String, Object> sampledRawSizeMap = rawSize.asMap(10);
+      Map<String, Object> sampledFieldsBySize = (Map<String, Object>)sampledRawSizeMap.get(IndexSizeEstimator.FIELDS_BY_SIZE);
+      assertNotNull("sampled fieldsBySize missing", sampledFieldsBySize);
+      fieldsBySize.forEach((k, v) -> {
+        double size = fromHumanReadableUnits((String)v);
+        double sampledSize = fromHumanReadableUnits((String)sampledFieldsBySize.get(k));
+        assertNotNull("sampled size missing for field " + k + " in " + sampledFieldsBySize, sampledSize);
+        double delta = size * 0.5;
+        assertEquals("sampled size of " + k + " is wildly off", size, sampledSize, delta);
+      });
+    }
+
+  }
+
+  private static double fromHumanReadableUnits(String value) {
+    String[] parts = value.split(" ");
+    assertEquals("invalid value", 2, parts.length);
+    double result = Double.parseDouble(parts[0]);
+    if (parts[1].equals("GB")) {
+      result = result * RamUsageEstimator.ONE_GB;
+    } else if (parts[1].equals("MB")) {
+      result = result * RamUsageEstimator.ONE_MB;
+    } else if (parts[1].equals("KB")) {
+      result = result * RamUsageEstimator.ONE_KB;
+    } else if (parts[1].equals("bytes")) {
+      // do nothing
+    } else {
+      fail("invalid unit in " + value);
+    }
+    return result;
+  }
+
+  private static SolrInputDocument addDocs(String collection, int n) throws Exception {
+    UpdateRequest ureq = new UpdateRequest();
+    SolrInputDocument doc = null;
+    for (int i = 0; i < n; i++) {
+      doc = new SolrInputDocument();
+      doc.addField("id", "id-" + i);
+      doc.addField("long_l", i);
+      doc.addField("long_tl", i);
+      doc.addField("multival_long_ll", i);
+      doc.addField("multival_long_ll", i + 1);
+      // indexed, not stored
+      doc.addField("string_sI", TestUtil.randomAnalysisString(random(), 100, true));
+      // stored, not indexed
+      doc.addField("string_sS", TestUtil.randomAnalysisString(random(), 100, true));
+      // multival, stored, indexed, tv, pos, offsets
+      doc.addField("tv_mv_string", TestUtil.randomAnalysisString(random(), 100, true));
+      doc.addField("tv_mv_string", TestUtil.randomAnalysisString(random(), 100, true));
+      //binary
+      doc.addField("payload", TestUtil.randomBinaryTerm(random()).bytes);
+      // points
+      doc.addField("point", random().nextInt(100) + "," + random().nextInt(100));
+      ureq.add(doc);
+    }
+    solrClient.request(ureq, collection);
+    solrClient.commit(collection);
+    // verify the number of docs
+    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+      QueryResponse rsp = solrClient.query(collection, params("q", "*:*", "rows", "0"));
+      if (rsp.getResults().getNumFound() == n) {
+        break;
+      }
+      timeOut.sleep(500);
+    }
+    assertFalse("timed out waiting for documents to be added", timeOut.hasTimedOut());
+    return doc;
+  }
+
+}
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index d255142..9de67d1 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -1566,6 +1566,78 @@ and their corresponding Solr schema types.
 Optional boolean. If true then additional information about the index files
 size and their RAM usage will be provided.
 
+==== Index Size Analysis Tool
+The `COLSTATUS` command also provides a tool for analyzing and estimating the composition of raw index data. Please note that
+this tool should be used with care because it generates a significant IO load on all shard leaders of the
+analyzed collections. A sampling threshold and a sampling percent parameters can be adjusted to reduce this
+load to some degree.
+
+Size estimates produced by this tool are only approximate and represent the aggregated size of uncompressed
+index data. In reality these values would never occur, because Lucene (and Solr) always stores data in a
+compressed format - still, these values help to understand what occupies most of the space and the relative size
+of each type of data and each field in the index.
+
+In the following sections whenever "size" is mentioned it means an estimated aggregated size of
+uncompressed (raw) data.
+
+The following parameters are specific to this tool:
+
+`rawSize`::
+Optional boolean. If true then run the raw index data analysis tool (other boolean options below imply
+this option if any of them are true). Command response will include sections that show estimated breakdown of
+data size per field and per data type.
+
+`rawSizeSummary`::
+Optional boolean. If true then include also a more detailed breakdown of data size per field and per type.
+
+`rawSizeDetails`::
+Optional boolean. If true then provide exhaustive details that include statistical distribution of items per
+field and per type as well as top 20 largest items per field.
+
+`rawSizeSamplingPercent`::
+Optional float. When the index is larger than a certain threshold (100k documents per shard) only a part of
+data is actually retrieved and analyzed in order to reduce the IO load, and then the final results are extrapolated.
+Values must be greater than 0 and less or equal to 100.0. Default value is 5.0. Very small values (between 0.0 and 1.0)
+may introduce significant estimation errors. Also, values that would result in less than 10 documents being sampled
+are rejected with an exception.
+
+Response for this command always contains two sections:
+
+* `fieldsBySize` is a map where field names are keys and values are estimated sizes of raw (uncompressed) data
+that belongs to the field. The map is sorted by size so that it's easy to see what field occupies most space.
+
+* `typesBySize` is a map where data types are the keys and values are estimates sizes of raw (uncompressed) data
+of particular type. This map is also sorted by size.
+
+Optional sections include:
+
+* `summary` section containing a breakdown of data sizes for each field by data type.
+
+* `details` section containing detailed statistical summary of size distribution within each field, per data type.
+This section also shows `topN` values by size from each field.
+
+Data types shown in the response can be roughly divided into the following groups:
+
+* `storedFields` - represents the raw uncompressed data in stored fields. Eg. for UTF-8 strings this represents
+the aggregated sum of the number of bytes in the strings' UTF-8 representation, for long numbers this is 8 bytes per value, etc.
+
+* `terms_terms` - represents the aggregated size of the term dictionary. The size of this data is affected by the
+the number and length of unique terms, which in turn depends on the field size and the analysis chain.
+
+* `terms_postings` - represents the aggregated size of all term position and offset information, if present.
+This information may be absent if position-based searching, such as phrase queries, is not needed.
+
+* `terms_payloads` - represents the aggregated size of all per-term payload data, if present.
+
+* `norms` - represents the aggregated size of field norm information. This information may be omitted if a field
+has an `omitNorms` flag in the schema, which is common for fields that don't need weighting or scoring by field length.
+
+* `termVectors` - represents the aggregated size of term vectors.
+
+* `docValues_*` - represents aggregated size of doc values, by type (eg. `docValues_numeric`, `docValues_binary`, etc).
+
+* `points` - represents aggregated size of point values.
+
 === COLSTATUS Response
 The response will include an overview of the collection status, the number of
 active or inactive shards and replicas, and additional index information
@@ -1717,6 +1789,201 @@ http://localhost:8983/solr/admin/collections?action=COLSTATUS&collection=getting
                                   }}}}}}}}}}}
 ----
 
+Example of using the raw index data analysis tool:
+
+*Input*
+
+[source,text]
+----
+http://localhost:8983/solr/admin/collections?action=COLSTATUS&collection=gettingstarted&rawSize=true&rawSizeSamplingPercent=0.1
+----
+
+*Output*
+
+[source,json]
+----
+{
+    "responseHeader": {
+        "status": 0,
+        "QTime": 26812
+    },
+    "gettingstarted": {
+        "stateFormat": 2,
+        "znodeVersion": 33,
+        "properties": {
+            "autoAddReplicas": "false",
+            "maxShardsPerNode": "-1",
+            "nrtReplicas": "2",
+            "pullReplicas": "0",
+            "replicationFactor": "2",
+            "router": {
+                "name": "compositeId"
+            },
+            "tlogReplicas": "0"
+        },
+        "activeShards": 2,
+        "inactiveShards": 0,
+        "schemaNonCompliant": [
+            "(NONE)"
+        ],
+        "shards": {
+            "shard1": {
+                "state": "active",
+                "range": "80000000-ffffffff",
+                "replicas": {
+                    "total": 2,
+                    "active": 2,
+                    "down": 0,
+                    "recovering": 0,
+                    "recovery_failed": 0
+                },
+                "leader": {
+                    "coreNode": "core_node5",
+                    "core": "gettingstarted_shard1_replica_n2",
+                    "base_url": "http://192.168.0.80:8983/solr",
+                    "node_name": "192.168.0.80:8983_solr",
+                    "state": "active",
+                    "type": "NRT",
+                    "force_set_state": "false",
+                    "leader": "true",
+                    "segInfos": {
+                        "info": {
+                            "minSegmentLuceneVersion": "9.0.0",
+                            "commitLuceneVersion": "9.0.0",
+                            "numSegments": 46,
+                            "segmentsFileName": "segments_4h",
+                            "totalMaxDoc": 3283741,
+                            "userData": {
+                                "commitCommandVer": "1635676266902323200",
+                                "commitTimeMSec": "1559902446318"
+                            }
+                        },
+                        "rawSize": {
+                            "fieldsBySize": {
+                                "revision.text": "7.9 GB",
+                                "revision.text_str": "734.7 MB",
+                                "revision.comment_str": "259.1 MB",
+                                "revision": "239.2 MB",
+                                "revision.sha1": "211.9 MB",
+                                "revision.comment": "201.3 MB",
+                                "title": "114.9 MB",
+                                "revision.contributor": "103.5 MB",
+                                "revision.sha1_str": "96.4 MB",
+                                "revision.id": "75.2 MB",
+                                "ns": "75.2 MB",
+                                "revision.timestamp": "75.2 MB",
+                                "revision.contributor.id": "74.7 MB",
+                                "revision.format": "69 MB",
+                                "id": "65 MB",
+                                "title_str": "26.8 MB",
+                                "revision.model_str": "25.4 MB",
+                                "_version_": "24.9 MB",
+                                "_root_": "24.7 MB",
+                                "revision.contributor.ip_str": "22 MB",
+                                "revision.contributor_str": "21.8 MB",
+                                "revision_str": "15.5 MB",
+                                "revision.contributor.ip": "13.5 MB",
+                                "restrictions_str": "428.7 KB",
+                                "restrictions": "164.2 KB",
+                                "name_str": "84 KB",
+                                "includes_str": "8.8 KB"
+                            },
+                            "typesBySize": {
+                                "storedFields": "7.8 GB",
+                                "docValues_sortedSet": "1.2 GB",
+                                "terms_postings": "788.8 MB",
+                                "terms_terms": "342.2 MB",
+                                "norms": "237 MB",
+                                "docValues_sortedNumeric": "124.3 MB",
+                                "points": "115.7 MB",
+                                "docValues_numeric": "24.9 MB",
+                                "docValues_sorted": "18.5 MB"
+                            }
+                        }
+                    }
+                }
+            },
+            "shard2": {
+                "state": "active",
+                "range": "0-7fffffff",
+                "replicas": {
+                    "total": 2,
+                    "active": 2,
+                    "down": 0,
+                    "recovering": 0,
+                    "recovery_failed": 0
+                },
+                "leader": {
+                    "coreNode": "core_node8",
+                    "core": "gettingstarted_shard2_replica_n6",
+                    "base_url": "http://192.168.0.80:8983/solr",
+                    "node_name": "192.168.0.80:8983_solr",
+                    "state": "active",
+                    "type": "NRT",
+                    "force_set_state": "false",
+                    "leader": "true",
+                    "segInfos": {
+                        "info": {
+                            "minSegmentLuceneVersion": "9.0.0",
+                            "commitLuceneVersion": "9.0.0",
+                            "numSegments": 55,
+                            "segmentsFileName": "segments_4d",
+                            "totalMaxDoc": 3284863,
+                            "userData": {
+                                "commitCommandVer": "1635676259742646272",
+                                "commitTimeMSec": "1559902445005"
+                            }
+                        },
+                        "rawSize": {
+                            "fieldsBySize": {
+                                "revision.text": "8.3 GB",
+                                "revision.text_str": "687.5 MB",
+                                "revision": "238.9 MB",
+                                "revision.sha1": "212 MB",
+                                "revision.comment_str": "211.5 MB",
+                                "revision.comment": "201.7 MB",
+                                "title": "115.9 MB",
+                                "revision.contributor": "103.4 MB",
+                                "revision.sha1_str": "96.3 MB",
+                                "ns": "75.2 MB",
+                                "revision.id": "75.2 MB",
+                                "revision.timestamp": "75.2 MB",
+                                "revision.contributor.id": "74.6 MB",
+                                "revision.format": "69 MB",
+                                "id": "67 MB",
+                                "title_str": "29.5 MB",
+                                "_version_": "24.8 MB",
+                                "revision.model_str": "24 MB",
+                                "revision.contributor_str": "21.7 MB",
+                                "revision.contributor.ip_str": "20.9 MB",
+                                "revision_str": "15.5 MB",
+                                "revision.contributor.ip": "13.8 MB",
+                                "restrictions_str": "411.1 KB",
+                                "restrictions": "132.9 KB",
+                                "name_str": "42 KB",
+                                "includes_str": "41 KB"
+                            },
+                            "typesBySize": {
+                                "storedFields": "8.2 GB",
+                                "docValues_sortedSet": "1.1 GB",
+                                "terms_postings": "787.4 MB",
+                                "terms_terms": "337.5 MB",
+                                "norms": "236.6 MB",
+                                "docValues_sortedNumeric": "124.1 MB",
+                                "points": "115.7 MB",
+                                "docValues_numeric": "24.9 MB",
+                                "docValues_sorted": "20.5 MB"
+                            }
+                        }
+                    }
+                }
+            }
+        }
+    }
+}
+----
+
+
 [[migrate]]
 == MIGRATE: Migrate Documents to Another Collection
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index b0e5c94..68f828d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -911,6 +911,10 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     protected Boolean withFieldInfo = null;
     protected Boolean withCoreInfo = null;
     protected Boolean withSizeInfo = null;
+    protected Boolean withRawSizeInfo = null;
+    protected Boolean withRawSizeSummary = null;
+    protected Boolean withRawSizeDetails = null;
+    protected Float rawSizeSamplingPercent = null;
 
     private ColStatus(String collection) {
       super(CollectionAction.COLSTATUS, collection);
@@ -936,6 +940,26 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       return this;
     }
 
+    public ColStatus setWithRawSizeInfo(boolean withRawSizeInfo) {
+      this.withRawSizeInfo = withRawSizeInfo;
+      return this;
+    }
+
+    public ColStatus setWithRawSizeSummary(boolean withRawSizeSummary) {
+      this.withRawSizeSummary = withRawSizeSummary;
+      return this;
+    }
+
+    public ColStatus setWithRawSizeDetails(boolean withRawSizeDetails) {
+      this.withRawSizeDetails = withRawSizeDetails;
+      return this;
+    }
+
+    public ColStatus setRawSizeSamplingPercent(float rawSizeSamplingPercent) {
+      this.rawSizeSamplingPercent = rawSizeSamplingPercent;
+      return this;
+    }
+
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams)super.getParams();
@@ -943,6 +967,10 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       params.setNonNull("fieldInfo", withFieldInfo);
       params.setNonNull("coreInfo", withCoreInfo);
       params.setNonNull("sizeInfo", withSizeInfo);
+      params.setNonNull("rawSizeInfo", withRawSizeInfo);
+      params.setNonNull("rawSizeSummary", withRawSizeSummary);
+      params.setNonNull("rawSizeDetails", withRawSizeDetails);
+      params.setNonNull("rawSizeSamplingPercent", rawSizeSamplingPercent);
       return params;
     }
   }