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/26 14:38:22 UTC

[lucene-solr] branch branch_8_1 updated: LUCENE-8855: Add Accountable to some Query implementations.

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

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


The following commit(s) were added to refs/heads/branch_8_1 by this push:
     new 5dfdc30  LUCENE-8855: Add Accountable to some Query implementations.
5dfdc30 is described below

commit 5dfdc3029b231202e4a5af27c69d357e1a9a7a0e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Jun 26 15:26:44 2019 +0200

    LUCENE-8855: Add Accountable to some Query implementations.
---
 lucene/CHANGES.txt                                 |   6 +
 .../src/java/org/apache/lucene/index/Term.java     |  14 +-
 .../org/apache/lucene/search/AutomatonQuery.java   |  15 +-
 .../org/apache/lucene/search/LRUQueryCache.java    |   2 +-
 .../org/apache/lucene/search/PointInSetQuery.java  |  16 +-
 .../java/org/apache/lucene/util/ByteBlockPool.java |  13 +-
 .../src/java/org/apache/lucene/util/BytesRef.java  |   4 +-
 .../java/org/apache/lucene/util/BytesRefHash.java  |  16 +-
 .../src/java/org/apache/lucene/util/IntsRef.java   |   2 +-
 .../java/org/apache/lucene/util/LongBitSet.java    |   9 +-
 .../org/apache/lucene/util/RamUsageEstimator.java  | 274 ++++++++++++++++++++-
 .../lucene/util/automaton/CompiledAutomaton.java   |  17 +-
 .../apache/lucene/util/automaton/RunAutomaton.java |  17 +-
 .../apache/lucene/util/TestRamUsageEstimator.java  |  94 ++++++-
 .../lucene/search/join/GlobalOrdinalsQuery.java    |  19 +-
 .../search/join/GlobalOrdinalsWithScoreQuery.java  |  18 +-
 .../search/join/PointInSetIncludingScoreQuery.java |  17 +-
 .../search/join/TermsIncludingScoreQuery.java      |  20 +-
 .../org/apache/lucene/search/join/TermsQuery.java  |  18 +-
 .../java/org/apache/lucene/search/BM25FQuery.java  |  17 +-
 .../org/apache/lucene/search/CoveringQuery.java    |  14 +-
 .../lucene/search/DocValuesNumbersQuery.java       |  12 +-
 .../apache/lucene/search/DocValuesTermsQuery.java  |  12 +-
 .../java/org/apache/lucene/search/LongHashSet.java |  11 +-
 .../apache/lucene/search/TermAutomatonQuery.java   |  16 +-
 .../lucene/spatial3d/PointInGeo3DShapeQuery.java   |  14 +-
 .../search/suggest/document/ContextQuery.java      |  20 +-
 27 files changed, 665 insertions(+), 42 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 186397f..a6ebcdc 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -15,6 +15,12 @@ Bug fixes
 * LUCENE-8775: Improve tessellator to handle better cases where a hole share a vertex
   with the polygon. (Ignacio Vera)
 
+======================= Lucene 8.1.2 =======================
+
+Improvements
+
+* LUCENE-8855: Add Accountable to some Query implementations (ab, Adrien Grand)
+
 ======================= Lucene 8.1.1 =======================
 (No Changes)
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/Term.java b/lucene/core/src/java/org/apache/lucene/index/Term.java
index feb31e0..4ee8b42 100644
--- a/lucene/core/src/java/org/apache/lucene/index/Term.java
+++ b/lucene/core/src/java/org/apache/lucene/index/Term.java
@@ -23,8 +23,10 @@ import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.nio.charset.StandardCharsets;
 
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
   A Term represents a word from text.  This is the unit of search.  It is
@@ -34,7 +36,10 @@ import org.apache.lucene.util.BytesRefBuilder;
   Note that terms may represent more than words from text fields, but also
   things like dates, email addresses, urls, etc.  */
 
-public final class Term implements Comparable<Term> {
+public final class Term implements Comparable<Term>, Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(Term.class) +
+      RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
+
   String field;
   BytesRef bytes;
 
@@ -162,4 +167,11 @@ public final class Term implements Comparable<Term> {
 
   @Override
   public final String toString() { return field + ":" + text(); }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        (bytes != null ? bytes.bytes.length + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER : 0L);
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java b/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
index a4a5344..ed71c4a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
@@ -22,7 +22,9 @@ import java.io.IOException;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.Operations;
@@ -45,7 +47,9 @@ import org.apache.lucene.util.automaton.Operations;
  * </p>
  * @lucene.experimental
  */
-public class AutomatonQuery extends MultiTermQuery {
+public class AutomatonQuery extends MultiTermQuery implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(AutomatonQuery.class);
+
   /** the automaton to match index terms against */
   protected final Automaton automaton;
   protected final CompiledAutomaton compiled;
@@ -53,6 +57,8 @@ public class AutomatonQuery extends MultiTermQuery {
   protected final Term term;
   protected final boolean automatonIsBinary;
 
+  private final long ramBytesUsed; // cache
+
   /**
    * Create a new AutomatonQuery from an {@link Automaton}.
    * 
@@ -102,6 +108,8 @@ public class AutomatonQuery extends MultiTermQuery {
     this.automatonIsBinary = isBinary;
     // TODO: we could take isFinite too, to save a bit of CPU in CompiledAutomaton ctor?:
     this.compiled = new CompiledAutomaton(automaton, null, true, maxDeterminizedStates, isBinary);
+
+    this.ramBytesUsed = BASE_RAM_BYTES + term.ramBytesUsed() + automaton.ramBytesUsed() + compiled.ramBytesUsed();
   }
 
   @Override
@@ -168,4 +176,9 @@ public class AutomatonQuery extends MultiTermQuery {
   public boolean isAutomatonBinary() {
     return automatonIsBinary;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index c2448ed..95c1547 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -89,7 +89,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
 
   // approximate memory usage that we assign to all queries
   // this maps roughly to a BooleanQuery with a couple term clauses
-  static final long QUERY_DEFAULT_RAM_BYTES_USED = 1024;
+  static final long QUERY_DEFAULT_RAM_BYTES_USED = RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED;
 
   static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
       2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index ad9e258..7e1881f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -31,11 +31,13 @@ import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.PrefixCodedTerms;
 import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Abstract query class to find all documents whose single or multi-dimensional point values, previously indexed with e.g. {@link IntPoint},
@@ -49,13 +51,16 @@ import org.apache.lucene.util.FutureArrays;
  * @see PointValues
  * @lucene.experimental */
 
-public abstract class PointInSetQuery extends Query {
+public abstract class PointInSetQuery extends Query implements Accountable {
+  protected static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(PointInSetQuery.class);
+
   // A little bit overkill for us, since all of our "terms" are always in the same field:
   final PrefixCodedTerms sortedPackedPoints;
   final int sortedPackedPointsHashCode;
   final String field;
   final int numDims;
   final int bytesPerDim;
+  final long ramBytesUsed; // cache
   
   /** 
    * Iterator of encoded point values.
@@ -103,6 +108,10 @@ public abstract class PointInSetQuery extends Query {
     }
     sortedPackedPoints = builder.finish();
     sortedPackedPointsHashCode = sortedPackedPoints.hashCode();
+    ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(sortedPackedPoints);
+
   }
 
   @Override
@@ -423,4 +432,9 @@ public abstract class PointInSetQuery extends Query {
    * @return human readable value for debugging
    */
   protected abstract String toString(byte[] value);
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
index bd0b8ea..b543886 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
@@ -41,7 +41,9 @@ import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
  * 
  * @lucene.internal
  **/
-public final class ByteBlockPool {
+public final class ByteBlockPool implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(ByteBlockPool.class);
+
   public final static int BYTE_BLOCK_SHIFT = 15;
   public final static int BYTE_BLOCK_SIZE = 1 << BYTE_BLOCK_SHIFT;
   public final static int BYTE_BLOCK_MASK = BYTE_BLOCK_SIZE - 1;
@@ -392,5 +394,14 @@ public final class ByteBlockPool {
     byte[] buffer = buffers[bufferIndex];
     return buffer[pos];
   }
+
+  @Override
+  public long ramBytesUsed() {
+    long size = BASE_RAM_BYTES;
+    for (byte[] buffer : buffers) {
+      size += RamUsageEstimator.sizeOfObject(buffer);
+    }
+    return size;
+  }
 }
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
index a9a05e6..1debba4 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
@@ -31,9 +31,9 @@ package org.apache.lucene.util;
  * are sorted lexicographically, numerically treating elements as unsigned.
  * This is identical to Unicode codepoint order.
  */
-public final class BytesRef implements Comparable<BytesRef>,Cloneable {
+public final class BytesRef implements Comparable<BytesRef>, Cloneable {
   /** An empty byte array for convenience */
-  public static final byte[] EMPTY_BYTES = new byte[0]; 
+  public static final byte[] EMPTY_BYTES = new byte[0];
 
   /** The contents of the BytesRef. Should never be {@code null}. */
   public byte[] bytes;
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
index 6200fc5..f017ccb 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
@@ -42,7 +42,12 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
  * 
  * @lucene.internal
  */
-public final class BytesRefHash {
+public final class BytesRefHash implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(BytesRefHash.class) +
+      // size of scratch1
+      RamUsageEstimator.shallowSizeOf(BytesRef.class) +
+      // size of Counter
+      RamUsageEstimator.shallowSizeOf(Counter.class);
 
   public static final int DEFAULT_CAPACITY = 16;
 
@@ -472,6 +477,15 @@ public final class BytesRefHash {
     return bytesStart[bytesID];
   }
 
+  @Override
+  public long ramBytesUsed() {
+    long size = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(bytesStart) +
+        RamUsageEstimator.sizeOfObject(ids) +
+        RamUsageEstimator.sizeOfObject(pool);
+    return size;
+  }
+
   /**
    * Thrown if a {@link BytesRef} exceeds the {@link BytesRefHash} limit of
    * {@link ByteBlockPool#BYTE_BLOCK_SIZE}-2.
diff --git a/lucene/core/src/java/org/apache/lucene/util/IntsRef.java b/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
index 0c6cfa5..aacc102 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
@@ -116,7 +116,7 @@ public final class IntsRef implements Comparable<IntsRef>, Cloneable {
     sb.append(']');
     return sb.toString();
   }
-  
+
   /**
    * Creates a new IntsRef that points to a copy of the ints from 
    * <code>other</code>
diff --git a/lucene/core/src/java/org/apache/lucene/util/LongBitSet.java b/lucene/core/src/java/org/apache/lucene/util/LongBitSet.java
index 89b4765..c1ddc96 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LongBitSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LongBitSet.java
@@ -26,7 +26,8 @@ import java.util.Arrays;
  * 
  * @lucene.internal
  */
-public final class LongBitSet {
+public final class LongBitSet implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(LongBitSet.class);
 
   private final long[] bits; // Array of longs holding the bits 
   private final long numBits; // The number of bits in use
@@ -428,4 +429,10 @@ public final class LongBitSet {
     // empty sets from returning 0, which is too common.
     return (int) ((h>>32) ^ h) + 0x98761234;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(bits);
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java b/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
index 0d72e5e..d2a043a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
+++ b/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
@@ -25,10 +25,17 @@ import java.security.AccessController;
 import java.security.PrivilegedAction;
 import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.Locale;
 import java.util.Map;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
 /**
  * Estimates the size (memory representation) of Java objects.
  * <p>
@@ -82,25 +89,41 @@ public final class RamUsageEstimator {
   public final static int NUM_BYTES_OBJECT_ALIGNMENT;
 
   /**
+   * Approximate memory usage that we assign to all unknown queries -
+   * this maps roughly to a BooleanQuery with a couple term clauses.
+   */
+  public static final int QUERY_DEFAULT_RAM_BYTES_USED = 1024;
+
+  /**
+   * Approximate memory usage that we assign to all unknown objects -
+   * this maps roughly to a few primitive fields and a couple short String-s.
+   */
+  public static final int UNKNOWN_DEFAULT_RAM_BYTES_USED = 256;
+
+  /**
    * Sizes of primitive classes.
    */
-  private static final Map<Class<?>,Integer> primitiveSizes = new IdentityHashMap<>();
+  public static final Map<Class<?>,Integer> primitiveSizes;
+
   static {
-    primitiveSizes.put(boolean.class, 1);
-    primitiveSizes.put(byte.class, 1);
-    primitiveSizes.put(char.class, Integer.valueOf(Character.BYTES));
-    primitiveSizes.put(short.class, Integer.valueOf(Short.BYTES));
-    primitiveSizes.put(int.class, Integer.valueOf(Integer.BYTES));
-    primitiveSizes.put(float.class, Integer.valueOf(Float.BYTES));
-    primitiveSizes.put(double.class, Integer.valueOf(Double.BYTES));
-    primitiveSizes.put(long.class, Integer.valueOf(Long.BYTES));
+    Map<Class<?>, Integer> primitiveSizesMap = new IdentityHashMap<>();
+    primitiveSizesMap.put(boolean.class, 1);
+    primitiveSizesMap.put(byte.class, 1);
+    primitiveSizesMap.put(char.class, Integer.valueOf(Character.BYTES));
+    primitiveSizesMap.put(short.class, Integer.valueOf(Short.BYTES));
+    primitiveSizesMap.put(int.class, Integer.valueOf(Integer.BYTES));
+    primitiveSizesMap.put(float.class, Integer.valueOf(Float.BYTES));
+    primitiveSizesMap.put(double.class, Integer.valueOf(Double.BYTES));
+    primitiveSizesMap.put(long.class, Integer.valueOf(Long.BYTES));
+
+    primitiveSizes = Collections.unmodifiableMap(primitiveSizesMap);
   }
 
   /**
    * JVMs typically cache small longs. This tries to find out what the range is.
    */
   static final long LONG_CACHE_MIN_VALUE, LONG_CACHE_MAX_VALUE;
-  static final int LONG_SIZE;
+  static final int LONG_SIZE, STRING_SIZE;
   
   /** For testing only */
   static final boolean JVM_IS_HOTSPOT_64BIT;
@@ -181,8 +204,9 @@ public final class RamUsageEstimator {
     LONG_CACHE_MIN_VALUE = longCacheMinValue;
     LONG_CACHE_MAX_VALUE = longCacheMaxValue;
     LONG_SIZE = (int) shallowSizeOfInstance(Long.class);
+    STRING_SIZE = (int) shallowSizeOfInstance(String.class);
   }
-  
+
   /** 
    * Aligns an object size to be the next multiple of {@link #NUM_BYTES_OBJECT_ALIGNMENT}. 
    */
@@ -242,6 +266,234 @@ public final class RamUsageEstimator {
     return alignObjectSize((long) NUM_BYTES_ARRAY_HEADER + (long) Double.BYTES * arr.length);
   }
 
+  /** Returns the size in bytes of the String[] object. */
+  public static long sizeOf(String[] arr) {
+    long size = shallowSizeOf(arr);
+    for (String s : arr) {
+      if (s == null) {
+        continue;
+      }
+      size += sizeOf(s);
+    }
+    return size;
+  }
+
+  /** Recurse only into immediate descendants. */
+  public static final int MAX_DEPTH = 1;
+
+  /** Returns the size in bytes of a Map object, including sizes of its keys and values, supplying
+   * {@link #UNKNOWN_DEFAULT_RAM_BYTES_USED} when object type is not well known.
+   * This method recurses up to {@link #MAX_DEPTH}.
+   */
+  public static long sizeOfMap(Map<?, ?> map) {
+    return sizeOfMap(map, 0, UNKNOWN_DEFAULT_RAM_BYTES_USED);
+  }
+
+  /** Returns the size in bytes of a Map object, including sizes of its keys and values, supplying
+   * default object size when object type is not well known.
+   * This method recurses up to {@link #MAX_DEPTH}.
+   */
+  public static long sizeOfMap(Map<?, ?> map, long defSize) {
+    return sizeOfMap(map, 0, defSize);
+  }
+
+  private static long sizeOfMap(Map<?, ?> map, int depth, long defSize) {
+    if (map == null) {
+      return 0;
+    }
+    long size = shallowSizeOf(map);
+    if (depth > MAX_DEPTH) {
+      return size;
+    }
+    long sizeOfEntry = -1;
+    for (Map.Entry<?, ?> entry : map.entrySet()) {
+      if (sizeOfEntry == -1) {
+        sizeOfEntry = shallowSizeOf(entry);
+      }
+      size += sizeOfEntry;
+      size += sizeOfObject(entry.getKey(), depth, defSize);
+      size += sizeOfObject(entry.getValue(), depth, defSize);
+    }
+    return alignObjectSize(size);
+  }
+
+  /** Returns the size in bytes of a Collection object, including sizes of its values, supplying
+   * {@link #UNKNOWN_DEFAULT_RAM_BYTES_USED} when object type is not well known.
+   * This method recurses up to {@link #MAX_DEPTH}.
+   */
+  public static long sizeOfCollection(Collection<?> collection) {
+    return sizeOfCollection(collection, 0, UNKNOWN_DEFAULT_RAM_BYTES_USED);
+  }
+
+  /** Returns the size in bytes of a Collection object, including sizes of its values, supplying
+   * default object size when object type is not well known.
+   * This method recurses up to {@link #MAX_DEPTH}.
+   */
+  public static long sizeOfCollection(Collection<?> collection, long defSize) {
+    return sizeOfCollection(collection, 0, defSize);
+  }
+
+  private static long sizeOfCollection(Collection<?> collection, int depth, long defSize) {
+    if (collection == null) {
+      return 0;
+    }
+    long size = shallowSizeOf(collection);
+    if (depth > MAX_DEPTH) {
+      return size;
+    }
+    // assume array-backed collection and add per-object references
+    size += NUM_BYTES_ARRAY_HEADER * 2;
+    for (Object o : collection) {
+      size += sizeOfObject(o, depth, defSize);
+    }
+    return alignObjectSize(size);
+  }
+
+  private static final class RamUsageQueryVisitor extends QueryVisitor {
+    long total = 0;
+    long defSize;
+
+    RamUsageQueryVisitor(long defSize) {
+      this.defSize = defSize;
+    }
+
+    @Override
+    public void consumeTerms(Query query, Term... terms) {
+      if (defSize > 0) {
+        total += defSize;
+      } else {
+        total += shallowSizeOf(query);
+      }
+      if (terms != null) {
+        for (Term t : terms) {
+          total += sizeOf(t);
+        }
+      }
+    }
+
+    @Override
+    public void visitLeaf(Query query) {
+      if (query instanceof Accountable) {
+        total += ((Accountable)query).ramBytesUsed();
+      } else {
+        if (defSize > 0) {
+          total += defSize;
+        } else {
+          total += shallowSizeOf(query);
+        }
+      }
+    }
+
+    @Override
+    public QueryVisitor getSubVisitor(BooleanClause.Occur occur, Query parent) {
+      return this;
+    }
+  }
+
+  /**
+   * Returns the size in bytes of a Query object. Unknown query types will be estimated
+   * as {@link #QUERY_DEFAULT_RAM_BYTES_USED}.
+   */
+  public static long sizeOf(Query q) {
+    return sizeOf(q, QUERY_DEFAULT_RAM_BYTES_USED);
+  }
+
+  /**
+   * Returns the size in bytes of a Query object. Unknown query types will be estimated
+   * using {@link #shallowSizeOf(Object)}, or using the supplied <code>defSize</code> parameter
+   * if its value is greater than 0.
+   */
+  public static long sizeOf(Query q, long defSize) {
+    if (q instanceof Accountable) {
+      return ((Accountable)q).ramBytesUsed();
+    } else {
+      RamUsageQueryVisitor visitor = new RamUsageQueryVisitor(defSize);
+      q.visit(visitor);
+      return visitor.total;
+    }
+  }
+
+  /** Best effort attempt to estimate the size in bytes of an undetermined object. Known types
+   * will be estimated according to their formulas, and all other object sizes will be estimated
+   * as {@link #UNKNOWN_DEFAULT_RAM_BYTES_USED}.
+   */
+  public static long sizeOfObject(Object o) {
+    return sizeOfObject(o, 0, UNKNOWN_DEFAULT_RAM_BYTES_USED);
+  }
+
+  /** Best effort attempt to estimate the size in bytes of an undetermined object. Known types
+   * will be estimated according to their formulas, and all other object sizes will be estimated
+   * using {@link #shallowSizeOf(Object)}, or using the supplied <code>defSize</code> parameter if
+   * its value is greater than 0.
+   */
+  public static long sizeOfObject(Object o, long defSize) {
+    return sizeOfObject(o, 0, defSize);
+  }
+
+  private static long sizeOfObject(Object o, int depth, long defSize) {
+    if (o == null) {
+      return 0;
+    }
+    long size;
+    if (o instanceof Accountable) {
+      size = ((Accountable)o).ramBytesUsed();
+    } else if (o instanceof String) {
+      size = sizeOf((String)o);
+    } else if (o instanceof boolean[]) {
+      size = sizeOf((boolean[])o);
+    } else if (o instanceof byte[]) {
+      size = sizeOf((byte[])o);
+    } else if (o instanceof char[]) {
+      size = sizeOf((char[])o);
+    } else if (o instanceof double[]) {
+      size = sizeOf((double[])o);
+    } else if (o instanceof float[]) {
+      size = sizeOf((float[])o);
+    } else if (o instanceof int[]) {
+      size = sizeOf((int[])o);
+    } else if (o instanceof Long) {
+      size = sizeOf((Long)o);
+    } else if (o instanceof long[]) {
+      size = sizeOf((long[])o);
+    } else if (o instanceof short[]) {
+      size = sizeOf((short[])o);
+    } else if (o instanceof String[]) {
+      size = sizeOf((String[]) o);
+    } else if (o instanceof Query) {
+      size = sizeOf((Query)o, defSize);
+    } else if (o instanceof Map) {
+      size = sizeOfMap((Map) o, ++depth, defSize);
+    } else if (o instanceof Collection) {
+      size = sizeOfCollection((Collection)o, ++depth, defSize);
+    } else {
+      if (defSize > 0) {
+        size = defSize;
+      } else {
+        size = shallowSizeOf(o);
+      }
+    }
+    return size;
+  }
+
+  /** Returns the size in bytes of the {@link Accountable} object, using its
+   * {@link Accountable#ramBytesUsed()} method.
+   */
+  public static long sizeOf(Accountable accountable) {
+    return accountable.ramBytesUsed();
+  }
+
+  /** Returns the size in bytes of the String object. */
+  public static long sizeOf(String s) {
+    if (s == null) {
+      return 0;
+    }
+    // may not be true in Java 9+ and CompactStrings - but we have no way to determine this
+
+    // char[] + hashCode
+    long size = STRING_SIZE + (long)NUM_BYTES_ARRAY_HEADER + (long)Character.BYTES * s.length();
+    return alignObjectSize(size);
+  }
+
   /** Returns the shallow size in bytes of the Object[] object. */
   // Use this method instead of #shallowSizeOf(Object) to avoid costly reflection
   public static long shallowSizeOf(Object[] arr) {
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
index 41acad4..e3ad266 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
@@ -24,9 +24,11 @@ import java.util.List;
 import org.apache.lucene.index.SingleTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -37,7 +39,9 @@ import org.apache.lucene.util.UnicodeUtil;
  *
  * @lucene.experimental
  */
-public class CompiledAutomaton {
+public class CompiledAutomaton implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(CompiledAutomaton.class);
+
   /**
    * Automata are compiled into different internal forms for the
    * most efficient execution depending upon the language they accept.
@@ -461,4 +465,15 @@ public class CompiledAutomaton {
 
     return true;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(automaton) +
+        RamUsageEstimator.sizeOfObject(commonSuffixRef) +
+        RamUsageEstimator.sizeOfObject(runAutomaton) +
+        RamUsageEstimator.sizeOfObject(term) +
+        RamUsageEstimator.sizeOfObject(transition);
+  }
+
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
index 4f53926..a1b17de 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
@@ -31,12 +31,17 @@ package org.apache.lucene.util.automaton;
 
 import java.util.Arrays;
 
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.RamUsageEstimator;
+
 /**
  * Finite-state automaton with fast run operation.  The initial state is always 0.
  * 
  * @lucene.experimental
  */
-public abstract class RunAutomaton {
+public abstract class RunAutomaton implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(RunAutomaton.class);
+
   final Automaton automaton;
   final int alphabetSize;
   final int size;
@@ -204,4 +209,14 @@ public abstract class RunAutomaton {
     if (!Arrays.equals(transitions, other.transitions)) return false;
     return true;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(accept) +
+        RamUsageEstimator.sizeOfObject(automaton) +
+        RamUsageEstimator.sizeOfObject(classmap) +
+        RamUsageEstimator.sizeOfObject(points) +
+        RamUsageEstimator.sizeOfObject(transitions);
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java b/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java
index 34128ad..273574f 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java
@@ -20,9 +20,30 @@ package org.apache.lucene.util;
 import static org.apache.lucene.util.RamUsageEstimator.*;
 import static org.apache.lucene.util.RamUsageTester.sizeOf;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Random;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+
 public class TestRamUsageEstimator extends LuceneTestCase {
+
+  static final String[] strings = new String[] {
+      "test string",
+      "hollow",
+      "catchmaster"
+  };
+
   public void testSanity() {
     assertTrue(sizeOf("test string") > shallowSizeOfInstance(String.class));
 
@@ -36,11 +57,6 @@ public class TestRamUsageEstimator extends LuceneTestCase {
     assertTrue(
         shallowSizeOfInstance(Holder.class)         == shallowSizeOfInstance(HolderSubclass2.class));
 
-    String[] strings = new String[] {
-        "test string",
-        "hollow",
-        "catchmaster"
-    };
     assertTrue(sizeOf(strings) > shallowSizeOf(strings));
   }
 
@@ -86,7 +102,73 @@ public class TestRamUsageEstimator extends LuceneTestCase {
       assertEquals(sizeOf(array), sizeOf((Object) array));
     }
   }
-  
+
+  public void testStrings() {
+    long actual = sizeOf(strings);
+    long estimated = RamUsageEstimator.sizeOf(strings);
+    assertEquals(actual, estimated);
+  }
+
+  public void testBytesRefHash() {
+    BytesRefHash bytes = new BytesRefHash();
+    for (int i = 0; i < 100; i++) {
+      bytes.add(new BytesRef("foo bar " + i));
+      bytes.add(new BytesRef("baz bam " + i));
+    }
+    long actual = sizeOf(bytes);
+    long estimated = RamUsageEstimator.sizeOf(bytes);
+    assertEquals(actual, estimated);
+  }
+
+  public void testMap() {
+    Map<String, Object> map = new HashMap<>();
+    map.put("primitive", 1234L);
+    map.put("string", "string");
+    long actual = sizeOf(map);
+    long estimated = RamUsageEstimator.sizeOfObject(map);
+    assertTrue(estimated > actual); // RamUsageTester under-estimates the size of map
+
+    // test recursion
+    map.clear();
+    map.put("string[]", new String[]{"foo", "bar"});
+    map.put("map", Collections.singletonMap("foo", "bar"));
+    map.put("self", map);
+    actual = sizeOf(map);
+    estimated = RamUsageEstimator.sizeOfObject(map);
+    assertTrue(estimated > actual);
+  }
+
+  public void testCollection() {
+    List<Object> list = new ArrayList<>();
+    list.add(1234L);
+    list.add("string");
+    list.add(new Term("foo", "bar"));
+    long actual = sizeOf(list);
+    long estimated = RamUsageEstimator.sizeOfObject(list);
+    assertEquals(actual, estimated);
+
+    // test recursion
+    list.clear();
+    list.add(1234L);
+    list.add(list);
+    actual = sizeOf(list);
+    estimated = RamUsageEstimator.sizeOfObject(list);
+    assertEquals(actual + RamUsageEstimator.shallowSizeOf(list), estimated);
+  }
+
+  public void testQuery() {
+    DisjunctionMaxQuery dismax = new DisjunctionMaxQuery(
+        Arrays.asList(new TermQuery(new Term("foo", "bar")), new TermQuery(new Term("baz", "bam"))), 1.0f);
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("foo", "bar")), BooleanClause.Occur.SHOULD)
+        .add(new FuzzyQuery(new Term("foo", "baz")), BooleanClause.Occur.MUST_NOT)
+        .add(dismax, BooleanClause.Occur.MUST)
+        .build();
+    long actual = sizeOf(bq);
+    long estimated = RamUsageEstimator.sizeOfObject(bq);
+    assertTrue(actual < estimated);
+  }
+
   public void testReferenceSize() {
     assertTrue(NUM_BYTES_OBJECT_REF == 4 || NUM_BYTES_OBJECT_REF == 8);
     if (Constants.JRE_IS_64BIT) {
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
index 6bfdd1b..84e4671 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
@@ -32,11 +32,14 @@ import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.RamUsageEstimator;
 
-final class GlobalOrdinalsQuery extends Query {
+final class GlobalOrdinalsQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(GlobalOrdinalsQuery.class);
 
   // All the ords of matching docs found with OrdinalsCollector.
   private final LongBitSet foundOrds;
@@ -50,6 +53,8 @@ final class GlobalOrdinalsQuery extends Query {
   // id of the context rather than the context itself in order not to hold references to index readers
   private final Object indexReaderContextId;
 
+  private final long ramBytesUsed; // cache
+
   GlobalOrdinalsQuery(LongBitSet foundOrds, String joinField, OrdinalMap globalOrds, Query toQuery,
                       Query fromQuery, Object indexReaderContextId) {
     this.foundOrds = foundOrds;
@@ -58,6 +63,13 @@ final class GlobalOrdinalsQuery extends Query {
     this.toQuery = toQuery;
     this.fromQuery = fromQuery;
     this.indexReaderContextId = indexReaderContextId;
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(this.foundOrds) +
+        RamUsageEstimator.sizeOfObject(this.globalOrds) +
+        RamUsageEstimator.sizeOfObject(this.joinField) +
+        RamUsageEstimator.sizeOfObject(this.fromQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED) +
+        RamUsageEstimator.sizeOfObject(this.toQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED);
   }
 
   @Override
@@ -103,6 +115,11 @@ final class GlobalOrdinalsQuery extends Query {
         '}';
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
   final class W extends ConstantScoreWeight {
 
     private final Weight approximationWeight;
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
index 4cdf59b..cd1f1f0 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
@@ -32,10 +32,13 @@ import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.RamUsageEstimator;
 
-final class GlobalOrdinalsWithScoreQuery extends Query {
+final class GlobalOrdinalsWithScoreQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(GlobalOrdinalsWithScoreQuery.class);
 
   private final GlobalOrdinalsWithScoreCollector collector;
   private final String joinField;
@@ -51,6 +54,8 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
   // id of the context rather than the context itself in order not to hold references to index readers
   private final Object indexReaderContextId;
 
+  private final long ramBytesUsed; // cache
+
   GlobalOrdinalsWithScoreQuery(GlobalOrdinalsWithScoreCollector collector, ScoreMode scoreMode, String joinField,
                                OrdinalMap globalOrds, Query toQuery, Query fromQuery, int min, int max,
                                Object indexReaderContextId) {
@@ -63,6 +68,12 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
     this.min = min;
     this.max = max;
     this.indexReaderContextId = indexReaderContextId;
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(this.fromQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED) +
+        RamUsageEstimator.sizeOfObject(this.globalOrds) +
+        RamUsageEstimator.sizeOfObject(this.joinField) +
+        RamUsageEstimator.sizeOfObject(this.toQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED);
   }
 
   @Override
@@ -124,6 +135,11 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
         '}';
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
   final class W extends FilterWeight {
 
     W(Query query, Weight approximationWeight) {
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
index 01a8221..e0f48b2 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
@@ -45,13 +45,16 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 
 // A TermsIncludingScoreQuery variant for point values:
-abstract class PointInSetIncludingScoreQuery extends Query {
+abstract class PointInSetIncludingScoreQuery extends Query implements Accountable {
+  protected static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(PointInSetIncludingScoreQuery.class);
 
   static BiFunction<byte[], Class<? extends Number>, String> toString = (value, numericType) -> {
     if (Integer.class.equals(numericType)) {
@@ -77,6 +80,8 @@ abstract class PointInSetIncludingScoreQuery extends Query {
 
   final List<Float> aggregatedJoinScores;
 
+  private final long ramBytesUsed; // cache
+
   static abstract class Stream extends PointInSetQuery.Stream {
 
     float score;
@@ -118,6 +123,11 @@ abstract class PointInSetIncludingScoreQuery extends Query {
     }
     sortedPackedPoints = builder.finish();
     sortedPackedPointsHashCode = sortedPackedPoints.hashCode();
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(this.field) +
+        RamUsageEstimator.sizeOfObject(this.originalQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED) +
+        RamUsageEstimator.sizeOfObject(this.sortedPackedPoints);
   }
 
   @Override
@@ -341,4 +351,9 @@ abstract class PointInSetIncludingScoreQuery extends Query {
   }
 
   protected abstract String toString(byte[] value);
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
index 1ae0ef2..099d246 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
@@ -33,12 +33,15 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 
-class TermsIncludingScoreQuery extends Query {
+class TermsIncludingScoreQuery extends Query implements Accountable {
+  protected static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(TermsIncludingScoreQuery.class);
 
   private final ScoreMode scoreMode;
   private final String toField;
@@ -53,6 +56,8 @@ class TermsIncludingScoreQuery extends Query {
   // id of the context rather than the context itself in order not to hold references to index readers
   private final Object topReaderContextId;
 
+  private final long ramBytesUsed; // cache
+
   TermsIncludingScoreQuery(ScoreMode scoreMode, String toField, boolean multipleValuesPerDocument, BytesRefHash terms, float[] scores,
                            String fromField, Query fromQuery, Object indexReaderContextId) {
     this.scoreMode = scoreMode;
@@ -65,6 +70,14 @@ class TermsIncludingScoreQuery extends Query {
     this.fromField = fromField;
     this.fromQuery = fromQuery;
     this.topReaderContextId = indexReaderContextId;
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(fromField) +
+        RamUsageEstimator.sizeOfObject(fromQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED) +
+        RamUsageEstimator.sizeOfObject(ords) +
+        RamUsageEstimator.sizeOfObject(scores) +
+        RamUsageEstimator.sizeOfObject(terms) +
+        RamUsageEstimator.sizeOfObject(toField);
   }
 
   @Override
@@ -99,6 +112,11 @@ class TermsIncludingScoreQuery extends Query {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
+  @Override
   public Weight createWeight(IndexSearcher searcher, org.apache.lucene.search.ScoreMode scoreMode, float boost) throws IOException {
     if (scoreMode.needsScores() == false) {
       // We don't need scores then quickly change the query:
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
index 0f8b3d7..e79365c 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
@@ -24,8 +24,10 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * A query that has an array of terms from a specific field. This query will match documents have one or more terms in
@@ -33,7 +35,8 @@ import org.apache.lucene.util.BytesRefHash;
  *
  * @lucene.experimental
  */
-class TermsQuery extends MultiTermQuery {
+class TermsQuery extends MultiTermQuery implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(TermsQuery.class);
 
   private final BytesRefHash terms;
   private final int[] ords;
@@ -44,6 +47,8 @@ class TermsQuery extends MultiTermQuery {
   // id of the context rather than the context itself in order not to hold references to index readers
   private final Object indexReaderContextId;
 
+  private final long ramBytesUsed; // cache
+
   /**
    * @param toField               The field that should contain terms that are specified in the next parameter.
    * @param terms                 The terms that matching documents should have. The terms must be sorted by natural order.
@@ -56,6 +61,13 @@ class TermsQuery extends MultiTermQuery {
     this.fromField = fromField;
     this.fromQuery = fromQuery;
     this.indexReaderContextId = indexReaderContextId;
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(fromField) +
+        RamUsageEstimator.sizeOfObject(fromQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED) +
+        RamUsageEstimator.sizeOfObject(ords) +
+        RamUsageEstimator.sizeOfObject(terms);
   }
 
   @Override
@@ -102,4 +114,8 @@ class TermsQuery extends MultiTermQuery {
     return classHash() + Objects.hash(field, fromField, fromQuery, indexReaderContextId);
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/BM25FQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/BM25FQuery.java
index b7c7a28..86757da 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/BM25FQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/BM25FQuery.java
@@ -37,7 +37,9 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.SimilarityBase;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * A {@link Query} that treats multiple fields as a single stream and scores
@@ -53,7 +55,8 @@ import org.apache.lucene.util.BytesRef;
  *
  * @lucene.experimental
  */
-public final class BM25FQuery extends Query {
+public final class BM25FQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(BM25FQuery.class);
 
   /**
    * A builder for {@link BM25FQuery}.
@@ -143,6 +146,8 @@ public final class BM25FQuery extends Query {
   // array of terms per field, sorted
   private final Term fieldTerms[];
 
+  private final long ramBytesUsed;
+
   private BM25FQuery(BM25Similarity similarity, TreeMap<String, FieldAndWeight> fieldAndWeights, BytesRef[] terms) {
     this.similarity = similarity;
     this.fieldAndWeights = fieldAndWeights;
@@ -159,6 +164,11 @@ public final class BM25FQuery extends Query {
         fieldTerms[pos++] = new Term(field, term);
       }
     }
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(fieldAndWeights) +
+        RamUsageEstimator.sizeOfObject(fieldTerms) +
+        RamUsageEstimator.sizeOfObject(terms);
   }
 
   public List<Term> getTerms() {
@@ -203,6 +213,11 @@ public final class BM25FQuery extends Query {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
+  @Override
   public Query rewrite(IndexReader reader) throws IOException {
     // optimize zero and single field cases
     if (terms.length == 0) {
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
index 84a2753..4d0a1fd 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
@@ -27,17 +27,21 @@ import java.util.stream.Collectors;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** A {@link Query} that allows to have a configurable number or required
  *  matches per document. This is typically useful in order to build queries
  *  whose query terms must all appear in documents.
  *  @lucene.experimental
  */
-public final class CoveringQuery extends Query {
+public final class CoveringQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(CoveringQuery.class);
 
   private final Collection<Query> queries;
   private final LongValuesSource minimumNumberMatch;
   private final int hashCode;
+  private final long ramBytesUsed;
 
   /**
    * Sole constructor.
@@ -60,6 +64,9 @@ public final class CoveringQuery extends Query {
     this.queries.addAll(queries);
     this.minimumNumberMatch = Objects.requireNonNull(minimumNumberMatch);
     this.hashCode = computeHashCode();
+
+    this.ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(this.queries, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED);
   }
 
   @Override
@@ -95,6 +102,11 @@ public final class CoveringQuery extends Query {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
+  @Override
   public Query rewrite(IndexReader reader) throws IOException {
     Multiset<Query> rewritten = new Multiset<>();
     boolean actuallyRewritten = false;
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
index cdfe23d..35c07be 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
@@ -28,6 +28,8 @@ import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Like {@link DocValuesTermsQuery}, but this query only
@@ -43,7 +45,8 @@ import org.apache.lucene.index.SortedNumericDocValues;
  *
  * @lucene.experimental
  */
-public class DocValuesNumbersQuery extends Query {
+public class DocValuesNumbersQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(DocValuesNumbersQuery.class);
 
   private final String field;
   private final LongHashSet numbers;
@@ -103,6 +106,13 @@ public class DocValuesNumbersQuery extends Query {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(numbers);
+  }
+
+  @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
     return new ConstantScoreWeight(this, boost) {
 
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index ef67b02..111148f 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -29,10 +29,12 @@ import org.apache.lucene.index.PrefixCodedTerms;
 import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LongBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * A {@link Query} that only accepts documents whose
@@ -91,7 +93,8 @@ import org.apache.lucene.util.LongBitSet;
  *
  * @lucene.experimental
  */
-public class DocValuesTermsQuery extends Query {
+public class DocValuesTermsQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(DocValuesTermsQuery.class);
 
   private final String field;
   private final PrefixCodedTerms termData;
@@ -165,6 +168,13 @@ public class DocValuesTermsQuery extends Query {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(termData);
+  }
+
+  @Override
   public void visit(QueryVisitor visitor) {
     if (visitor.acceptField(field)) {
       visitor.visitLeaf(this);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
index 3a6af5f..c0c241c 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
@@ -21,9 +21,12 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
 
-final class LongHashSet extends AbstractSet<Long> {
+final class LongHashSet extends AbstractSet<Long> implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(LongHashSet.class);
 
   private static final long MISSING = Long.MIN_VALUE;
 
@@ -115,6 +118,12 @@ final class LongHashSet extends AbstractSet<Long> {
   }
 
   @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(table);
+  }
+
+  @Override
   public boolean contains(Object o) {
     return o instanceof Long && contains(((Long) o).longValue());
   }
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
index 2e99a06..adff36a 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
@@ -34,8 +34,10 @@ import org.apache.lucene.index.TermStates;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
@@ -67,7 +69,9 @@ import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZ
  *
  *  @lucene.experimental */
 
-public class TermAutomatonQuery extends Query {
+public class TermAutomatonQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(TermAutomatonQuery.class);
+
   private final String field;
   private final Automaton.Builder builder;
   Automaton det;
@@ -265,6 +269,16 @@ public class TermAutomatonQuery extends Query {
     return System.identityHashCode(this);
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(builder) +
+        RamUsageEstimator.sizeOfObject(det) +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(idToTerm) +
+        RamUsageEstimator.sizeOfObject(termToID);
+  }
+
   /** Returns the dot (graphviz) representation of this automaton.
    *  This is extremely useful for visualizing the automaton. */
   public String toDot() {
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
index 6b0f788..f5ccaed 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
@@ -33,7 +33,9 @@ import org.apache.lucene.spatial3d.geom.BasePlanetObject;
 import org.apache.lucene.spatial3d.geom.GeoShape;
 import org.apache.lucene.spatial3d.geom.PlanetModel;
 import org.apache.lucene.spatial3d.geom.XYZBounds;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /** Finds all previously indexed points that fall within the specified polygon.
  *
@@ -41,7 +43,9 @@ import org.apache.lucene.util.DocIdSetBuilder;
  *
  * @lucene.experimental */
 
-final class PointInGeo3DShapeQuery extends Query {
+final class PointInGeo3DShapeQuery extends Query implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(PointInGeo3DShapeQuery.class);
+
   final String field;
   final GeoShape shape;
   final XYZBounds shapeBounds;
@@ -162,4 +166,12 @@ final class PointInGeo3DShapeQuery extends Query {
     sb.append(shape);
     return sb.toString();
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(field) +
+        RamUsageEstimator.sizeOfObject(shape) +
+        RamUsageEstimator.sizeOfObject(shapeBounds);
+  }
 }
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
index bb2131d..f5062d2 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
@@ -27,10 +27,12 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Operations;
@@ -76,13 +78,17 @@ import org.apache.lucene.util.fst.Util;
  *
  * @lucene.experimental
  */
-public class ContextQuery extends CompletionQuery {
+public class ContextQuery extends CompletionQuery implements Accountable {
+  private static final long BASE_RAM_BYTES = RamUsageEstimator.shallowSizeOfInstance(ContextQuery.class);
+
   private IntsRefBuilder scratch = new IntsRefBuilder();
   private Map<IntsRef, ContextMetaData> contexts;
   private boolean matchAllContexts = false;
   /** Inner completion query */
   protected CompletionQuery innerQuery;
 
+  private long ramBytesUsed;
+
   /**
    * Constructs a context completion query that matches
    * documents specified by <code>query</code>.
@@ -98,6 +104,13 @@ public class ContextQuery extends CompletionQuery {
     }
     this.innerQuery = query;
     contexts = new HashMap<>();
+    updateRamBytesUsed();
+  }
+
+  private void updateRamBytesUsed() {
+    ramBytesUsed = BASE_RAM_BYTES +
+        RamUsageEstimator.sizeOfObject(contexts) +
+        RamUsageEstimator.sizeOfObject(innerQuery, RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_USED);
   }
 
   /**
@@ -129,6 +142,7 @@ public class ContextQuery extends CompletionQuery {
       }
     }
     contexts.put(IntsRef.deepCopyOf(Util.toIntsRef(new BytesRef(context), scratch)), new ContextMetaData(boost, exact));
+    updateRamBytesUsed();
   }
 
   /**
@@ -342,4 +356,8 @@ public class ContextQuery extends CompletionQuery {
     visitor.visitLeaf(this);
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }