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

[lucene-solr] branch master updated: LUCENE-8642: RamUsageTester.sizeOf ignores arrays and collections if --illegal-access=deny.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new efef89a  LUCENE-8642: RamUsageTester.sizeOf ignores arrays and collections if --illegal-access=deny.
efef89a is described below

commit efef89adc6147c91b2b108b55f53a85a45762c1d
Author: Dawid Weiss <dw...@apache.org>
AuthorDate: Fri Jan 18 11:55:53 2019 +0100

    LUCENE-8642: RamUsageTester.sizeOf ignores arrays and collections if --illegal-access=deny.
---
 .../apache/lucene/search/TermInSetQueryTest.java   |   1 -
 .../lucene/index/BaseIndexFileFormatTestCase.java  |  27 ++-
 .../org/apache/lucene/util/RamUsageTester.java     | 182 ++++++++++++---------
 3 files changed, 122 insertions(+), 88 deletions(-)

diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index aba8f23..caabfef 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -173,7 +173,6 @@ public class TermInSetQueryTest extends LuceneTestCase {
     QueryUtils.checkEqual(query1, query2);
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-8641")
   public void testRamBytesUsed() {
     List<BytesRef> terms = new ArrayList<>();
     final int numTerms = 10000 + random().nextInt(1000);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index 331ec96..780cfa0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
@@ -289,12 +290,26 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
       new SimpleMergedSegmentWarmer(InfoStream.NO_OUTPUT).warm(reader);
     }
 
-    final long actualBytes = RamUsageTester.sizeOf(reader2, new Accumulator(reader2)) - RamUsageTester.sizeOf(reader1, new Accumulator(reader1));
-    final long expectedBytes = ((SegmentReader) reader2).ramBytesUsed() - ((SegmentReader) reader1).ramBytesUsed();
-    final long absoluteError = actualBytes - expectedBytes;
-    final double relativeError = (double) absoluteError / actualBytes;
-    final String message = "Actual RAM usage " + actualBytes + ", but got " + expectedBytes + ", " + 100*relativeError + "% error";
-    assertTrue(message, Math.abs(relativeError) < 0.20d || Math.abs(absoluteError) < 1000);
+    long act1 = RamUsageTester.sizeOf(reader2, new Accumulator(reader2));
+    long act2 = RamUsageTester.sizeOf(reader1, new Accumulator(reader1));
+    final long measuredBytes = act1 - act2;
+
+    long reported1 = ((SegmentReader) reader2).ramBytesUsed();
+    long reported2 = ((SegmentReader) reader1).ramBytesUsed();
+    final long reportedBytes = reported1 - reported2;
+
+    final long absoluteError = Math.abs(measuredBytes - reportedBytes);
+    final double relativeError = (double) absoluteError / measuredBytes;
+    final String message = String.format(Locale.ROOT,
+        "RamUsageTester reports %d bytes but ramBytesUsed() returned %d (%.1f error). " +
+        " [Measured: %d, %d. Reported: %d, %d]",
+        measuredBytes,
+        reportedBytes,
+        (100 * relativeError),
+        act1, act2,
+        reported1, reported2);
+
+    assertTrue(message, relativeError < 0.20d || absoluteError < 1000);
 
     reader1.close();
     reader2.close();
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index 9b7b666..fdd7e2e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -32,8 +32,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Predicate;
 import java.util.function.ToLongFunction;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -112,80 +114,18 @@ public final class RamUsageTester {
       }
       seen.add(ob);
 
+      final long obSize;
       final Class<?> obClazz = ob.getClass();
       assert obClazz != null : "jvm bug detected (Object.getClass() == null). please report this to your vendor";
       if (obClazz.isArray()) {
-        /*
-         * Consider an array, possibly of primitive types. Push any of its references to
-         * the processing stack and accumulate this array's shallow size. 
-         */
-        final long shallowSize = RamUsageEstimator.shallowSizeOf(ob);
-        final int len = Array.getLength(ob);
-        final List<Object> values;
-        Class<?> componentClazz = obClazz.getComponentType();
-        if (componentClazz.isPrimitive()) {
-          values = Collections.emptyList();
-        } else {
-          values = new AbstractList<Object>() {
-
-            @Override
-            public Object get(int index) {
-              return Array.get(ob, index);
-            }
-
-            @Override
-            public int size() {
-              return len;
-            }
-            
-          };
-        }
-        totalSize += accumulator.accumulateArray(ob, shallowSize, values, stack);
+        obSize = handleArray(accumulator, stack, ob, obClazz);
       } else {
-        /*
-         * Consider an object. Push any references it has to the processing stack
-         * and accumulate this object's shallow size. 
-         */
-        try {
-          ClassCache cachedInfo = classCache.get(obClazz);
-          if (cachedInfo == null) {
-            classCache.put(obClazz, cachedInfo = createCacheEntry(obClazz));
-          }
-          
-          boolean needsReflection = true;
-          if (Constants.JRE_IS_MINIMUM_JAVA9 && obClazz.getName().startsWith("java.")) {
-            // Java 9: Best guess for some known types, as we cannot precisely look into runtime classes:
-            final ToLongFunction<Object> func = SIMPLE_TYPES.get(obClazz);
-            if (func != null) { // some simple type like String where the size is easy to get from public properties
-              totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize + func.applyAsLong(ob), 
-                  Collections.emptyMap(), stack);
-              needsReflection = false;
-            } else if (ob instanceof Iterable) {
-              final List<Object> values = StreamSupport.stream(((Iterable<?>) ob).spliterator(), false)
-                  .collect(Collectors.toList());
-              totalSize += accumulator.accumulateArray(ob, cachedInfo.alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack);
-              needsReflection = false;
-            }  else if (ob instanceof Map) {
-              final List<Object> values = ((Map<?,?>) ob).entrySet().stream()
-                  .flatMap(e -> Stream.of(e.getKey(), e.getValue()))
-                  .collect(Collectors.toList());
-              totalSize += accumulator.accumulateArray(ob, cachedInfo.alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack);
-              totalSize += RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
-              needsReflection = false;
-            }
-          }
-          if (needsReflection) {
-            final Map<Field, Object> fieldValues = new HashMap<>();
-            for (Field f : cachedInfo.referenceFields) {
-              fieldValues.put(f, f.get(ob));
-            }
-            totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
-          }
-        } catch (IllegalAccessException e) {
-          // this should never happen as we enabled setAccessible().
-          throw new RuntimeException("Reflective field access failed?", e);
-        }
+        obSize = handleOther(accumulator, classCache, stack, ob, obClazz);
       }
+
+      totalSize += obSize;
+      // Dump size of each object for comparisons across JVMs and flags.
+      // System.out.println("  += " + obClazz + " | " + obSize);
     }
 
     // Help the GC (?).
@@ -195,7 +135,91 @@ public final class RamUsageTester {
 
     return totalSize;
   }
-  
+
+  private static long handleOther(Accumulator accumulator, IdentityHashMap<Class<?>, ClassCache> classCache, ArrayList<Object> stack, Object ob, Class<?> obClazz) {
+    /*
+     * Consider an object. Push any references it has to the processing stack
+     * and accumulate this object's shallow size.
+     */
+    try {
+      if (Constants.JRE_IS_MINIMUM_JAVA9) {
+        long alignedShallowInstanceSize = RamUsageEstimator.shallowSizeOf(ob);
+
+        Predicate<Class<?>> isJavaModule = (clazz) -> {
+          return clazz.getName().startsWith("java.");
+        };
+
+        // Java 9: Best guess for some known types, as we cannot precisely look into runtime classes:
+        final ToLongFunction<Object> func = SIMPLE_TYPES.get(obClazz);
+        if (func != null) { // some simple type like String where the size is easy to get from public properties
+          return accumulator.accumulateObject(ob, alignedShallowInstanceSize + func.applyAsLong(ob),
+              Collections.emptyMap(), stack);
+        } else if (ob instanceof Enum) {
+          return alignedShallowInstanceSize;
+        } else if (ob instanceof ByteBuffer) {
+          // Approximate ByteBuffers with their underlying storage (ignores field overhead).
+          return byteArraySize(((ByteBuffer) ob).capacity());
+        }  else if (isJavaModule.test(obClazz) && ob instanceof Map) {
+          final List<Object> values = ((Map<?,?>) ob).entrySet().stream()
+              .flatMap(e -> Stream.of(e.getKey(), e.getValue()))
+              .collect(Collectors.toList());
+          return accumulator.accumulateArray(ob, alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack)
+              + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
+        } else if (isJavaModule.test(obClazz) && ob instanceof Iterable) {
+          final List<Object> values = StreamSupport.stream(((Iterable<?>) ob).spliterator(), false)
+            .collect(Collectors.toList());
+          return accumulator.accumulateArray(ob, alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack)
+              + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
+        } else {
+          // Fallback to reflective access.
+        }
+      }
+
+      ClassCache cachedInfo = classCache.get(obClazz);
+      if (cachedInfo == null) {
+        classCache.put(obClazz, cachedInfo = createCacheEntry(obClazz));
+      }
+
+      final Map<Field, Object> fieldValues = new HashMap<>();
+      for (Field f : cachedInfo.referenceFields) {
+        fieldValues.put(f, f.get(ob));
+      }
+      return accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
+    } catch (IllegalAccessException e) {
+      // this should never happen as we enabled setAccessible().
+      throw new RuntimeException("Reflective field access failed?", e);
+    }
+  }
+
+  private static long handleArray(Accumulator accumulator, ArrayList<Object> stack, Object ob, Class<?> obClazz) {
+    /*
+     * Consider an array, possibly of primitive types. Push any of its references to
+     * the processing stack and accumulate this array's shallow size.
+     */
+    final long shallowSize = RamUsageEstimator.shallowSizeOf(ob);
+    final int len = Array.getLength(ob);
+    final List<Object> values;
+    Class<?> componentClazz = obClazz.getComponentType();
+    if (componentClazz.isPrimitive()) {
+      values = Collections.emptyList();
+    } else {
+      values = new AbstractList<Object>() {
+
+        @Override
+        public Object get(int index) {
+          return Array.get(ob, index);
+        }
+
+        @Override
+        public int size() {
+          return len;
+        }
+
+      };
+    }
+    return accumulator.accumulateArray(ob, shallowSize, values, stack);
+  }
+
   /**
    * This map contains a function to calculate sizes of some "simple types" like String just from their public properties.
    * This is needed for Java 9, which does not allow to look into runtime class fields.
@@ -212,8 +236,6 @@ public final class RamUsageTester {
       a(StringBuffer.class, v -> charArraySize(v.capacity()));
       // Types with large buffers:
       a(ByteArrayOutputStream.class, v -> byteArraySize(v.size()));
-      // Approximate ByteBuffers with their underling storage (ignores field overhead).
-      a(ByteBuffer.class, v -> byteArraySize(v.capacity()));
       // For File and Path, we just take the length of String representation as approximation:
       a(File.class, v -> charArraySize(v.toString().length()));
       a(Path.class, v -> charArraySize(v.toString().length()));
@@ -227,10 +249,6 @@ public final class RamUsageTester {
     private long charArraySize(int len) {
       return RamUsageEstimator.alignObjectSize((long)RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long)Character.BYTES * len);
     }
-    
-    private long byteArraySize(int len) {
-      return RamUsageEstimator.alignObjectSize((long)RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + len);
-    }
   });
   
   /**
@@ -271,11 +289,10 @@ public final class RamUsageTester {
                 f.setAccessible(true);
                 referenceFields.add(f);
               } catch (RuntimeException re) {
-                if ("java.lang.reflect.InaccessibleObjectException".equals(re.getClass().getName())) {
-                  // LUCENE-7595: this is Java 9, which prevents access to fields in foreign modules
-                } else {
-                  throw re;
-                }
+                throw new RuntimeException(String.format(Locale.ROOT,
+                    "Can't access field '%s' of class '%s' for RAM estimation.",
+                    f.getName(),
+                    clazz.getName()), re);
               }
             }
           }
@@ -289,4 +306,7 @@ public final class RamUsageTester {
     });
   }
 
+  private static long byteArraySize(int len) {
+    return RamUsageEstimator.alignObjectSize((long) RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + len);
+  }
 }