You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2018/02/16 18:37:35 UTC

[1/2] lucene-solr:master: SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 05e920104 -> 9876e8832
  refs/heads/master 32f3570f0 -> ee51b658e


SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives


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

Branch: refs/heads/master
Commit: ee51b658ece5b23431a2200e763f5198b53952fa
Parents: 32f3570
Author: Chris Hostetter <ho...@apache.org>
Authored: Fri Feb 16 11:28:09 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Fri Feb 16 11:28:09 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/core/DirectoryFactory.java  |   9 +-
 .../solr/core/EphemeralDirectoryFactory.java    |  16 +--
 .../apache/solr/core/TestDirectoryFactory.java  | 106 +++++++++++++++++++
 4 files changed, 124 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ee51b658/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b173872..ea03776 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -210,6 +210,8 @@ Bug Fixes
 
 * SOLR-11739: Fix race condition that made Solr accept duplicate async IDs in collection API operations (Tomás Fernánadez Löbbe)
 
+* SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives (hossman)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ee51b658/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index e7e33a5..fab3300 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -109,9 +109,14 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
   protected abstract LockFactory createLockFactory(String rawLockType) throws IOException;
   
   /**
-   * Returns true if a Directory exists for a given path.
+   * Returns true if a Directory exists for a given path in the underlying (stable) storage <em>and</em> 
+   * contains at least one file.  
+   * Note that the existence of a {@link Directory} <em>Object</em> as returned by a previous call to the 
+   * {@link #get} method (on the specified <code>path</code>) is not enough to cause this method to return 
+   * true.  Some prior user of that Directory must have written &amp; synced at least one file to that 
+   * Directory (and at least one file must still exist)
+   *
    * @throws IOException If there is a low-level I/O error.
-   * 
    */
   public abstract boolean exists(String path) throws IOException;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ee51b658/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
index d5bcbb8..c7708ea 100644
--- a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
@@ -33,16 +33,18 @@ public abstract class EphemeralDirectoryFactory extends CachingDirectoryFactory
   public boolean exists(String path) throws IOException {
     String fullPath = normalize(path);
     synchronized (this) {
-      CacheValue cacheValue = byPathCache.get(fullPath);
-      Directory directory = null;
-      if (cacheValue != null) {
-        directory = cacheValue.directory;
+      final CacheValue cacheValue = byPathCache.get(fullPath);
+      if (null == cacheValue) {
+        return false;
       }
-      if (directory == null) {
+      final Directory directory = cacheValue.directory;
+      if (null == directory) {
         return false;
-      } else {
-        return true;
       }
+      if (0 < directory.listAll().length) {
+        return true;
+      } 
+      return false;
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ee51b658/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java b/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
new file mode 100644
index 0000000..0eccb6a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.core;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.IOContext;
+
+import org.apache.solr.common.util.NamedList;
+
+import org.apache.solr.SolrTestCaseJ4;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Test some expected equivilencies of all DirectoryFactory implementations.
+ * <p>
+ * TODO: test more methods besides exists(String)
+ * </p>
+ */
+public class TestDirectoryFactory extends SolrTestCaseJ4 {
+
+  // TODO: what do we need to setup to be able to test HdfsDirectoryFactory?
+  public static final List<Class<? extends DirectoryFactory>> ALL_CLASSES
+    = Arrays.asList(MMapDirectoryFactory.class,
+                    MockDirectoryFactory.class,
+                    MockFSDirectoryFactory.class,
+                    NRTCachingDirectoryFactory.class,
+                    NIOFSDirectoryFactory.class,
+                    RAMDirectoryFactory.class,
+                    SimpleFSDirectoryFactory.class,
+                    StandardDirectoryFactory.class);
+  
+  /* Test that MockDirectoryFactory's exist method behaves consistent w/other impls */
+  public void testExistsEquivilence() throws Exception {
+    // TODO: ideally we'd init all of these using DirectoryFactory.loadDirectoryFactory() ...
+    // ...but the scaffolding needed for dealing with the CoreContainer/SolrConfig is a PITA
+
+    for (Class<? extends DirectoryFactory> clazz : ALL_CLASSES) {
+      testExistsBehavior(clazz);
+    }
+  }
+
+  private void testExistsBehavior(Class<? extends DirectoryFactory> clazz) throws Exception {
+    final String path = createTempDir().toString() + "/" + clazz + "_somedir";
+    DirectoryFactory dirFac = null;
+    try {
+      dirFac = clazz.newInstance();
+      dirFac.initCoreContainer(null); // greybox testing directly against path
+      dirFac.init(new NamedList());
+
+      assertFalse(path + " should not exist yet", dirFac.exists(path));
+      Directory dir = dirFac.get(path, DirectoryFactory.DirContext.DEFAULT,
+                                 DirectoryFactory.LOCK_TYPE_SINGLE);
+      try {
+        assertFalse(path + " should still not exist", dirFac.exists(path));
+        try (IndexOutput file = dir.createOutput("test_file", IOContext.DEFAULT)) {
+          file.writeInt(42);
+
+          // TODO: even StandardDirectoryFactory & NRTCachingDirectoryFactory can't agree on this...
+          // ... should we consider this explicitly undefinied?
+          // ... or should *all* Caching DirFactories consult the cache as well as the disk itself?
+          // assertFalse(path + " should still not exist until file is closed", dirFac.exists(path));
+          
+        } // implicitly close file...
+        
+        // TODO: even StandardDirectoryFactory & NRTCachingDirectoryFactory can't agree on this...
+        // ... should we consider this explicitly undefinied?
+        // ... or should *all* Caching DirFactories consult the cache as well as the disk itself?
+        // assertTrue(path + " should exist once file is closed", dirFac.exists(path));
+        
+        dir.sync(Collections.singleton("test_file"));
+        assertTrue(path + " should exist once file is synced", dirFac.exists(path));
+
+        
+      } finally {
+        dirFac.release(dir);
+      }
+      assertTrue(path + " should still exist even after being released", dirFac.exists(path));
+      
+    } catch (AssertionError ae) {
+      throw new AssertionError(clazz + ": " + ae.getMessage());
+    } finally {
+      if (null != dirFac) {
+        dirFac.close();
+      }
+    }
+  }
+}


[2/2] lucene-solr:branch_7x: SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives

Posted by ho...@apache.org.
SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives

(cherry picked from commit ee51b658ece5b23431a2200e763f5198b53952fa)


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

Branch: refs/heads/branch_7x
Commit: 9876e8832ad40288e8a852f1594e520495da2cfa
Parents: 05e9201
Author: Chris Hostetter <ho...@apache.org>
Authored: Fri Feb 16 11:28:09 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Fri Feb 16 11:28:21 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/core/DirectoryFactory.java  |   9 +-
 .../solr/core/EphemeralDirectoryFactory.java    |  16 +--
 .../apache/solr/core/TestDirectoryFactory.java  | 106 +++++++++++++++++++
 4 files changed, 124 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9876e883/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 11ac84b..948ddf7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -190,6 +190,8 @@ Bug Fixes
 
 * SOLR-11739: Fix race condition that made Solr accept duplicate async IDs in collection API operations (Tomás Fernánadez Löbbe)
 
+* SOLR-11988: Fix exists() method in EphemeralDirectoryFactory/MockDirectoryFactory to prevent false positives (hossman)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9876e883/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index e7e33a5..fab3300 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -109,9 +109,14 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
   protected abstract LockFactory createLockFactory(String rawLockType) throws IOException;
   
   /**
-   * Returns true if a Directory exists for a given path.
+   * Returns true if a Directory exists for a given path in the underlying (stable) storage <em>and</em> 
+   * contains at least one file.  
+   * Note that the existence of a {@link Directory} <em>Object</em> as returned by a previous call to the 
+   * {@link #get} method (on the specified <code>path</code>) is not enough to cause this method to return 
+   * true.  Some prior user of that Directory must have written &amp; synced at least one file to that 
+   * Directory (and at least one file must still exist)
+   *
    * @throws IOException If there is a low-level I/O error.
-   * 
    */
   public abstract boolean exists(String path) throws IOException;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9876e883/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
index d5bcbb8..c7708ea 100644
--- a/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
@@ -33,16 +33,18 @@ public abstract class EphemeralDirectoryFactory extends CachingDirectoryFactory
   public boolean exists(String path) throws IOException {
     String fullPath = normalize(path);
     synchronized (this) {
-      CacheValue cacheValue = byPathCache.get(fullPath);
-      Directory directory = null;
-      if (cacheValue != null) {
-        directory = cacheValue.directory;
+      final CacheValue cacheValue = byPathCache.get(fullPath);
+      if (null == cacheValue) {
+        return false;
       }
-      if (directory == null) {
+      final Directory directory = cacheValue.directory;
+      if (null == directory) {
         return false;
-      } else {
-        return true;
       }
+      if (0 < directory.listAll().length) {
+        return true;
+      } 
+      return false;
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9876e883/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java b/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
new file mode 100644
index 0000000..0eccb6a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestDirectoryFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.core;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.IOContext;
+
+import org.apache.solr.common.util.NamedList;
+
+import org.apache.solr.SolrTestCaseJ4;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Test some expected equivilencies of all DirectoryFactory implementations.
+ * <p>
+ * TODO: test more methods besides exists(String)
+ * </p>
+ */
+public class TestDirectoryFactory extends SolrTestCaseJ4 {
+
+  // TODO: what do we need to setup to be able to test HdfsDirectoryFactory?
+  public static final List<Class<? extends DirectoryFactory>> ALL_CLASSES
+    = Arrays.asList(MMapDirectoryFactory.class,
+                    MockDirectoryFactory.class,
+                    MockFSDirectoryFactory.class,
+                    NRTCachingDirectoryFactory.class,
+                    NIOFSDirectoryFactory.class,
+                    RAMDirectoryFactory.class,
+                    SimpleFSDirectoryFactory.class,
+                    StandardDirectoryFactory.class);
+  
+  /* Test that MockDirectoryFactory's exist method behaves consistent w/other impls */
+  public void testExistsEquivilence() throws Exception {
+    // TODO: ideally we'd init all of these using DirectoryFactory.loadDirectoryFactory() ...
+    // ...but the scaffolding needed for dealing with the CoreContainer/SolrConfig is a PITA
+
+    for (Class<? extends DirectoryFactory> clazz : ALL_CLASSES) {
+      testExistsBehavior(clazz);
+    }
+  }
+
+  private void testExistsBehavior(Class<? extends DirectoryFactory> clazz) throws Exception {
+    final String path = createTempDir().toString() + "/" + clazz + "_somedir";
+    DirectoryFactory dirFac = null;
+    try {
+      dirFac = clazz.newInstance();
+      dirFac.initCoreContainer(null); // greybox testing directly against path
+      dirFac.init(new NamedList());
+
+      assertFalse(path + " should not exist yet", dirFac.exists(path));
+      Directory dir = dirFac.get(path, DirectoryFactory.DirContext.DEFAULT,
+                                 DirectoryFactory.LOCK_TYPE_SINGLE);
+      try {
+        assertFalse(path + " should still not exist", dirFac.exists(path));
+        try (IndexOutput file = dir.createOutput("test_file", IOContext.DEFAULT)) {
+          file.writeInt(42);
+
+          // TODO: even StandardDirectoryFactory & NRTCachingDirectoryFactory can't agree on this...
+          // ... should we consider this explicitly undefinied?
+          // ... or should *all* Caching DirFactories consult the cache as well as the disk itself?
+          // assertFalse(path + " should still not exist until file is closed", dirFac.exists(path));
+          
+        } // implicitly close file...
+        
+        // TODO: even StandardDirectoryFactory & NRTCachingDirectoryFactory can't agree on this...
+        // ... should we consider this explicitly undefinied?
+        // ... or should *all* Caching DirFactories consult the cache as well as the disk itself?
+        // assertTrue(path + " should exist once file is closed", dirFac.exists(path));
+        
+        dir.sync(Collections.singleton("test_file"));
+        assertTrue(path + " should exist once file is synced", dirFac.exists(path));
+
+        
+      } finally {
+        dirFac.release(dir);
+      }
+      assertTrue(path + " should still exist even after being released", dirFac.exists(path));
+      
+    } catch (AssertionError ae) {
+      throw new AssertionError(clazz + ": " + ae.getMessage());
+    } finally {
+      if (null != dirFac) {
+        dirFac.close();
+      }
+    }
+  }
+}