You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/11/27 21:41:07 UTC

svn commit: r1642229 [2/7] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/lucene/benchmark/src/ dev-tools/idea/lucene/highlighter/ dev-tools/maven/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/a...

Modified: lucene/dev/branches/lucene6005/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/common-build.xml?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/common-build.xml (original)
+++ lucene/dev/branches/lucene6005/lucene/common-build.xml Thu Nov 27 20:41:01 2014
@@ -162,13 +162,13 @@
 
   <property name="javac.deprecation" value="off"/>
   <property name="javac.debug" value="on"/>
-  <property name="javac.source" value="1.7"/>
-  <property name="javac.target" value="1.7"/>
+  <property name="javac.source" value="1.8"/>
+  <property name="javac.target" value="1.8"/>
   <property name="javac.args" value="-Xlint -Xlint:-deprecation -Xlint:-serial -Xlint:-options"/>
-  <property name="javadoc.link" value="http://download.oracle.com/javase/7/docs/api/"/>
+  <property name="javadoc.link" value="http://download.oracle.com/javase/8/docs/api/"/>
   <property name="javadoc.link.junit" value="http://junit.sourceforge.net/javadoc/"/>
   <property name="javadoc.packagelist.dir" location="${common.dir}/tools/javadoc"/>
-  <available file="${javadoc.packagelist.dir}/java7/package-list" property="javadoc.java7.packagelist.exists"/>
+  <available file="${javadoc.packagelist.dir}/java8/package-list" property="javadoc.java8.packagelist.exists"/>
   <property name="javadoc.access" value="protected"/>
   <property name="javadoc.charset" value="utf-8"/>
   <property name="javadoc.dir" location="${common.dir}/build/docs"/>
@@ -287,7 +287,7 @@
   </propertyset>
 
   <patternset id="lucene.local.src.package.patterns"
-              excludes="**/pom.xml,**/*.iml,**/*.jar,build/**,dist/**,benchmark/work/**,benchmark/temp/**,tools/javadoc/java7/**,tools/clover/**"
+              excludes="**/pom.xml,**/*.iml,**/*.jar,build/**,dist/**,benchmark/work/**,benchmark/temp/**,tools/javadoc/java8/**,tools/clover/**"
   />
 
   <!-- Default exclude sources and javadoc jars from Ivy fetch to save time and bandwidth -->
@@ -304,9 +304,9 @@
     </condition>
   </fail>
 
-  <fail message="Minimum supported Java version is 1.7.">
+  <fail message="Minimum supported Java version is 1.8.">
     <condition>
-      <not><hasmethod classname="java.lang.Throwable" method="getSuppressed"/></not>
+      <not><hasmethod classname="java.util.Arrays" method="parallelSort"/></not>
     </condition>
   </fail>
 
@@ -332,7 +332,6 @@
    -->
   <condition property="build.java.runtime" value="${-cleaned.specification.version}" else="unknown">
     <or>
-      <equals arg1="${-cleaned.specification.version}" arg2="1.7"/>
       <equals arg1="${-cleaned.specification.version}" arg2="1.8"/>
       <equals arg1="${-cleaned.specification.version}" arg2="1.9"/>
     </or>
@@ -350,10 +349,7 @@
         <contains string="${java.vm.name}" substring="openjdk" casesensitive="false"/>
         <contains string="${java.vm.name}" substring="jrockit" casesensitive="false"/>
       </or>
-      <or>
-        <equals arg1="${build.java.runtime}" arg2="1.7"/>
-        <equals arg1="${build.java.runtime}" arg2="1.8"/>
-      </or>
+      <equals arg1="${build.java.runtime}" arg2="1.8"/>
       <!-- TODO: Fix this! For now only run this on 64bit, because jTIDY OOMs with default heap size: -->
       <contains string="${os.arch}" substring="64"/>
     </and>
@@ -361,15 +357,10 @@
 
   <!-- workaround for https://issues.apache.org/bugzilla/show_bug.cgi?id=53347 -->
   <condition property="build.compiler" value="javac1.7">
-    <and>
-      <not>
-        <equals arg1="${build.java.runtime}" arg2="1.7"/>
-      </not>
-      <or>
-        <antversion exactly="1.8.3" />
-        <antversion exactly="1.8.4" />
-      </or>
-    </and>
+    <or>
+      <antversion exactly="1.8.3" />
+      <antversion exactly="1.8.4" />
+    </or>
   </condition>
 
   <target name="-documentation-lint-unsupported" unless="documentation-lint.supported">
@@ -381,12 +372,8 @@
     <echo level="warning" message="WARN: Linting documentation HTML is not supported on this Java version (${build.java.runtime}) / JVM (${java.vm.name}). NOTHING DONE!"/>
   </target>
 
-  <!-- for now disable doclint on JDK 8+: -->
-  <condition property="javadoc.args" value="-Xdoclint:none" else="">
-    <not>
-      <equals arg1="${build.java.runtime}" arg2="1.7"/>
-    </not>
-  </condition>
+  <!-- for now disable doclint: -->
+  <property name="javadoc.args" value="-Xdoclint:none"/>
 
   <!-- Import custom ANT tasks. -->
   <import file="${common.dir}/tools/custom-tasks.xml" />
@@ -411,10 +398,14 @@
     <!-- <property name="ivy.settings.uptodate" value="true"/> -->
   </target>
 
+  <condition property="ivy.symlink">
+    <os family="unix"/>
+  </condition>
+
   <target name="resolve" depends="ivy-availability-check,ivy-configure">
     <!-- todo, make this a property or something. 
          only special cases need bundles -->
-    <ivy:retrieve type="jar,bundle,test,test-jar,tests" log="download-only"
+    <ivy:retrieve type="jar,bundle,test,test-jar,tests" log="download-only" symlink="${ivy.symlink}"
                   conf="${ivy.default.configuration}" sync="${ivy.sync}"/>
   </target>
 
@@ -919,7 +910,7 @@
             <value><!-- empty/ default encoding. --></value>
 
             <!--
-            Disabled because of Java 1.7 bug on Linux/ Unix:
+            Disabled because of Java 1.8 bug on Linux/ Unix:
             http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7181721
 
             <value>UTF-16</value>     
@@ -1875,10 +1866,7 @@ ${ant.project.name}.test.dependencies=${
   <!-- ECJ Javadoc linting: -->
   
   <condition property="ecj-javadoc-lint.supported">
-    <or>
-      <equals arg1="${build.java.runtime}" arg2="1.7"/>
-      <equals arg1="${build.java.runtime}" arg2="1.8"/>
-    </or>
+    <equals arg1="${build.java.runtime}" arg2="1.8"/>
   </condition>
 
   <condition property="ecj-javadoc-lint-tests.supported">
@@ -1974,7 +1962,7 @@ ${ant.project.name}.test.dependencies=${
     <attribute name="overview" default="${src.dir}/overview.html"/>
     <attribute name="linksource" default="no"/>
     <sequential>
-      <antcall target="download-java7-javadoc-packagelist"/>
+      <antcall target="download-java8-javadoc-packagelist"/>
       <delete file="@{destdir}/stylesheet.css" failonerror="false"/>
       <copy todir="@{destdir}" file="${prettify.dir}/prettify.js" overwrite="false" />
       <record name="@{destdir}/log_javadoc.txt" action="start" append="no"/>
@@ -2004,7 +1992,7 @@ ${ant.project.name}.test.dependencies=${
         <tag name="lucene.internal"
         description="NOTE: This API is for internal purposes only and might change in incompatible ways in the next release."/>
       	<link offline="true" packagelistLoc="${javadoc.dir}"/>
-        <link offline="true" href="${javadoc.link}" packagelistLoc="${javadoc.packagelist.dir}/java7"/>
+        <link offline="true" href="${javadoc.link}" packagelistLoc="${javadoc.packagelist.dir}/java8"/>
         <bottom><![CDATA[
           <i>Copyright &copy; ${year} Apache Software Foundation.  All Rights Reserved.</i>
           <script src='{@docRoot}/prettify.js' type='text/javascript'></script>
@@ -2058,72 +2046,9 @@ ${ant.project.name}.test.dependencies=${
           </and>
         </condition>
       </fail>
-
-      <patch-javadoc dir="@{destdir}" docencoding="${javadoc.charset}"/>
    </sequential>
   </macrodef>
 
-  <!--
-    Patch frame injection bugs in javadoc generated files - see CVE-2013-1571, http://www.kb.cert.org/vuls/id/225657
-    
-    Feel free to use this macro in your own Ant build file. This macro works together with the javadoc task on Ant
-    and should be invoked directly after its execution to patch broken javadocs, e.g.:
-      <patch-javadoc dir="..." docencoding="UTF-8"/>
-    Please make sure that the docencoding parameter uses the same charset like javadoc's docencoding. Default
-    is the platform default encoding (like the javadoc task).
-    The specified dir is the destination directory of the javadoc task.
-  -->
-  <macrodef name="patch-javadoc">
-    <attribute name="dir"/>
-    <attribute name="docencoding" default="${file.encoding}"/>
-    <sequential>
-      <replace encoding="@{docencoding}" summary="true" taskname="patch-javadoc">
-        <fileset dir="@{dir}" casesensitive="false" includes="**/index.html,**/index.htm,**/toc.html,**/toc.htm">
-          <!-- TODO: add encoding="@{docencoding}" to contains check, when we are on ANT 1.9.0: -->
-          <not><contains text="function validURL(url) {" casesensitive="true" /></not>
-        </fileset>
-        <replacetoken><![CDATA[function loadFrames() {]]></replacetoken>
-        <replacevalue expandProperties="false"><![CDATA[if (targetPage != "" && !validURL(targetPage))
-        targetPage = "undefined";
-    function validURL(url) {
-        var pos = url.indexOf(".html");
-        if (pos == -1 || pos != url.length - 5)
-            return false;
-        var allowNumber = false;
-        var allowSep = false;
-        var seenDot = false;
-        for (var i = 0; i < url.length - 5; i++) {
-            var ch = url.charAt(i);
-            if ('a' <= ch && ch <= 'z' ||
-                    'A' <= ch && ch <= 'Z' ||
-                    ch == '$' ||
-                    ch == '_') {
-                allowNumber = true;
-                allowSep = true;
-            } else if ('0' <= ch && ch <= '9'
-                    || ch == '-') {
-                if (!allowNumber)
-                     return false;
-            } else if (ch == '/' || ch == '.') {
-                if (!allowSep)
-                    return false;
-                allowNumber = false;
-                allowSep = false;
-                if (ch == '.')
-                     seenDot = true;
-                if (ch == '/' && seenDot)
-                     return false;
-            } else {
-                return false;
-            }
-        }
-        return true;
-    }
-    function loadFrames() {]]></replacevalue>
-      </replace>
-    </sequential>
-  </macrodef>
-
   <target name="check-javadocs-uptodate">
     <uptodate property="javadocs-uptodate-${name}" targetfile="${build.dir}/${final.name}-javadoc.jar">
       <srcfiles dir="${src.dir}">
@@ -2144,10 +2069,10 @@ ${ant.project.name}.test.dependencies=${
     </sequential>
   </macrodef>
 
-  <target name="download-java7-javadoc-packagelist" unless="javadoc.java7.packagelist.exists">
-    <mkdir dir="${javadoc.packagelist.dir}/java7"/>
+  <target name="download-java8-javadoc-packagelist" unless="javadoc.java8.packagelist.exists">
+    <mkdir dir="${javadoc.packagelist.dir}/java8"/>
     <get src="${javadoc.link}/package-list"
-         dest="${javadoc.packagelist.dir}/java7/package-list" ignoreerrors="true"/>
+         dest="${javadoc.packagelist.dir}/java8/package-list" ignoreerrors="true"/>
   </target>
 
   <!-- VALIDATION work -->
@@ -2339,7 +2264,7 @@ ${ant.project.name}.test.dependencies=${
   <property name="forbidden-sysout-excludes" value=""/>
   
   <target name="-install-forbidden-apis" unless="forbidden-apis.loaded" depends="ivy-availability-check,ivy-configure">
-    <ivy:cachepath organisation="de.thetaphi" module="forbiddenapis" revision="1.6.1"
+    <ivy:cachepath organisation="de.thetaphi" module="forbiddenapis" revision="1.7"
       inline="true" conf="default" transitive="true" pathid="forbidden-apis.classpath"/>
     <taskdef name="forbidden-apis" classname="de.thetaphi.forbiddenapis.AntTask" classpathref="forbidden-apis.classpath"/>
     <property name="forbidden-apis.loaded" value="true"/>

Modified: lucene/dev/branches/lucene6005/lucene/core/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/build.xml?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/build.xml (original)
+++ lucene/dev/branches/lucene6005/lucene/core/build.xml Thu Nov 27 20:41:01 2014
@@ -214,7 +214,7 @@
       
       if (!properties["lockverify.count"]) {
         int count = Boolean.parseBoolean(properties["tests.nightly"]) ?
-          30000 : 2000;
+          30000 : 500;
         count *= Integer.parseInt(properties["tests.multiplier"]);
         properties["lockverify.count"] = count;
       }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java Thu Nov 27 20:41:01 2014
@@ -93,11 +93,9 @@ public abstract class FieldsConsumer imp
       final FieldsProducer f = mergeState.fieldsProducers[readerIndex];
 
       final int maxDoc = mergeState.maxDocs[readerIndex];
-      if (f != null) {
-        f.checkIntegrity();
-        slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
-        fields.add(f);
-      }
+      f.checkIntegrity();
+      slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
+      fields.add(f);
       docBase += maxDoc;
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Thu Nov 27 20:41:01 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene5
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Collections;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
@@ -33,7 +32,6 @@ import org.apache.lucene.index.IndexOpti
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -1302,11 +1300,6 @@ public final class Lucene50PostingsReade
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED;
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   @Override
   public void checkIntegrity() throws IOException {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java Thu Nov 27 20:41:01 2014
@@ -1198,6 +1198,15 @@ public class FieldTypes {
     this.defaultSimilarity = defaultSimilarity;
   }
 
+  public FieldTypes(FieldTypes other) {
+    readOnly = true;
+    indexCreatedVersion = other.indexCreatedVersion;
+    this.defaultIndexAnalyzer = null;
+    this.defaultQueryAnalyzer = other.defaultQueryAnalyzer;
+    this.defaultSimilarity = other.defaultSimilarity;
+    addAll(other);
+  }
+
   // nocommit messy we steal this from commitdata namespace...
   /** Key used to store the field types inside {@link IndexWriter#setCommitData}. */
   public static final String FIELD_TYPES_KEY = "FieldTypes";

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java Thu Nov 27 20:41:01 2014
@@ -116,11 +116,6 @@ class BufferedUpdatesStream implements A
   public long ramBytesUsed() {
     return bytesUsed.get();
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   public static class ApplyDeletesResult {
     
@@ -383,10 +378,6 @@ class BufferedUpdatesStream implements A
   private synchronized long applyTermDeletes(Iterable<Term> termsIter, ReadersAndUpdates rld, SegmentReader reader) throws IOException {
     long delCount = 0;
     Fields fields = reader.fields();
-    if (fields == null) {
-      // This reader has no postings
-      return 0;
-    }
 
     TermsEnum termsEnum = null;
 
@@ -456,10 +447,6 @@ class BufferedUpdatesStream implements A
   private synchronized void applyDocValuesUpdates(Iterable<? extends DocValuesUpdate> updates, 
       ReadersAndUpdates rld, SegmentReader reader, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
     Fields fields = reader.fields();
-    if (fields == null) {
-      // This reader has no postings
-      return;
-    }
 
     // TODO: we can process the updates per DV field, from last to first so that
     // if multiple terms affect same document for the same field, we add an update

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Nov 27 20:41:01 2014
@@ -1119,11 +1119,6 @@ public class CheckIndex implements Close
     final Status.TermIndexStatus status = new Status.TermIndexStatus();
     int computedFieldCount = 0;
     
-    if (fields == null) {
-      msg(infoStream, "OK [no fields/terms]");
-      return status;
-    }
-    
     DocsEnum docs = null;
     DocsEnum docsAndFreqs = null;
     DocsAndPositionsEnum postings = null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Thu Nov 27 20:41:01 2014
@@ -334,33 +334,7 @@ public class ConcurrentMergeScheduler ex
     // pending merges, until it's empty:
     while (true) {
 
-      long startStallTime = 0;
-      while (writer.hasPendingMerges() && mergeThreadCount() >= maxMergeCount) {
-        // This means merging has fallen too far behind: we
-        // have already created maxMergeCount threads, and
-        // now there's at least one more merge pending.
-        // Note that only maxThreadCount of
-        // those created merge threads will actually be
-        // running; the rest will be paused (see
-        // updateMergeThreads).  We stall this producer
-        // thread to prevent creation of new segments,
-        // until merging has caught up:
-        startStallTime = System.currentTimeMillis();
-        if (verbose()) {
-          message("    too many merges; stalling...");
-        }
-        try {
-          wait();
-        } catch (InterruptedException ie) {
-          throw new ThreadInterruptedException(ie);
-        }
-      }
-
-      if (verbose()) {
-        if (startStallTime != 0) {
-          message("  stalled for " + (System.currentTimeMillis()-startStallTime) + " msec");
-        }
-      }
+      maybeStall();
 
       MergePolicy.OneMerge merge = writer.getNextMerge();
       if (merge == null) {
@@ -400,6 +374,44 @@ public class ConcurrentMergeScheduler ex
     }
   }
 
+  /** This is invoked by {@link #merge} to possibly stall the incoming
+   *  thread when there are too many merges running or pending.  The 
+   *  default behavior is to force this thread, which is producing too
+   *  many segments for merging to keep up, to wait until merges catch
+   *  up. Applications that can take other less drastic measures, such
+   *  as limiting how many threads are allowed to index, can do nothing
+   *  here and throttle elsewhere. */
+
+  protected synchronized void maybeStall() {
+    long startStallTime = 0;
+    while (writer.hasPendingMerges() && mergeThreadCount() >= maxMergeCount) {
+      // This means merging has fallen too far behind: we
+      // have already created maxMergeCount threads, and
+      // now there's at least one more merge pending.
+      // Note that only maxThreadCount of
+      // those created merge threads will actually be
+      // running; the rest will be paused (see
+      // updateMergeThreads).  We stall this producer
+      // thread to prevent creation of new segments,
+      // until merging has caught up:
+      startStallTime = System.currentTimeMillis();
+      if (verbose()) {
+        message("    too many merges; stalling...");
+      }
+      try {
+        wait();
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
+      }
+    }
+
+    if (verbose()) {
+      if (startStallTime != 0) {
+        message("  stalled for " + (System.currentTimeMillis()-startStallTime) + " msec");
+      }
+    }
+  }
+
   /** Does the actual merge, by calling {@link IndexWriter#merge} */
   protected void doMerge(MergePolicy.OneMerge merge) throws IOException {
     writer.merge(merge);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocValues.java Thu Nov 27 20:41:01 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -199,12 +200,31 @@ public final class DocValues {
   // some helpers, for transition from fieldcache apis.
   // as opposed to the LeafReader apis (which must be strict for consistency), these are lenient
   
+  // helper method: to give a nice error when LeafReader.getXXXDocValues returns null.
+  private static void checkField(LeafReader in, String field, DocValuesType... expected) {
+    FieldInfo fi = in.getFieldInfos().fieldInfo(field);
+    if (fi != null) {
+      DocValuesType actual = fi.getDocValuesType();
+      throw new IllegalStateException("unexpected docvalues type " + actual + 
+                                        " for field '" + field + "' " +
+                                        (expected.length == 1 
+                                        ? "(expected=" + expected[0]
+                                        : "(expected one of " + Arrays.toString(expected)) + "). " +
+                                        "Use UninvertingReader or index with docvalues.");
+    }
+  }
+  
   /**
-   * Returns NumericDocValues for the reader, or {@link #emptyNumeric()} if it has none. 
+   * Returns NumericDocValues for the field, or {@link #emptyNumeric()} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#NUMERIC}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static NumericDocValues getNumeric(LeafReader in, String field) throws IOException {
-    NumericDocValues dv = in.getNumericDocValues(field);
+  public static NumericDocValues getNumeric(LeafReader reader, String field) throws IOException {
+    NumericDocValues dv = reader.getNumericDocValues(field);
     if (dv == null) {
+      checkField(reader, field, DocValuesType.NUMERIC);
       return emptyNumeric();
     } else {
       return dv;
@@ -212,13 +232,19 @@ public final class DocValues {
   }
   
   /**
-   * Returns BinaryDocValues for the reader, or {@link #emptyBinary} if it has none. 
+   * Returns BinaryDocValues for the field, or {@link #emptyBinary} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#BINARY}
+   *                               or {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static BinaryDocValues getBinary(LeafReader in, String field) throws IOException {
-    BinaryDocValues dv = in.getBinaryDocValues(field);
+  public static BinaryDocValues getBinary(LeafReader reader, String field) throws IOException {
+    BinaryDocValues dv = reader.getBinaryDocValues(field);
     if (dv == null) {
-      dv = in.getSortedDocValues(field);
+      dv = reader.getSortedDocValues(field);
       if (dv == null) {
+        checkField(reader, field, DocValuesType.BINARY, DocValuesType.SORTED);
         return emptyBinary();
       }
     }
@@ -226,11 +252,16 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedDocValues for the reader, or {@link #emptySorted} if it has none. 
+   * Returns SortedDocValues for the field, or {@link #emptySorted} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedDocValues getSorted(LeafReader in, String field) throws IOException {
-    SortedDocValues dv = in.getSortedDocValues(field);
+  public static SortedDocValues getSorted(LeafReader reader, String field) throws IOException {
+    SortedDocValues dv = reader.getSortedDocValues(field);
     if (dv == null) {
+      checkField(reader, field, DocValuesType.SORTED);
       return emptySorted();
     } else {
       return dv;
@@ -238,29 +269,41 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedNumericDocValues for the reader, or {@link #emptySortedNumeric} if it has none. 
+   * Returns SortedNumericDocValues for the field, or {@link #emptySortedNumeric} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED_NUMERIC}
+   *                               or {@link DocValuesType#NUMERIC}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedNumericDocValues getSortedNumeric(LeafReader in, String field) throws IOException {
-    SortedNumericDocValues dv = in.getSortedNumericDocValues(field);
+  public static SortedNumericDocValues getSortedNumeric(LeafReader reader, String field) throws IOException {
+    SortedNumericDocValues dv = reader.getSortedNumericDocValues(field);
     if (dv == null) {
-      NumericDocValues single = in.getNumericDocValues(field);
+      NumericDocValues single = reader.getNumericDocValues(field);
       if (single == null) {
-        return emptySortedNumeric(in.maxDoc());
+        checkField(reader, field, DocValuesType.SORTED_NUMERIC, DocValuesType.NUMERIC);
+        return emptySortedNumeric(reader.maxDoc());
       }
-      Bits bits = in.getDocsWithField(field);
+      Bits bits = reader.getDocsWithField(field);
       return singleton(single, bits);
     }
     return dv;
   }
   
   /**
-   * Returns SortedSetDocValues for the reader, or {@link #emptySortedSet} if it has none. 
+   * Returns SortedSetDocValues for the field, or {@link #emptySortedSet} if it has none. 
+   * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED_SET}
+   *                               or {@link DocValuesType#SORTED}.
+   * @throws IOException if an I/O error occurs.
    */
-  public static SortedSetDocValues getSortedSet(LeafReader in, String field) throws IOException {
-    SortedSetDocValues dv = in.getSortedSetDocValues(field);
+  public static SortedSetDocValues getSortedSet(LeafReader reader, String field) throws IOException {
+    SortedSetDocValues dv = reader.getSortedSetDocValues(field);
     if (dv == null) {
-      SortedDocValues sorted = in.getSortedDocValues(field);
+      SortedDocValues sorted = reader.getSortedDocValues(field);
       if (sorted == null) {
+        checkField(reader, field, DocValuesType.SORTED, DocValuesType.SORTED_SET);
         return emptySortedSet();
       }
       return singleton(sorted);
@@ -269,12 +312,21 @@ public final class DocValues {
   }
   
   /**
-   * Returns Bits for the reader, or {@link Bits} matching nothing if it has none. 
+   * Returns Bits for the field, or {@link Bits} matching nothing if it has none. 
+   * @return bits instance, or an empty instance if {@code field} does not exist in this reader.
+   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
+   * @throws IOException if an I/O error occurs.
    */
-  public static Bits getDocsWithField(LeafReader in, String field) throws IOException {
-    Bits dv = in.getDocsWithField(field);
+  public static Bits getDocsWithField(LeafReader reader, String field) throws IOException {
+    Bits dv = reader.getDocsWithField(field);
     if (dv == null) {
-      return new Bits.MatchNoBits(in.maxDoc());
+      assert DocValuesType.values().length == 6; // we just don't want NONE
+      checkField(reader, field, DocValuesType.BINARY, 
+                            DocValuesType.NUMERIC, 
+                            DocValuesType.SORTED, 
+                            DocValuesType.SORTED_NUMERIC, 
+                            DocValuesType.SORTED_SET);
+      return new Bits.MatchNoBits(reader.maxDoc());
     } else {
       return dv;
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Nov 27 20:41:01 2014
@@ -21,7 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Queue;
@@ -314,12 +313,6 @@ final class DocumentsWriter implements C
   }
 
   boolean anyChanges() {
-    if (infoStream.isEnabled("DW")) {
-      infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get()
-          + " deletes=" + anyDeletions() + " hasTickets:"
-          + ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
-          + pendingChangesInCurrentFullFlush);
-    }
     /*
      * changes are either in a DWPT or in the deleteQueue.
      * yet if we currently flush deletes and / or dwpt there
@@ -327,7 +320,16 @@ final class DocumentsWriter implements C
      * before they are published to the IW. ie we need to check if the 
      * ticket queue has any tickets.
      */
-    return numDocsInRAM.get() != 0 || anyDeletions() || ticketQueue.hasTickets() || pendingChangesInCurrentFullFlush;
+    boolean anyChanges = numDocsInRAM.get() != 0 || anyDeletions() || ticketQueue.hasTickets() || pendingChangesInCurrentFullFlush;
+    if (infoStream.isEnabled("DW")) {
+      if (anyChanges) {
+        infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get()
+                           + " deletes=" + anyDeletions() + " hasTickets:"
+                           + ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
+                           + pendingChangesInCurrentFullFlush);
+      }
+    }
+    return anyChanges;
   }
   
   public int getBufferedDeleteTermsSize() {
@@ -681,11 +683,6 @@ final class DocumentsWriter implements C
   public long ramBytesUsed() {
     return flushControl.ramBytesUsed();
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   static final class ApplyDeletesEvent implements Event {
     static final Event INSTANCE = new ApplyDeletesEvent();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -457,11 +456,6 @@ final class DocumentsWriterDeleteQueue i
   }
 
   @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
-
-  @Override
   public String toString() {
     return "DWDQ: [ generation: " + generation + " ]";
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -435,15 +434,10 @@ final class DocumentsWriterFlushControl 
 
   @Override
   public long ramBytesUsed() {
+    // TODO: improve this to return more detailed info?
     return getDeleteBytesUsed() + netBytes();
   }
-
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    // TODO: improve this?
-    return Collections.emptyList();
-  }
-
+  
   synchronized int numFlushingDWPT() {
     return flushingWriters.size();
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java Thu Nov 27 20:41:01 2014
@@ -80,11 +80,7 @@ public class ExitableDirectoryReader ext
 
     @Override
     public Fields fields() throws IOException {
-      Fields fields = super.fields();
-      if (fields == null) {
-        return null;
-      }
-      return new ExitableFields(fields, queryTimeout);
+      return new ExitableFields(super.fields(), queryTimeout);
     }
     
     @Override

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java Thu Nov 27 20:41:01 2014
@@ -100,7 +100,7 @@ public abstract class IndexReader implem
    */
   public static interface ReaderClosedListener {
     /** Invoked when the {@link IndexReader} is closed. */
-    public void onClose(IndexReader reader);
+    public void onClose(IndexReader reader) throws IOException;
   }
 
   // nocommit need getFieldTypes; how should MultiReader impl?
@@ -199,7 +199,7 @@ public abstract class IndexReader implem
    */
   public final void incRef() {
     if (!tryIncRef()) {
-       ensureOpen();
+      ensureOpen();
     }
   }
   

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Thu Nov 27 20:41:01 2014
@@ -475,11 +475,6 @@ public class IndexWriter implements Clos
     ensureOpen();
     return docWriter.ramBytesUsed();
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   /** Holds shared SegmentReader instances. IndexWriter uses
    *  SegmentReaders for 1) applying deletes, 2) doing
@@ -3822,6 +3817,14 @@ public class IndexWriter implements Clos
         merge.readers.set(i, null);
       }
     }
+
+    try {
+      merge.mergeFinished();
+    } catch (Throwable t) {
+      if (th == null) {
+        th = t;
+      }
+    }
     
     // If any error occured, throw it.
     if (!suppressExceptions) {
@@ -4414,10 +4417,13 @@ public class IndexWriter implements Clos
   synchronized boolean nrtIsCurrent(SegmentInfos infos) {
     //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
     ensureOpen();
+    boolean isCurrent = infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
+      if (isCurrent == false) {
+        infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
+      }
     }
-    return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
+    return isCurrent;
   }
 
   synchronized boolean isClosed() {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReader.java Thu Nov 27 20:41:01 2014
@@ -80,7 +80,7 @@ public abstract class LeafReader extends
   public static interface CoreClosedListener {
     /** Invoked when the shared core of the original {@code
      *  SegmentReader} has closed. */
-    public void onClose(Object ownerCoreCacheKey);
+    public void onClose(Object ownerCoreCacheKey) throws IOException;
   }
 
   private static class CoreClosedListenerWrapper implements ReaderClosedListener {
@@ -92,7 +92,7 @@ public abstract class LeafReader extends
     }
 
     @Override
-    public void onClose(IndexReader reader) {
+    public void onClose(IndexReader reader) throws IOException {
       listener.onClose(reader.getCoreCacheKey());
     }
 
@@ -136,18 +136,13 @@ public abstract class LeafReader extends
 
   /**
    * Returns {@link Fields} for this reader.
-   * This method may return null if the reader has no
-   * postings.
+   * This method will not return null.
    */
   public abstract Fields fields() throws IOException;
 
   @Override
   public final int docFreq(Term term) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    final Terms terms = fields.terms(term.field());
+    final Terms terms = terms(term.field());
     if (terms == null) {
       return 0;
     }
@@ -166,11 +161,7 @@ public abstract class LeafReader extends
    * away. */
   @Override
   public final long totalTermFreq(Term term) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    final Terms terms = fields.terms(term.field());
+    final Terms terms = terms(term.field());
     if (terms == null) {
       return 0;
     }
@@ -211,11 +202,7 @@ public abstract class LeafReader extends
 
   /** This may return null if the field does not exist.*/
   public final Terms terms(String field) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return null;
-    }
-    return fields.terms(field);
+    return fields().terms(field);
   }
 
   /** Returns {@link DocsEnum} for the specified term.
@@ -225,14 +212,11 @@ public abstract class LeafReader extends
   public final DocsEnum termDocsEnum(Term term) throws IOException {
     assert term.field() != null;
     assert term.bytes() != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(term.field());
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term.bytes())) {
-          return termsEnum.docs(getLiveDocs(), null);
-        }
+    final Terms terms = terms(term.field());
+    if (terms != null) {
+      final TermsEnum termsEnum = terms.iterator(null);
+      if (termsEnum.seekExact(term.bytes())) {
+        return termsEnum.docs(getLiveDocs(), null);
       }
     }
     return null;
@@ -245,14 +229,11 @@ public abstract class LeafReader extends
   public final DocsAndPositionsEnum termPositionsEnum(Term term) throws IOException {
     assert term.field() != null;
     assert term.bytes() != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(term.field());
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term.bytes())) {
-          return termsEnum.docsAndPositions(getLiveDocs(), null);
-        }
+    final Terms terms = terms(term.field());
+    if (terms != null) {
+      final TermsEnum termsEnum = terms.iterator(null);
+      if (termsEnum.seekExact(term.bytes())) {
+        return termsEnum.docsAndPositions(getLiveDocs(), null);
       }
     }
     return null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReaderContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReaderContext.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReaderContext.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LeafReaderContext.java Thu Nov 27 20:41:01 2014
@@ -66,4 +66,9 @@ public final class LeafReaderContext ext
   public LeafReader reader() {
     return reader;
   }
-}
\ No newline at end of file
+
+  @Override
+  public String toString() {
+    return "LeafReaderContext(" + reader + " docBase=" + docBase + " ord=" + ord + ")";
+  }
+}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Thu Nov 27 20:41:01 2014
@@ -129,6 +129,10 @@ public abstract class MergePolicy {
       totalDocCount = count;
     }
 
+    /** Called by {@link IndexWriter} after the merge is done and all readers have been closed. */
+    public void mergeFinished() throws IOException {
+    }
+
     /** Expert: Get the list of readers to merge. Note that this list does not
      *  necessarily match the list of segments to merge and should only be used
      *  to feed SegmentMerger to initialize a merge. When a {@link OneMerge}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -29,7 +28,6 @@ import org.apache.lucene.codecs.StoredFi
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.packed.PackedInts;
@@ -107,6 +105,11 @@ public class MergeState {
 
     for(int i=0;i<numReaders;i++) {
       final LeafReader reader = readers.get(i);
+      // nocommit segment merger should do this?
+      FieldTypes readerFieldTypes = reader.getFieldTypes();
+      if (readerFieldTypes != null) {
+        fieldTypes.addAll(readerFieldTypes);
+      }
 
       maxDocs[i] = reader.maxDoc();
       liveDocs[i] = reader.getLiveDocs();
@@ -135,10 +138,7 @@ public class MergeState {
         if (termVectorsReader != null) {
           termVectorsReader = termVectorsReader.getMergeInstance();
         }
-        fieldsProducer = segmentReader.fields();
-        if (fieldsProducer != null) {
-          fieldsProducer = fieldsProducer.getMergeInstance();
-        }
+        fieldsProducer = segmentReader.fields().getMergeInstance();
       } else {
         // A "foreign" reader
         normsProducer = readerToNormsProducer(reader);
@@ -183,11 +183,6 @@ public class MergeState {
       public long ramBytesUsed() {
         return 0;
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     };
   }
 
@@ -237,11 +232,6 @@ public class MergeState {
       public long ramBytesUsed() {
         return 0;
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     };
   }
 
@@ -270,11 +260,6 @@ public class MergeState {
       public long ramBytesUsed() {
         return 0;
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     };
   }
 
@@ -303,11 +288,6 @@ public class MergeState {
       public long ramBytesUsed() {
         return 0;
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     };
   }
 
@@ -342,11 +322,6 @@ public class MergeState {
       public long ramBytesUsed() {
         return 0;
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     };
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Thu Nov 27 20:41:01 2014
@@ -431,11 +431,6 @@ public class MultiDocValues {
       public long ramBytesUsed() {
         return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(newToOld) + RamUsageEstimator.sizeOf(oldToNew);
       }
-
-      @Override
-      public Iterable<? extends Accountable> getChildResources() {
-        return Collections.emptyList();
-      }
     }
 
     /**

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Thu Nov 27 20:41:01 2014
@@ -63,9 +63,6 @@ public final class MultiFields extends F
   public static Fields getFields(IndexReader reader) throws IOException {
     final List<LeafReaderContext> leaves = reader.leaves();
     switch (leaves.size()) {
-      case 0:
-        // no fields
-        return null;
       case 1:
         // already an atomic reader / reader with one leave
         return leaves.get(0).reader().fields();
@@ -75,14 +72,10 @@ public final class MultiFields extends F
         for (final LeafReaderContext ctx : leaves) {
           final LeafReader r = ctx.reader();
           final Fields f = r.fields();
-          if (f != null) {
-            fields.add(f);
-            slices.add(new ReaderSlice(ctx.docBase, r.maxDoc(), fields.size()-1));
-          }
+          fields.add(f);
+          slices.add(new ReaderSlice(ctx.docBase, r.maxDoc(), fields.size()-1));
         }
-        if (fields.isEmpty()) {
-          return null;
-        } else if (fields.size() == 1) {
+        if (fields.size() == 1) {
           return fields.get(0);
         } else {
           return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
@@ -125,12 +118,7 @@ public final class MultiFields extends F
 
   /**  This method may return null if the field does not exist.*/
   public static Terms getTerms(IndexReader r, String field) throws IOException {
-    final Fields fields = getFields(r);
-    if (fields == null) {
-      return null;
-    } else {
-      return fields.terms(field);
-    }
+    return getFields(r).terms(field);
   }
   
   /** Returns {@link DocsEnum} for the specified field &

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java Thu Nov 27 20:41:01 2014
@@ -29,7 +29,6 @@ import java.util.TreeMap;
 import org.apache.lucene.document.FieldTypes;
 import org.apache.lucene.util.Bits;
 
-
 /** An {@link LeafReader} which reads multiple, parallel indexes.  Each index
  * added must have the same number of documents, but typically each contains
  * different fields. Deletions are taken from the first reader.
@@ -125,12 +124,10 @@ public class ParallelLeafReader extends 
     // build Fields instance
     for (final LeafReader reader : this.parallelReaders) {
       final Fields readerFields = reader.fields();
-      if (readerFields != null) {
-        for (String field : readerFields) {
-          // only add if the reader responsible for that field name is the current:
-          if (fieldToReader.get(field) == reader) {
-            this.fields.addField(field, readerFields.terms(field));
-          }
+      for (String field : readerFields) {
+        // only add if the reader responsible for that field name is the current:
+        if (fieldToReader.get(field) == reader) {
+          this.fields.addField(field, readerFields.terms(field));
         }
       }
     }
@@ -334,4 +331,10 @@ public class ParallelLeafReader extends 
       reader.checkIntegrity();
     }
   }
+
+  /** Returns the {@link LeafReader}s that were passed on init. */
+  public LeafReader[] getParallelReaders() {
+    ensureOpen();
+    return parallelReaders;
+  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Iterator;
 
 import org.apache.lucene.store.IndexInput;
@@ -44,11 +43,6 @@ class PrefixCodedTerms implements Iterab
   public long ramBytesUsed() {
     return buffer.ramBytesUsed();
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   /** @return iterator over the bytes */
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java Thu Nov 27 20:41:01 2014
@@ -67,6 +67,19 @@ public final class ReaderManager extends
     current = DirectoryReader.open(dir);
   }
 
+  /**
+   * Creates and returns a new ReaderManager from the given
+   * already-opened {@link DirectoryReader}, stealing
+   * the incoming reference.
+   *
+   * @param reader the directoryReader to use for future reopens
+   *        
+   * @throws IOException If there is a low-level I/O error
+   */
+  public ReaderManager(DirectoryReader reader) throws IOException {
+    current = reader;
+  }
+
   @Override
   protected void decRef(DirectoryReader reference) throws IOException {
     reference.decRef();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Thu Nov 27 20:41:01 2014
@@ -227,9 +227,4 @@ final class SegmentCoreReaders implement
         ((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) + 
         ((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
   }
-
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java Thu Nov 27 20:41:01 2014
@@ -37,8 +37,7 @@ final class SegmentDocValues {
 
   private final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<>();
 
-  private RefCount<DocValuesProducer> newDocValuesProducer(SegmentCommitInfo si, IOContext context, Directory dir,
-      DocValuesFormat dvFormat, final Long gen, FieldInfos infos) throws IOException {
+  private RefCount<DocValuesProducer> newDocValuesProducer(SegmentCommitInfo si, Directory dir, final Long gen, FieldInfos infos) throws IOException {
     Directory dvDir = dir;
     String segmentSuffix = "";
     if (gen.longValue() != -1) {
@@ -47,7 +46,8 @@ final class SegmentDocValues {
     }
 
     // set SegmentReadState to list only the fields that are relevant to that gen
-    SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, context, segmentSuffix);
+    SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, IOContext.READ, segmentSuffix);
+    DocValuesFormat dvFormat = si.info.getCodec().docValuesFormat();
     return new RefCount<DocValuesProducer>(dvFormat.fieldsProducer(srs)) {
       @SuppressWarnings("synthetic-access")
       @Override
@@ -61,11 +61,10 @@ final class SegmentDocValues {
   }
 
   /** Returns the {@link DocValuesProducer} for the given generation. */
-  synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentCommitInfo si, IOContext context, Directory dir, 
-      DocValuesFormat dvFormat, FieldInfos infos) throws IOException {
+  synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentCommitInfo si, Directory dir, FieldInfos infos) throws IOException {
     RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
     if (dvp == null) {
-      dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos);
+      dvp = newDocValuesProducer(si, dir, gen, infos);
       assert dvp != null;
       genDVProducers.put(gen, dvp);
     } else {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java Thu Nov 27 20:41:01 2014
@@ -26,10 +26,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
@@ -48,26 +46,35 @@ class SegmentDocValuesProducer extends D
   final Set<DocValuesProducer> dvProducers = Collections.newSetFromMap(new IdentityHashMap<DocValuesProducer,Boolean>());
   final List<Long> dvGens = new ArrayList<>();
   
-  SegmentDocValuesProducer(SegmentCommitInfo si, Directory dir, FieldInfos fieldInfos, SegmentDocValues segDocValues, DocValuesFormat dvFormat) throws IOException {
+  /**
+   * Creates a new producer that handles updated docvalues fields
+   * @param si commit point
+   * @param dir directory
+   * @param coreInfos fieldinfos for the segment
+   * @param allInfos all fieldinfos including updated ones
+   * @param segDocValues producer map
+   */
+  SegmentDocValuesProducer(SegmentCommitInfo si, Directory dir, FieldInfos coreInfos, FieldInfos allInfos, SegmentDocValues segDocValues) throws IOException {
     boolean success = false;
     try {
       DocValuesProducer baseProducer = null;
-      for (FieldInfo fi : fieldInfos) {
+      for (FieldInfo fi : allInfos) {
         if (fi.getDocValuesType() == DocValuesType.NONE) {
           continue;
         }
         long docValuesGen = fi.getDocValuesGen();
         if (docValuesGen == -1) {
           if (baseProducer == null) {
-            // the base producer gets all the fields, so the Codec can validate properly
-            baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
+            // the base producer gets the original fieldinfos it wrote
+            baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, dir, coreInfos);
             dvGens.add(docValuesGen);
             dvProducers.add(baseProducer);
           }
           dvProducersByField.put(fi.name, baseProducer);
         } else {
           assert !dvGens.contains(docValuesGen);
-          final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(new FieldInfo[] { fi }));
+          // otherwise, producer sees only the one fieldinfo it wrote
+          final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, dir, new FieldInfos(new FieldInfo[] { fi }));
           dvGens.add(docValuesGen);
           dvProducers.add(dvp);
           dvProducersByField.put(fi.name, dvp);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Thu Nov 27 20:41:01 2014
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FieldsProducer;
@@ -164,15 +163,14 @@ public final class SegmentReader extends
    */
   private DocValuesProducer initDocValuesProducer() throws IOException {
     final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
-    final DocValuesFormat dvFormat = si.info.getCodec().docValuesFormat();
 
     if (!fieldInfos.hasDocValues()) {
       return null;
     } else if (si.hasFieldUpdates()) {
-      return new SegmentDocValuesProducer(si, dir, fieldInfos, segDocValues, dvFormat);
+      return new SegmentDocValuesProducer(si, dir, core.coreFieldInfos, fieldInfos, segDocValues);
     } else {
       // simple case, no DocValues updates
-      return segDocValues.getDocValuesProducer(-1L, si, IOContext.READ, dir, dvFormat, fieldInfos);
+      return segDocValues.getDocValuesProducer(-1L, si, dir, fieldInfos);
     }
   }
 
@@ -515,9 +513,7 @@ public final class SegmentReader extends
   public Iterable<? extends Accountable> getChildResources() {
     ensureOpen();
     List<Accountable> resources = new ArrayList<>();
-    if (core.fields != null) {
-      resources.add(Accountables.namedAccountable("postings", core.fields));
-    }
+    resources.add(Accountables.namedAccountable("postings", core.fields));
     if (core.normsProducer != null) {
       resources.add(Accountables.namedAccountable("norms", core.normsProducer));
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermContext.java Thu Nov 27 20:41:01 2014
@@ -88,16 +88,13 @@ public final class TermContext {
     //if (DEBUG) System.out.println("prts.build term=" + term);
     for (final LeafReaderContext ctx : context.leaves()) {
       //if (DEBUG) System.out.println("  r=" + leaves[i].reader);
-      final Fields fields = ctx.reader().fields();
-      if (fields != null) {
-        final Terms terms = fields.terms(field);
-        if (terms != null) {
-          final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(bytes)) { 
-            final TermState termState = termsEnum.termState();
-            //if (DEBUG) System.out.println("    found");
-            perReaderTermState.register(termState, ctx.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
-          }
+      final Terms terms = ctx.reader().terms(field);
+      if (terms != null) {
+        final TermsEnum termsEnum = terms.iterator(null);
+        if (termsEnum.seekExact(bytes)) { 
+          final TermState termState = termsEnum.termState();
+          //if (DEBUG) System.out.println("    found");
+          perReaderTermState.register(termState, ctx.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
         }
       }
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java Thu Nov 27 20:41:01 2014
@@ -22,6 +22,7 @@ import static org.apache.lucene.search.D
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
@@ -146,8 +147,11 @@ public class CachingWrapperFilter extend
   }
 
   @Override
-  public synchronized Iterable<? extends Accountable> getChildResources() {
-    // Sync only to pull the current set of values:
-    return Accountables.namedAccountables("segment", cache);
+  public Iterable<? extends Accountable> getChildResources() {
+    // Sync to pull the current set of values:
+    synchronized (cache) {
+      // no need to clone, Accountable#namedAccountables already copies the data
+      return Accountables.namedAccountables("segment", cache);
+    }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Collections;
 
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
@@ -93,9 +92,4 @@ public abstract class DocIdSet implement
   public boolean isCacheable() {
     return false;
   }
-
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Thu Nov 27 20:41:01 2014
@@ -84,14 +84,7 @@ public class MultiTermQueryWrapperFilter
    */
   @Override
   public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
-    final LeafReader reader = context.reader();
-    final Fields fields = reader.fields();
-    if (fields == null) {
-      // reader has no fields
-      return null;
-    }
-
-    final Terms terms = fields.terms(query.field);
+    final Terms terms = context.reader().terms(query.field);
     if (terms == null) {
       // field does not exist
       return null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java Thu Nov 27 20:41:01 2014
@@ -47,13 +47,7 @@ abstract class TermCollectingRewrite<Q e
   final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
     IndexReaderContext topReaderContext = reader.getContext();
     for (LeafReaderContext context : topReaderContext.leaves()) {
-      final Fields fields = context.reader().fields();
-      if (fields == null) {
-        // reader has no fields
-        continue;
-      }
-
-      final Terms terms = fields.terms(query.field);
+      final Terms terms = context.reader().terms(query.field);
       if (terms == null) {
         // field does not exist
         continue;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Thu Nov 27 20:41:01 2014
@@ -767,7 +767,7 @@ public abstract class TFIDFSimilarity ex
 
   private Explanation explainScore(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) {
     Explanation result = new Explanation();
-    result.setDescription("score(doc="+doc+",freq="+freq+"), product of:");
+    result.setDescription("score(doc="+doc+",freq="+freq.getValue()+"), product of:");
 
     // explain query weight
     Explanation queryExpl = new Explanation();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java Thu Nov 27 20:41:01 2014
@@ -110,7 +110,10 @@ public class SpanMultiTermQueryWrapper<Q
   public String toString(String field) {
     StringBuilder builder = new StringBuilder();
     builder.append("SpanMultiTermQueryWrapper(");
-    builder.append(query.toString(field));
+    // NOTE: query.toString must be placed in a temp local to avoid compile errors on Java 8u20
+    // see https://bugs.openjdk.java.net/browse/JDK-8056984?page=com.atlassian.streams.streams-jira-plugin:activity-stream-issue-tab
+    String queryStr = query.toString(field);
+    builder.append(queryStr);
     builder.append(")");
     if (getBoost() != 1F) {
       builder.append('^');

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Thu Nov 27 20:41:01 2014
@@ -93,16 +93,11 @@ public class SpanTermQuery extends SpanQ
     if (termContext == null) {
       // this happens with span-not query, as it doesn't include the NOT side in extractTerms()
       // so we seek to the term now in this segment..., this sucks because its ugly mostly!
-      final Fields fields = context.reader().fields();
-      if (fields != null) {
-        final Terms terms = fields.terms(term.field());
-        if (terms != null) {
-          final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(term.bytes())) { 
-            state = termsEnum.termState();
-          } else {
-            state = null;
-          }
+      final Terms terms = context.reader().terms(term.field());
+      if (terms != null) {
+        final TermsEnum termsEnum = terms.iterator(null);
+        if (termsEnum.seekExact(term.bytes())) { 
+          state = termsEnum.termState();
         } else {
           state = null;
         }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMFile.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMFile.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.store;
  */
 
 import java.util.ArrayList;
-import java.util.Collections;
 
 import org.apache.lucene.util.Accountable;
 
@@ -82,11 +81,6 @@ public class RAMFile implements Accounta
   public synchronized long ramBytesUsed() {
     return sizeInBytes;
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java Thu Nov 27 20:41:01 2014
@@ -138,7 +138,17 @@ public abstract class RateLimiter {
             // NOTE: except maybe on real-time JVMs, minimum realistic sleep time
             // is 1 msec; if you pass just 1 nsec the default impl rounds
             // this up to 1 msec:
-            Thread.sleep((int) (pauseNS/1000000), (int) (pauseNS % 1000000));
+            int sleepNS;
+            int sleepMS;
+            if (pauseNS > 100000L * Integer.MAX_VALUE) {
+              // Not really practical (sleeping for 25 days) but we shouldn't overflow int:
+              sleepMS = Integer.MAX_VALUE;
+              sleepNS = 0;
+            } else {
+              sleepMS = (int) (pauseNS/1000000);
+              sleepNS = (int) (pauseNS % 1000000);
+            }
+            Thread.sleep(sleepMS, sleepNS);
           } catch (InterruptedException ie) {
             throw new ThreadInterruptedException(ie);
           }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Accountable.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Accountable.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Accountable.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Accountable.java Thu Nov 27 20:41:01 2014
@@ -17,6 +17,8 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.Collections;
+
 /**
  * An object whose RAM usage can be computed.
  *
@@ -34,7 +36,8 @@ public interface Accountable {
    * The result should be a point-in-time snapshot (to avoid race conditions).
    * @see Accountables
    */
-  // TODO: on java8 make this a default method returning emptyList
-  Iterable<? extends Accountable> getChildResources();
+  default Iterable<? extends Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
 
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BitSet.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BitSet.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BitSet.java Thu Nov 27 20:41:01 2014
@@ -18,7 +18,6 @@ package org.apache.lucene.util;
  */
 
 import java.io.IOException;
-import java.util.Collections;
 
 import org.apache.lucene.search.DocIdSetIterator;
 
@@ -145,9 +144,4 @@ public abstract class BitSet implements 
 
     });
   }
-
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Constants.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Constants.java Thu Nov 27 20:41:01 2014
@@ -92,7 +92,7 @@ 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);
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java Thu Nov 27 20:41:01 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.util;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Collections;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -135,11 +134,6 @@ public final class PagedBytes implements
       }
       return size;
     }
-    
-    @Override
-    public Iterable<? extends Accountable> getChildResources() {
-      return Collections.emptyList();
-    }
 
     @Override
     public String toString() {
@@ -257,11 +251,6 @@ public final class PagedBytes implements
     }
     return size;
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   /** Copy bytes in, writing the length as a 1 or 2 byte
    *  vInt prefix. */

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java Thu Nov 27 20:41:01 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.util.fst;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.store.DataInput;
@@ -481,11 +480,6 @@ class BytesStore extends DataOutput impl
     }
     return size;
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java Thu Nov 27 20:41:01 2014
@@ -20,8 +20,6 @@ package org.apache.lucene.util.packed;
 import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
 import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
-import java.util.Collections;
-
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.LongValues;
@@ -116,11 +114,6 @@ abstract class AbstractPagedMutable<T ex
     }
     return bytesUsed;
   }
-  
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
 
   protected abstract T newUnfilledCopy(long newSize);
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java?rev=1642229&r1=1642228&r2=1642229&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java Thu Nov 27 20:41:01 2014
@@ -27,7 +27,6 @@ import static org.apache.lucene.util.pac
 import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
 import java.io.IOException;
-import java.util.Collections;
 
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
@@ -100,11 +99,6 @@ public final class BlockPackedReader ext
     }
     return size;
   }
-
-  @Override
-  public Iterable<? extends Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
   
   @Override
   public String toString() {