You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2015/01/06 15:35:47 UTC

svn commit: r1649812 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/contrib/ solr/contrib/analytics/src/test-files/solr/collection1/conf/ solr/core/ solr/core/src/java/org/apache/solr/core/ solr/core/src/java/org/apache/solr/update/ solr/core/src/t...

Author: shalin
Date: Tue Jan  6 14:35:46 2015
New Revision: 1649812

URL: http://svn.apache.org/r1649812
Log:
SOLR-6897: Nuke non-NRT mode from code and configuration

Removed:
    lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-warmer-no-reopen.xml
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestNonNRTOpen.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/contrib/   (props changed)
    lucene/dev/branches/branch_5x/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
    lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
    lucene/dev/branches/branch_5x/solr/example/   (props changed)
    lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/db/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/server/   (props changed)
    lucene/dev/branches/branch_5x/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
    lucene/dev/branches/branch_5x/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Tue Jan  6 14:35:46 2015
@@ -108,6 +108,10 @@ Upgrading from Solr 4.x
   then it should be backwards compatible but you'll get a deprecation warning on startup.  See
   SOLR-6797.
 
+* The <nrtMode> configuration in solrconfig.xml has been discontinued and should be removed from
+  solrconfig.xml. Solr defaults to using NRT searchers regardless of the value in configuration
+  and a warning is logged on startup if the solrconfig.xml has <nrtMode> specified.
+
 Detailed Change List
 ----------------------
 
@@ -584,6 +588,8 @@ Other Changes
 * SOLR-6905: Test pseudo-field retrieval in distributed search.
   (Ramkumar Aiyengar via shalin)
 
+* SOLR-6897: Nuke non-NRT mode from code and configuration. (Hossman, shalin)
+
 ==================  4.10.3 ==================
 
 Bug Fixes

Modified: lucene/dev/branches/branch_5x/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/contrib/analytics/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml Tue Jan  6 14:35:46 2015
@@ -35,7 +35,6 @@ A solrconfig.xml snippet containing inde
   <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
 
   <mergeScheduler class="${solr.tests.mergeScheduler}" />
-  <nrtMode>${solr.tests.nrtMode:true}</nrtMode>
 
   <writeLockTimeout>1000</writeLockTimeout>
   <commitLockTimeout>10000</commitLockTimeout>

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java Tue Jan  6 14:35:46 2015
@@ -20,6 +20,7 @@ package org.apache.solr.core;
 import org.apache.lucene.util.Version;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.update.SolrIndexConfig;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.common.util.XMLErrorLogger;
@@ -162,6 +163,21 @@ public class Config {
     }
   }
 
+  /*
+     * Assert that assertCondition is true.
+     * If not, prints reason as log warning.
+     * If failCondition is true, then throw exception instead of warning
+     */
+  public static void assertWarnOrFail(String reason, boolean assertCondition, boolean failCondition) {
+    if (assertCondition) {
+      return;
+    } else if (failCondition) {
+      throw new SolrException(SolrException.ErrorCode.FORBIDDEN, reason);
+    } else {
+      log.warn(reason);
+    }
+  }
+
   protected Properties getSubstituteProperties() {
     return loader.getCoreProperties();
   }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java Tue Jan  6 14:35:46 2015
@@ -60,9 +60,8 @@ public abstract class IndexReaderFactory
   /**
    * Creates a new IndexReader instance using the given IndexWriter.
    * <p>
-   * This is used for opening the initial reader in NRT mode ({@code nrtMode=true}
-   * in solrconfig.xml)
-   * 
+   * This is used for opening the initial reader in NRT mode
+   *
    * @param writer IndexWriter
    * @param core {@link SolrCore} instance where this reader will be used. NOTE:
    * this SolrCore instance may not be fully configured yet, but basic things like

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java Tue Jan  6 14:35:46 2015
@@ -205,7 +205,11 @@ public class SolrConfig extends Config i
       defaultIndexConfig = mainIndexConfig = null;
       indexConfigPrefix = "indexConfig";
     }
-    nrtMode = getBool(indexConfigPrefix+"/nrtMode", true);
+    assertWarnOrFail("The <nrtMode> config has been discontinued and NRT mode is always used by Solr." +
+            " This config will be removed in future versions.", getNode(indexConfigPrefix + "/nrtMode", false) != null,
+        false
+    );
+
     // Parse indexConfig section, using mainIndex as backup in case old config is used
     indexConfig = new SolrIndexConfig(this, "indexConfig", mainIndexConfig);
 
@@ -431,7 +435,6 @@ public class SolrConfig extends Config i
   public final int queryResultWindowSize;
   public final int queryResultMaxDocsCached;
   public final boolean enableLazyFieldLoading;
-  public final boolean nrtMode;
   // DocSet
   public final float hashSetInverseLoadFactor;
   public final int hashDocSetMaxSize;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java Tue Jan  6 14:35:46 2015
@@ -441,7 +441,7 @@ public final class SolrCore implements S
     solrCoreState.increfSolrCoreState();
     SolrCore currentCore;
     boolean indexDirChange = !getNewIndexDir().equals(getIndexDir());
-    if (indexDirChange || !coreConfig.getSolrConfig().nrtMode) {
+    if (indexDirChange) {
       // the directory is changing, don't pass on state
       currentCore = null;
     } else {
@@ -462,13 +462,6 @@ public final class SolrCore implements S
   }
 
 
-  // gets a non-caching searcher
-  public SolrIndexSearcher newSearcher(String name) throws IOException {
-    return new SolrIndexSearcher(this, getNewIndexDir(), getLatestSchema(), getSolrConfig().indexConfig, 
-                                 name, false, directoryFactory);
-  }
-
-
    private void initDirectoryFactory() {
     DirectoryFactory dirFactory;
     PluginInfo info = solrConfig.getPluginInfo(DirectoryFactory.class.getName());
@@ -852,13 +845,7 @@ public final class SolrCore implements S
 
             @Override
             public DirectoryReader call() throws Exception {
-              if(getSolrConfig().nrtMode) {
-                // if in NRT mode, need to open from the previous writer
-                return indexReaderFactory.newReader(iw, core);
-              } else {
-                // if not NRT, need to create a new reader from the directory
-                return indexReaderFactory.newReader(iw.getDirectory(), core);
-              }
+              return indexReaderFactory.newReader(iw, core);
             }
           };
         }
@@ -1500,7 +1487,6 @@ public final class SolrCore implements S
 
     SolrIndexSearcher tmp;
     RefCounted<SolrIndexSearcher> newestSearcher = null;
-    boolean nrt = solrConfig.nrtMode && updateHandlerReopens;
 
     openSearcherLock.lock();
     try {
@@ -1509,7 +1495,7 @@ public final class SolrCore implements S
       String newIndexDirFile = null;
 
       // if it's not a normal near-realtime update, check that paths haven't changed.
-      if (!nrt) {
+      if (!updateHandlerReopens) {
         indexDirFile = getDirectoryFactory().normalize(getIndexDir());
         newIndexDirFile = getDirectoryFactory().normalize(newIndexDir);
       }
@@ -1521,7 +1507,7 @@ public final class SolrCore implements S
         }
       }
 
-      if (newestSearcher != null && (nrt || indexDirFile.equals(newIndexDirFile))) {
+      if (newestSearcher != null && (updateHandlerReopens || indexDirFile.equals(newIndexDirFile))) {
 
         DirectoryReader newReader;
         DirectoryReader currentReader = newestSearcher.get().getRawReader();
@@ -1531,12 +1517,11 @@ public final class SolrCore implements S
         RefCounted<IndexWriter> writer = getUpdateHandler().getSolrCoreState()
             .getIndexWriter(null);
         try {
-          if (writer != null && solrConfig.nrtMode) {
+          if (writer != null) {
             // if in NRT mode, open from the writer
             newReader = DirectoryReader.openIfChanged(currentReader, writer.get(), true);
           } else {
             // verbose("start reopen without writer, reader=", currentReader);
-            // if not in NRT mode, just re-open the reader
             newReader = DirectoryReader.openIfChanged(currentReader);
             // verbose("reopen result", newReader);
           }
@@ -1583,7 +1568,7 @@ public final class SolrCore implements S
           DirectoryReader newReader = newReaderCreator.call();
           tmp = new SolrIndexSearcher(this, newIndexDir, getLatestSchema(), 
               (realtime ? "realtime":"main"), newReader, true, !realtime, true, directoryFactory);
-        } else if (solrConfig.nrtMode) {
+        } else  {
           RefCounted<IndexWriter> writer = getUpdateHandler().getSolrCoreState().getIndexWriter(this);
           DirectoryReader newReader = null;
           try {
@@ -1593,12 +1578,6 @@ public final class SolrCore implements S
           }
           tmp = new SolrIndexSearcher(this, newIndexDir, getLatestSchema(),
               (realtime ? "realtime":"main"), newReader, true, !realtime, true, directoryFactory);
-        } else {
-         // normal open that happens at startup
-        // verbose("non-reopen START:");
-        tmp = new SolrIndexSearcher(this, newIndexDir, getLatestSchema(), getSolrConfig().indexConfig,
-                                    "main", true, directoryFactory);
-        // verbose("non-reopen DONE: searcher=",tmp);
         }
       }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Tue Jan  6 14:35:46 2015
@@ -20,10 +20,7 @@ package org.apache.solr.update;
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.PrintStreamInfoStream;
 import org.apache.lucene.util.Version;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.MapSerializable;
@@ -34,12 +31,11 @@ import org.apache.solr.util.SolrPluginUt
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.PrintStream;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.solr.core.Config.assertWarnOrFail;
+
 /**
  * This config object encapsulates IndexWriter config params,
  * defined in the &lt;indexConfig&gt; section of solrconfig.xml
@@ -127,13 +123,13 @@ public class SolrIndexConfig implements
     // Assert that end-of-life parameters or syntax is not in our config.
     // Warn for luceneMatchVersion's before LUCENE_3_6, fail fast above
     assertWarnOrFail("The <mergeScheduler>myclass</mergeScheduler> syntax is no longer supported in solrconfig.xml. Please use syntax <mergeScheduler class=\"myclass\"/> instead.",
-        !((solrConfig.getNode(prefix+"/mergeScheduler",false) != null) && (solrConfig.get(prefix+"/mergeScheduler/@class",null) == null)),
+        !((solrConfig.getNode(prefix + "/mergeScheduler", false) != null) && (solrConfig.get(prefix + "/mergeScheduler/@class", null) == null)),
         true);
     assertWarnOrFail("The <mergePolicy>myclass</mergePolicy> syntax is no longer supported in solrconfig.xml. Please use syntax <mergePolicy class=\"myclass\"/> instead.",
-        !((solrConfig.getNode(prefix+"/mergePolicy",false) != null) && (solrConfig.get(prefix+"/mergePolicy/@class",null) == null)),
+        !((solrConfig.getNode(prefix + "/mergePolicy", false) != null) && (solrConfig.get(prefix + "/mergePolicy/@class", null) == null)),
         true);
     assertWarnOrFail("The <luceneAutoCommit>true|false</luceneAutoCommit> parameter is no longer valid in solrconfig.xml.",
-        solrConfig.get(prefix+"/luceneAutoCommit", null) == null,
+        solrConfig.get(prefix + "/luceneAutoCommit", null) == null,
         true);
 
     defaultMergePolicyClassName = def.defaultMergePolicyClassName;
@@ -167,13 +163,10 @@ public class SolrIndexConfig implements
       }
     }
     mergedSegmentWarmerInfo = getPluginInfo(prefix + "/mergedSegmentWarmer", solrConfig, def.mergedSegmentWarmerInfo);
-    if (mergedSegmentWarmerInfo != null && solrConfig.nrtMode == false) {
-      throw new IllegalArgumentException("Supplying a mergedSegmentWarmer will do nothing since nrtMode is false");
-    }
 
     assertWarnOrFail("Begining with Solr 5.0, <checkIntegrityAtMerge> option is no longer supported and should be removed from solrconfig.xml (these integrity checks are now automatic)",
-                     (null == solrConfig.getNode(prefix+"/checkIntegrityAtMerge",false)),
-                     false);
+        (null == solrConfig.getNode(prefix + "/checkIntegrityAtMerge", false)),
+        false);
   }
   @Override
   public Map<String, Object> toMap() {
@@ -189,21 +182,6 @@ public class SolrIndexConfig implements
     return m;
   }
 
-  /*
-   * Assert that assertCondition is true.
-   * If not, prints reason as log warning.
-   * If failCondition is true, then throw exception instead of warning 
-   */
-  private void assertWarnOrFail(String reason, boolean assertCondition, boolean failCondition) {
-    if(assertCondition) {
-      return;
-    } else if(failCondition) {
-      throw new SolrException(ErrorCode.FORBIDDEN, reason);
-    } else {
-      log.warn(reason);
-    }
-  }
-
   private PluginInfo getPluginInfo(String path, SolrConfig solrConfig, PluginInfo def)  {
     List<PluginInfo> l = solrConfig.readPluginInfos(path, false, true);
     return l.isEmpty() ? def : l.get(0);

Modified: lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml Tue Jan  6 14:35:46 2015
@@ -35,7 +35,6 @@ A solrconfig.xml snippet containing inde
   <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
 
   <mergeScheduler class="${solr.tests.mergeScheduler}" />
-  <nrtMode>${solr.tests.nrtMode:true}</nrtMode>
 
   <writeLockTimeout>1000</writeLockTimeout>
   <commitLockTimeout>10000</commitLockTimeout>

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java Tue Jan  6 14:35:46 2015
@@ -54,14 +54,12 @@ public class TestArbitraryIndexDir exten
   @BeforeClass
   public static void beforeClass() {
     // this test wants to start solr, and then open a separate indexwriter of its own on the same dir.
-    System.setProperty("solr.tests.nrtMode", "false");
     System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
     savedFactory = System.getProperty("solr.DirectoryFactory");
     System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockFSDirectoryFactory");
   }
   @AfterClass
   public static void afterClass() {
-    System.clearProperty("solr.tests.nrtMode");
     if (savedFactory == null) {
       System.clearProperty("solr.directoryFactory");
     } else {
@@ -125,7 +123,7 @@ public class TestArbitraryIndexDir exten
     iw.close();
 
     //commit will cause searcher to open with the new index dir
-    assertU(commit());
+    assertU(commit());h.getCoreContainer().reload(h.getCore().getName());
     //new index dir contains just 1 doc.
     assertQ("return doc with id 2",
         req("id:2"),

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestBadConfig.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestBadConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestBadConfig.java Tue Jan  6 14:35:46 2015
@@ -27,10 +27,6 @@ public class TestBadConfig extends Abstr
     assertConfigs("bad_solrconfig.xml","schema.xml","unset.sys.property");
   }
 
-  public void testSegmentMergerWithoutReopen() throws Exception {
-      assertConfigs("bad-solrconfig-warmer-no-reopen.xml", "schema12.xml",
-                    "mergedSegmentWarmer");
-  }
   public void testMultipleDirectoryFactories() throws Exception {
       assertConfigs("bad-solrconfig-multiple-dirfactory.xml", "schema12.xml",
                     "directoryFactory");

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java Tue Jan  6 14:35:46 2015
@@ -346,46 +346,7 @@ public class TestReplicationHandler exte
 
 
   /**
-   * Verify that things still work if an IW has not been opened (and hence the CommitPoints have not been communicated to the deletion policy)
-   */
-  public void testNoWriter() throws Exception {
-    useFactory(null);    // force a persistent directory
-
-    // read-only setting (no opening from indexwriter)
-    System.setProperty("solr.tests.nrtMode", "false");
-    try {
-    // stop and start so they see the new directory setting
-    slaveJetty.stop();
-    masterJetty.stop();
-    slaveJetty.start(true);
-    masterJetty.start(true);
-
-    index(slaveClient, "id", "123456");
-    slaveClient.commit();
-    slaveJetty.stop();
-    slaveJetty.start(true);
-    } finally {
-      System.clearProperty("solr.tests.nrtMode"); // dont mess with other tests
-    }
-
-    // Currently we open a writer on-demand.  This is to test that we are correctly testing
-    // the code path when SolrDeletionPolicy.getLatestCommit() returns null.
-    // When we are using an ephemeral directory, an IW will always be opened to create the index and hence
-    // getLatestCommit will always be non-null.
-    CoreContainer cores = ((SolrDispatchFilter) slaveJetty.getDispatchFilter().getFilter()).getCores();
-    Collection<SolrCore> theCores = cores.getCores();
-    assertEquals(1, theCores.size());
-    SolrCore core = (SolrCore)theCores.toArray()[0];
-    assertNull( core.getDeletionPolicy().getLatestCommit() );
-
-
-    pullFromMasterToSlave();  // this will cause SnapPuller to be invoked and we will test when SolrDeletionPolicy.getLatestCommit() returns null
-
-    resetFactory();
-  }
-
-  /**
-   * Verify that empty commits and/or commits with openSearcher=false 
+   * Verify that empty commits and/or commits with openSearcher=false
    * on the master do not cause subsequent replication problems on the slave 
    */
   public void testEmptyCommits() throws Exception {

Modified: lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/db/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/db/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/db/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/db/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -276,15 +276,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/mail/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/mail/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/mail/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -279,15 +279,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/rss/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/rss/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/rss/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -276,15 +276,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/solr/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/solr/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -276,15 +276,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/tika/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/tika/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/example/example-DIH/solr/tika/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -277,15 +277,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -256,15 +256,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.

Modified: lucene/dev/branches/branch_5x/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml?rev=1649812&r1=1649811&r2=1649812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_5x/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml Tue Jan  6 14:35:46 2015
@@ -274,15 +274,6 @@
     <unlockOnStartup>false</unlockOnStartup>
       -->
 
-    <!-- If true, IndexReaders will be opened/reopened from the IndexWriter
-         instead of from the Directory. Hosts in a master/slave setup
-         should have this set to false while those in a SolrCloud
-         cluster need to be set to true. Default: true
-      -->
-    <!-- 
-    <nrtMode>true</nrtMode>
-      -->
-
     <!-- Commit Deletion Policy
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.