You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2017/12/04 17:57:31 UTC

lucene-solr:master: SOLR-11687: SolrCore.getNewIndexDir falsely returns {dataDir}/index on any IOException reading index.properties

Repository: lucene-solr
Updated Branches:
  refs/heads/master bdaf1baa8 -> 929ce7ca3


SOLR-11687: SolrCore.getNewIndexDir falsely returns {dataDir}/index on any IOException reading index.properties


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/929ce7ca
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/929ce7ca
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/929ce7ca

Branch: refs/heads/master
Commit: 929ce7ca30d5b4f9d08aee604ba5184f0d94d505
Parents: bdaf1ba
Author: Erick <er...@apache.org>
Authored: Mon Dec 4 09:56:58 2017 -0800
Committer: Erick <er...@apache.org>
Committed: Mon Dec 4 09:56:58 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../src/java/org/apache/solr/core/SolrCore.java | 113 +++++++++++--------
 2 files changed, 69 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/929ce7ca/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b20c589..7e36a96 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -159,6 +159,9 @@ Bug Fixes
 * SOLR-11616: Snapshot the segments more robustly such that segments created during a backup does does not fail the
   operation (Varun Thacker)
 
+* SOLR-11687: SolrCore.getNewIndexDir falsely returns {dataDir}/index on any IOException reading index.properties
+  (Nikolay Martynov, Erick Erickson)
+
 Optimizations
 ----------------------
 * SOLR-11285: Refactor autoscaling framework to avoid direct references to Zookeeper and Solr

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/929ce7ca/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index e650519..f1bae42 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -78,6 +78,7 @@ import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
@@ -337,42 +338,28 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   /**
    * Returns the indexdir as given in index.properties. If index.properties exists in dataDir and
    * there is a property <i>index</i> available and it points to a valid directory
-   * in dataDir that is returned Else dataDir/index is returned. Only called for creating new indexSearchers
+   * in dataDir that is returned. Else dataDir/index is returned. Only called for creating new indexSearchers
    * and indexwriters. Use the getIndexDir() method to know the active index directory
    *
    * @return the indexdir as given in index.properties
+   *
+   * @throws SolrException if for any reason the a reasonable index directory cannot be determined.
    */
   public String getNewIndexDir() {
-    String result = dataDir + "index/";
-    Properties p = new Properties();
     Directory dir = null;
     try {
       dir = getDirectoryFactory().get(getDataDir(), DirContext.META_DATA, getSolrConfig().indexConfig.lockType);
-      IndexInput input;
-      try {
-        input = dir.openInput(IndexFetcher.INDEX_PROPERTIES, IOContext.DEFAULT);
-      } catch (FileNotFoundException | NoSuchFileException e) {
-        input = null;
-      }
-
-      if (input != null) {
-        final InputStream is = new PropertiesInputStream(input);
-        try {
-          p.load(new InputStreamReader(is, StandardCharsets.UTF_8));
-
-          String s = p.getProperty("index");
-          if (s != null && s.trim().length() > 0) {
-              result = dataDir + s;
-          }
-
-        } catch (Exception e) {
-          log.error("Unable to load " + IndexFetcher.INDEX_PROPERTIES, e);
-        } finally {
-          IOUtils.closeQuietly(is);
-        }
+      String result = getIndexPropertyFromPropFile(dir);
+      if (!result.equals(lastNewIndexDir)) {
+        log.debug("New index directory detected: old=" + lastNewIndexDir + " new=" + result);
       }
+      lastNewIndexDir = result;
+      return result;
     } catch (IOException e) {
       SolrException.log(log, "", e);
+      // See SOLR-11687. It is inadvisable to assume we can do the right thing for any but a small
+      // number of exceptions that ware caught and swallowed in getIndexProperty.
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error in getNewIndexDir, exception: ", e);
     } finally {
       if (dir != null) {
         try {
@@ -382,12 +369,44 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         }
       }
     }
-    if (!result.equals(lastNewIndexDir)) {
-      log.debug("New index directory detected: old="+lastNewIndexDir + " new=" + result);
+  }
+
+  // This is guaranteed to return a string or throw an exception.
+  //
+  // NOTE: Not finding the index.properties file is normal.
+  //
+  // We return dataDir/index if there is an index.properties file with no value for "index"
+  // See SOLR-11687
+  //
+
+  private String getIndexPropertyFromPropFile(Directory dir) throws IOException {
+    IndexInput input;
+    try {
+      input = dir.openInput(IndexFetcher.INDEX_PROPERTIES, IOContext.DEFAULT);
+    } catch (FileNotFoundException | NoSuchFileException e) {
+      // Swallow this error, dataDir/index is the right thing to return
+      // if there is no index.properties file
+      // All other exceptions are will propagate to caller.
+      return dataDir + "index/";
+    }
+    final InputStream is = new PropertiesInputStream(input); // c'tor just assigns a variable here, no exception thrown.
+    try {
+      Properties p = new Properties();
+      p.load(new InputStreamReader(is, StandardCharsets.UTF_8));
+
+      String s = p.getProperty("index");
+      if (s != null && s.trim().length() > 0) {
+        return dataDir + s.trim();
+      }
+
+      // We'll return dataDir/index/ if the properties file has an "index" property with
+      // no associated value or does not have an index property at all.
+      return dataDir + "index/";
+    } finally {
+      IOUtils.closeQuietly(is);
     }
-    lastNewIndexDir = result;
-    return result;
   }
+
   private String lastNewIndexDir; // for debugging purposes only... access not synchronized, but that's ok
 
 
@@ -785,7 +804,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         throw inner;
       }
 
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error Instantiating " + msg + ", " + className + " failed to instantiate " + cast.getName(), e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error Instantiating " + msg + ", " + className + " failed to instantiate " + cast.getName(), e);
     }
   }
 
@@ -803,7 +822,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
             return UpdateHandler.class.cast(con.newInstance(this, updateHandler));
           }
         }
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " could not find proper constructor for " + UpdateHandler.class.getName());
+        throw new SolrException(ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " could not find proper constructor for " + UpdateHandler.class.getName());
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
@@ -814,7 +833,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         throw inner;
       }
 
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " failed to instantiate " + UpdateHandler.class.getName(), e);
+      throw new SolrException(ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " failed to instantiate " + UpdateHandler.class.getName(), e);
     }
   }
 
@@ -986,7 +1005,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         log.error("Error while closing", t);
       }
 
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e.getMessage(), e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, e.getMessage(), e);
     } finally {
       // allow firstSearcher events to fire and make sure it is released
       latch.countDown();
@@ -1188,7 +1207,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       try {
         VersionInfo.getAndCheckVersionField(schema);
       } catch (SolrException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+        throw new SolrException(ErrorCode.SERVER_ERROR,
                                 "Schema will not work with SolrCloud mode: " +
                                 e.getMessage(), e);
       }
@@ -1227,7 +1246,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
             dataDir = directoryFactory.getDataHome(coreDescriptor);
           }
         } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+          throw new SolrException(ErrorCode.SERVER_ERROR, e);
         }
       }
     }
@@ -1302,7 +1321,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       p.store(os, IndexFetcher.INDEX_PROPERTIES);
       dir.sync(Collections.singleton(tmpFileName));
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to write " + IndexFetcher.INDEX_PROPERTIES, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to write " + IndexFetcher.INDEX_PROPERTIES, e);
     } finally {
       IOUtils.closeQuietly(os);
     }
@@ -1330,7 +1349,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         Thread.sleep(milliSleep);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+        throw new SolrException(ErrorCode.SERVER_ERROR,
             "Caught InterruptedException whilst waiting for core " + getName() + " to close: "
                 + e.getMessage(), e);
       }
@@ -1360,12 +1379,12 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         if (null != ft.getPostingsFormat()) {
           String msg = "FieldType '" + ft.getTypeName() + "' is configured with a postings format, but the codec does not support it: " + factory.getClass();
           log.error(msg);
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
         }
         if (null != ft.getDocValuesFormat()) {
           String msg = "FieldType '" + ft.getTypeName() + "' is configured with a docValues format, but the codec does not support it: " + factory.getClass();
           log.error(msg);
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
         }
       }
     }
@@ -1428,7 +1447,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   {
     UpdateRequestProcessorChain chain = updateProcessorChains.get( name );
     if( chain == null ) {
-      throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
+      throw new SolrException( ErrorCode.BAD_REQUEST,
           "unknown UpdateRequestProcessorChain: "+name );
     }
     return chain;
@@ -1947,7 +1966,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
    */
   public RefCounted<SolrIndexSearcher>  openNewSearcher(boolean updateHandlerReopens, boolean realtime) {
     if (isClosed()) { // catch some errors quicker
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
+      throw new SolrException(ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
     }
 
     SolrIndexSearcher tmp;
@@ -2060,7 +2079,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         if (isClosed()) {
           newSearcher.decref(); // once for caller since we're not returning it
           newSearcher.decref(); // once for ourselves since it won't be "replaced"
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
+          throw new SolrException(ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
         }
 
         if (realtimeSearcher != null) {
@@ -2073,7 +2092,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       return newSearcher;
 
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error opening new searcher", e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error opening new searcher", e);
     }
     finally {
       openSearcherLock.unlock();
@@ -2321,7 +2340,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
     } catch (Exception e) {
       if (e instanceof SolrException) throw (SolrException)e;
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
     } finally {
 
       timerContext.close();
@@ -2466,7 +2485,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
       if (log.isWarnEnabled()) log.warn(logid + msg + ":" + req);
 
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, msg);
+      throw new SolrException(ErrorCode.BAD_REQUEST, msg);
     }
 
     preDecorateResponse(req, rsp);
@@ -2552,7 +2571,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
     if( ep != null ) {
       EchoParamStyle echoParams = EchoParamStyle.get( ep );
       if( echoParams == null ) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid value '" + ep + "' for " + CommonParams.HEADER_ECHO_PARAMS
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid value '" + ep + "' for " + CommonParams.HEADER_ECHO_PARAMS
             + " parameter, use '" + EchoParamStyle.EXPLICIT + "' or '" + EchoParamStyle.ALL + "'" );
       }
       if( echoParams == EchoParamStyle.EXPLICIT ) {
@@ -2674,7 +2693,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         result.put(e.getKey(), (T) o);
       } catch (Exception exp) {
         //should never happen
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to instantiate class", exp);
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to instantiate class", exp);
       }
     }
     return result;