You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2022/07/01 14:25:52 UTC

[lucene] branch branch_9x updated: Remove/deprecate obsolete constants in oal.util.Constants; remove code which is no longer executed after Java 9 (#978)

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

uschindler pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 1fa985438a5 Remove/deprecate obsolete constants in oal.util.Constants; remove code which is no longer executed after Java 9 (#978)
1fa985438a5 is described below

commit 1fa985438a5519664f731d7bb5e03cd4a4c37a0f
Author: Uwe Schindler <us...@apache.org>
AuthorDate: Fri Jul 1 16:24:18 2022 +0200

    Remove/deprecate obsolete constants in oal.util.Constants; remove code which is no longer executed after Java 9 (#978)
---
 lucene/CHANGES.txt                                 |  9 +++
 .../java/org/apache/lucene/index/IndexWriter.java  | 11 +--
 .../src/java/org/apache/lucene/util/Constants.java | 74 +++++++++++++------
 .../apache/lucene/search/TestLRUQueryCache.java    | 15 +---
 .../lucene/tests/index/BaseTestCheckIndex.java     |  1 -
 .../apache/lucene/tests/util/LuceneTestCase.java   | 40 -----------
 .../apache/lucene/tests/util/RamUsageTester.java   | 82 ++++++++++------------
 7 files changed, 105 insertions(+), 127 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a550a262638..2ebbdab37e5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -73,6 +73,9 @@ API Changes
 * LUCENE-10603: SortedSetDocValues#NO_MORE_ORDS marked @deprecated in favor of iterating with
   SortedSetDocValues#docValueCount(). (Greg Miller)
 
+* GITHUB#978: Deprecate (remove in Lucene 10) obsolete constants in oal.util.Constants; remove
+  code which is no longer executed after Java 9. (Uwe Schindler)
+
 New Features
 ---------------------
 * LUCENE-10550: Add getAllChildren functionality to facets (Yuting Gan)
@@ -110,6 +113,12 @@ Optimizations
 * GITHUB#984: Use primitive type data structures in FloatTaxonomyFacets and IntTaxonomyFacets
   #getAllChildren() internal implementation to avoid some garbage creation. (Greg Miller)
 
+Changes in runtime behavior
+---------------------
+
+* GITHUB#978: IndexWriter diagnostics written to index only contain java's runtime version
+  and vendor. (Uwe Schindler)
+
 Bug Fixes
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 1441adb5892..ecc2421ea87 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -21,12 +21,12 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.time.Instant;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Date;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -4776,14 +4776,9 @@ public class IndexWriter
     diagnostics.put("os", Constants.OS_NAME);
     diagnostics.put("os.arch", Constants.OS_ARCH);
     diagnostics.put("os.version", Constants.OS_VERSION);
-    diagnostics.put("java.version", Constants.JAVA_VERSION);
+    diagnostics.put("java.runtime.version", Runtime.version().toString());
     diagnostics.put("java.vendor", Constants.JAVA_VENDOR);
-    // On IBM J9 JVM this is better than java.version which is just 1.7.0 (no update level):
-    diagnostics.put(
-        "java.runtime.version", System.getProperty("java.runtime.version", "undefined"));
-    // Hotspot version, e.g. 2.8 for J9:
-    diagnostics.put("java.vm.version", System.getProperty("java.vm.version", "undefined"));
-    diagnostics.put("timestamp", Long.toString(new Date().getTime()));
+    diagnostics.put("timestamp", Long.toString(Instant.now().toEpochMilli()));
     if (details != null) {
       diagnostics.putAll(details);
     }
diff --git a/lucene/core/src/java/org/apache/lucene/util/Constants.java b/lucene/core/src/java/org/apache/lucene/util/Constants.java
index 74b4c0d4c45..e2014ce91e4 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Constants.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Constants.java
@@ -16,8 +16,6 @@
  */
 package org.apache.lucene.util;
 
-import java.util.StringTokenizer;
-
 /** Some useful constants. */
 public final class Constants {
   private Constants() {} // can't construct
@@ -25,12 +23,32 @@ public final class Constants {
   /** JVM vendor info. */
   public static final String JVM_VENDOR = System.getProperty("java.vm.vendor");
 
-  public static final String JVM_VERSION = System.getProperty("java.vm.version");
+  /** JVM vendor name. */
   public static final String JVM_NAME = System.getProperty("java.vm.name");
-  public static final String JVM_SPEC_VERSION = System.getProperty("java.specification.version");
 
-  /** The value of <code>System.getProperty("java.version")</code>. * */
-  public static final String JAVA_VERSION = System.getProperty("java.version");
+  /**
+   * Get the full version string of the current runtime.
+   *
+   * @deprecated To detect Java versions use {@link Runtime#version()}
+   */
+  @Deprecated public static final String JVM_VERSION = Runtime.version().toString();
+
+  /**
+   * Gets the specification version of the current runtime. This is the feature version converted to
+   * String.
+   *
+   * @see java.lang.Runtime.Version#feature()
+   * @deprecated To detect Java versions use {@link Runtime#version()}
+   */
+  @Deprecated
+  public static final String JVM_SPEC_VERSION = Integer.toString(Runtime.version().feature());
+
+  /**
+   * The value of <code>System.getProperty("java.version")</code>.
+   *
+   * @deprecated To detect Java versions use {@link Runtime#version()}
+   */
+  @Deprecated public static final String JAVA_VERSION = System.getProperty("java.version");
 
   /** The value of <code>System.getProperty("os.name")</code>. * */
   public static final String OS_NAME = System.getProperty("os.name");
@@ -45,24 +63,19 @@ public final class Constants {
   /** True iff running on FreeBSD */
   public static final boolean FREE_BSD = OS_NAME.startsWith("FreeBSD");
 
+  /** The value of <code>System.getProperty("os.arch")</code>. */
   public static final String OS_ARCH = System.getProperty("os.arch");
+
+  /** The value of <code>System.getProperty("os.version")</code>. */
   public static final String OS_VERSION = System.getProperty("os.version");
-  public static final String JAVA_VENDOR = System.getProperty("java.vendor");
 
-  private static final int JVM_MAJOR_VERSION;
-  private static final int JVM_MINOR_VERSION;
+  /** The value of <code>System.getProperty("java.vendor")</code>. */
+  public static final String JAVA_VENDOR = System.getProperty("java.vendor");
 
   /** True iff running on a 64bit JVM */
   public static final boolean JRE_IS_64BIT;
 
   static {
-    final StringTokenizer st = new StringTokenizer(JVM_SPEC_VERSION, ".");
-    JVM_MAJOR_VERSION = Integer.parseInt(st.nextToken());
-    if (st.hasMoreTokens()) {
-      JVM_MINOR_VERSION = Integer.parseInt(st.nextToken());
-    } else {
-      JVM_MINOR_VERSION = 0;
-    }
     boolean is64Bit = false;
     String datamodel = null;
     try {
@@ -84,10 +97,27 @@ public final class Constants {
     JRE_IS_64BIT = is64Bit;
   }
 
-  public static final boolean JRE_IS_MINIMUM_JAVA8 =
-      JVM_MAJOR_VERSION > 1 || (JVM_MAJOR_VERSION == 1 && JVM_MINOR_VERSION >= 8);
-  public static final boolean JRE_IS_MINIMUM_JAVA9 =
-      JVM_MAJOR_VERSION > 1 || (JVM_MAJOR_VERSION == 1 && JVM_MINOR_VERSION >= 9);
-  public static final boolean JRE_IS_MINIMUM_JAVA11 =
-      JVM_MAJOR_VERSION > 1 || (JVM_MAJOR_VERSION == 1 && JVM_MINOR_VERSION >= 11);
+  /**
+   * Always true.
+   *
+   * @deprecated This constant is useless and always {@code true}. To detect Java versions use
+   *     {@link Runtime#version()}
+   */
+  @Deprecated public static final boolean JRE_IS_MINIMUM_JAVA8 = true;
+
+  /**
+   * Always true.
+   *
+   * @deprecated This constant is useless and always {@code true}. To detect Java versions use
+   *     {@link Runtime#version()}
+   */
+  @Deprecated public static final boolean JRE_IS_MINIMUM_JAVA9 = true;
+
+  /**
+   * Always true.
+   *
+   * @deprecated This constant is useless and always {@code true}. To detect Java versions use
+   *     {@link Runtime#version()}
+   */
+  @Deprecated public static final boolean JRE_IS_MINIMUM_JAVA11 = true;
 }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 10826517b1d..53a8accfc0b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -70,7 +70,6 @@ import org.apache.lucene.tests.util.RamUsageTester;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 
 public class TestLRUQueryCache extends LuceneTestCase {
@@ -356,11 +355,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
 
   // This test makes sure that by making the same assumptions as LRUQueryCache, RAMUsageTester
   // computes the same memory usage.
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-7595")
   public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
-    assumeFalse(
-        "LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)",
-        Constants.JRE_IS_MINIMUM_JAVA9);
-
     final LRUQueryCache queryCache =
         new LRUQueryCache(
             1 + random().nextInt(5),
@@ -496,11 +492,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // that require very little memory. In that case most of the memory is taken
   // by the cache itself, not cache entries, and we want to make sure that
   // memory usage is not grossly underestimated.
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-7595")
   public void testRamBytesUsedConstantEntryOverhead() throws IOException {
-    assumeFalse(
-        "LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)",
-        Constants.JRE_IS_MINIMUM_JAVA9);
-
     final LRUQueryCache queryCache =
         new LRUQueryCache(1000000, 10000000, context -> true, Float.POSITIVE_INFINITY);
 
@@ -1171,10 +1164,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
     }
   }
 
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-7604")
   public void testDetectMutatedQueries() throws IOException {
-    LuceneTestCase.assumeFalse(
-        "LUCENE-7604: For some unknown reason the non-constant BadQuery#hashCode() does not trigger ConcurrentModificationException on Java 9 b150",
-        Constants.JRE_IS_MINIMUM_JAVA9);
     Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     w.addDocument(new Document());
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTestCheckIndex.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTestCheckIndex.java
index 98fd4003cd8..afd3b10362c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTestCheckIndex.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTestCheckIndex.java
@@ -92,7 +92,6 @@ public class BaseTestCheckIndex extends LuceneTestCase {
     assertEquals(18, seg.termVectorStatus.docCount);
     assertEquals(18, seg.termVectorStatus.totVectors);
 
-    assertNotNull(seg.diagnostics.get("java.vm.version"));
     assertNotNull(seg.diagnostics.get("java.runtime.version"));
 
     assertTrue(seg.diagnostics.size() > 0);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
index 14b56f05e1f..55ba5b33174 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
@@ -48,7 +48,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
 import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
-import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -83,7 +82,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -636,24 +634,6 @@ public abstract class LuceneTestCase extends Assert {
     return ignoreAfterMaxFailuresDelegate.getAndSet(newValue);
   }
 
-  /**
-   * Max 10mb of static data stored in a test suite class after the suite is complete. Prevents
-   * static data structures leaking and causing OOMs in subsequent tests.
-   */
-  private static final long STATIC_LEAK_THRESHOLD = 10 * 1024 * 1024;
-
-  /** By-name list of ignored types like loggers etc. */
-  private static final Set<String> STATIC_LEAK_IGNORED_TYPES =
-      Set.of(
-          "org.slf4j.Logger",
-          "org.apache.solr.SolrLogFormatter",
-          "java.io.File", // Solr sometimes refers to this in a static way, but it has a
-          // "java.nio.fs.Path" inside
-          Path.class
-              .getName(), // causes problems because interface is implemented by hidden classes
-          Class.class.getName(),
-          EnumSet.class.getName());
-
   /**
    * This controls how suite-level rules are nested. It is important that _all_ rules declared in
    * {@link LuceneTestCase} are executed in proper order if they depend on each other.
@@ -671,26 +651,6 @@ public abstract class LuceneTestCase extends Assert {
             .around(new TestRuleAssertionsRequired())
             .around(new TestRuleLimitSysouts(suiteFailureMarker))
             .around(tempFilesCleanupRule = new TestRuleTemporaryFilesCleanup(suiteFailureMarker));
-    // TODO LUCENE-7595: Java 9 does not allow to look into runtime classes, so we have to fix the
-    // RAM usage checker!
-    if (!Constants.JRE_IS_MINIMUM_JAVA9) {
-      r =
-          r.around(
-              new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {
-                @Override
-                protected boolean accept(java.lang.reflect.Field field) {
-                  // Don't count known classes that consume memory once.
-                  if (STATIC_LEAK_IGNORED_TYPES.contains(field.getType().getName())) {
-                    return false;
-                  }
-                  // Don't count references from ourselves, we're top-level.
-                  if (field.getDeclaringClass() == LuceneTestCase.class) {
-                    return false;
-                  }
-                  return super.accept(field);
-                }
-              });
-    }
     classRules =
         r.around(new NoClassHooksShadowingRule())
             .around(
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/RamUsageTester.java
index 870509d011c..42f7f60db53 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/RamUsageTester.java
@@ -44,7 +44,6 @@ import java.util.function.ToLongFunction;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
-import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.SuppressForbidden;
 
@@ -170,49 +169,44 @@ public final class RamUsageTester {
      * and accumulate this object's shallow size.
      */
     try {
-      if (Constants.JRE_IS_MINIMUM_JAVA9) {
-        long alignedShallowInstanceSize = RamUsageEstimator.shallowSizeOf(ob);
-
-        Predicate<Class<?>> isJavaModule = (clazz) -> 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.
-        }
+      long alignedShallowInstanceSize = RamUsageEstimator.shallowSizeOf(ob);
+
+      Predicate<Class<?>> isJavaModule = (clazz) -> 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;
       }
 
       ClassCache cachedInfo = classCache.get(obClazz);