You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by pa...@apache.org on 2022/04/16 00:28:57 UTC

[lucene] branch branch_9x updated: LUCENE-10482 Allow users to create their own DirectoryTaxonomyReaders with empty taxoArrays instead of letting the taxoEpoch decide (#762) (#813)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 5c2cbd71259 LUCENE-10482 Allow users to create their own DirectoryTaxonomyReaders with empty taxoArrays instead of letting the taxoEpoch decide (#762) (#813)
5c2cbd71259 is described below

commit 5c2cbd712590fe98d101f30b528bee976af173ee
Author: Gautam Worah <wo...@gmail.com>
AuthorDate: Fri Apr 15 17:28:50 2022 -0700

    LUCENE-10482 Allow users to create their own DirectoryTaxonomyReaders with empty taxoArrays instead of letting the taxoEpoch decide (#762) (#813)
---
 lucene/CHANGES.txt                                 |   4 +
 .../directory/DirectoryTaxonomyReader.java         |  20 ++-
 .../TestAlwaysRefreshDirectoryTaxonomyReader.java  | 194 +++++++++++++++++++++
 3 files changed, 213 insertions(+), 5 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c7c9cd78dc0..065ff6ac558 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,6 +10,10 @@ API Changes
 
 * LUCENE-10325: Facets API extended to support getTopFacets. (Yuting Gan)
 
+* LUCENE-10482: Allow users to create their own DirectoryTaxonomyReaders with empty taxoArrays instead of letting the
+  taxoEpoch decide. Add a test case that demonstrates the inconsistencies caused when you reuse taxoArrays on older
+  checkpoints. (Gautam Worah)
+
 New Features
 ---------------------
 
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
index 33fee851840..51ec0ef095d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
@@ -79,10 +79,20 @@ public class DirectoryTaxonomyReader extends TaxonomyReader implements Accountab
   private volatile TaxonomyIndexArrays taxoArrays;
 
   /**
-   * Called only from {@link #doOpenIfChanged()}. If the taxonomy has been recreated, you should
-   * pass {@code null} as the caches and parent/children arrays.
+   * Expert: Use this method to explicitly force the {@link DirectoryTaxonomyReader} to use specific
+   * parent/children arrays and caches.
+   *
+   * <p>Called from {@link #doOpenIfChanged()}. If the taxonomy has been recreated, you should pass
+   * {@code null} as the caches and parent/children arrays.
+   *
+   * @param indexReader An indexReader that is opened in the desired Directory
+   * @param taxoWriter The {@link DirectoryTaxonomyWriter} from which to obtain newly added
+   *     categories, in real-time.
+   * @param ordinalCache a FacetLabel to Integer ordinal mapping if it already exists
+   * @param categoryCache an ordinal to FacetLabel mapping if it already exists
+   * @param taxoArrays taxonomy arrays that store the parent, siblings, children information
    */
-  DirectoryTaxonomyReader(
+  protected DirectoryTaxonomyReader(
       DirectoryReader indexReader,
       DirectoryTaxonomyWriter taxoWriter,
       LRUHashMap<FacetLabel, Integer> ordinalCache,
@@ -208,7 +218,7 @@ public class DirectoryTaxonomyReader extends TaxonomyReader implements Accountab
 
       final DirectoryTaxonomyReader newtr;
       if (recreated) {
-        // if recreated, do not reuse anything from this instace. the information
+        // if recreated, do not reuse anything from this instance. the information
         // will be lazily computed by the new instance when needed.
         newtr = new DirectoryTaxonomyReader(r2, taxoWriter, null, null, null);
       } else {
@@ -239,7 +249,7 @@ public class DirectoryTaxonomyReader extends TaxonomyReader implements Accountab
    * Expert: returns the underlying {@link DirectoryReader} instance that is used by this {@link
    * TaxonomyReader}.
    */
-  DirectoryReader getInternalIndexReader() {
+  protected DirectoryReader getInternalIndexReader() {
     ensureOpen();
     return indexReader;
   }
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAlwaysRefreshDirectoryTaxonomyReader.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAlwaysRefreshDirectoryTaxonomyReader.java
new file mode 100644
index 00000000000..822500b3630
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAlwaysRefreshDirectoryTaxonomyReader.java
@@ -0,0 +1,194 @@
+/*
+ * 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.lucene.facet.taxonomy.directory;
+
+import static com.carrotsearch.randomizedtesting.RandomizedTest.sleep;
+import static org.apache.lucene.tests.mockfile.ExtrasFS.isExtra;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.time.Instant;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.util.IOUtils;
+
+// Nefarious FS will delay/stop deletion of index files and this test specifically does that
+@LuceneTestCase.SuppressFileSystems({"WindowsFS", "VirusCheckingFS"})
+public class TestAlwaysRefreshDirectoryTaxonomyReader extends FacetTestCase {
+
+  /**
+   * Tests the behavior of the {@link AlwaysRefreshDirectoryTaxonomyReader} by testing if the
+   * associated {@link SearcherTaxonomyManager} can successfully refresh and serve queries if the
+   * underlying taxonomy index is changed to an older checkpoint. Ideally, each checkpoint should be
+   * self-sufficient and should allow serving search queries when {@link
+   * SearcherTaxonomyManager#maybeRefresh()} is called.
+   *
+   * <p>It does not check whether the private taxoArrays were actually recreated or no. We are
+   * (correctly) hiding away that complexity away from the user.
+   */
+  private <T extends Throwable> void testAlwaysRefreshDirectoryTaxonomyReader(
+      Function<Directory, DirectoryTaxonomyReader> dtrProducer, Class<T> exceptionType)
+      throws IOException {
+    final Path taxoPath1 = createTempDir(String.valueOf(Instant.now()));
+    final Directory dir1 = newFSDirectory(taxoPath1);
+
+    final DirectoryTaxonomyWriter tw1 =
+        new DirectoryTaxonomyWriter(dir1, IndexWriterConfig.OpenMode.CREATE);
+    tw1.addCategory(new FacetLabel("a"));
+    tw1.commit(); // commit1
+
+    final Path taxoPath2 = createTempDir(String.valueOf(Instant.now()));
+    final Directory commit1 = newFSDirectory(taxoPath2);
+    // copy all index files from dir1
+    for (String file : dir1.listAll()) {
+      if (isExtra(file) == false) {
+        // the test framework creates these devious extra files just to chaos test the edge cases
+        commit1.copyFrom(dir1, file, file, IOContext.READ);
+      }
+    }
+
+    tw1.addCategory(new FacetLabel("b"));
+    tw1.commit(); // commit2
+    tw1.close();
+
+    final DirectoryReader dr1 = DirectoryReader.open(dir1);
+    final DirectoryTaxonomyReader dtr1 = dtrProducer.apply(dir1);
+    final SearcherTaxonomyManager mgr = new SearcherTaxonomyManager(dr1, dtr1, null);
+
+    final FacetsConfig config = new FacetsConfig();
+    SearcherTaxonomyManager.SearcherAndTaxonomy pair = mgr.acquire();
+    final FacetsCollector sfc = new FacetsCollector();
+    /**
+     * the call flow here initializes {@link DirectoryTaxonomyReader#taxoArrays}. These reused
+     * `taxoArrays` form the basis of the inconsistency *
+     */
+    getTaxonomyFacetCounts(pair.taxonomyReader, config, sfc);
+
+    // now try to go back to checkpoint 1 and refresh the SearcherTaxonomyManager
+
+    // delete all files from commit2
+    for (String file : dir1.listAll()) {
+      dir1.deleteFile(file);
+    }
+
+    while (dir1.getPendingDeletions().isEmpty() == false) {
+      // make the test more robust to the OS taking more time to actually delete files
+      sleep(5);
+    }
+
+    // copy all index files from commit1
+    for (String file : commit1.listAll()) {
+      if (isExtra(file) == false) {
+        dir1.copyFrom(commit1, file, file, IOContext.READ);
+      }
+    }
+
+    if (exceptionType != null) {
+      expectThrows(exceptionType, mgr::maybeRefresh);
+    } else {
+      mgr.maybeRefresh();
+      pair = mgr.acquire();
+      assertEquals(new FacetLabel("a"), pair.taxonomyReader.getPath(1));
+      assertEquals(-1, pair.taxonomyReader.getOrdinal(new FacetLabel("b")));
+    }
+
+    mgr.release(pair);
+    IOUtils.close(mgr, dtr1, dr1);
+    // closing commit1 and dir1 throws exceptions because of checksum mismatches
+    IOUtils.deleteFiles(commit1, List.of(commit1.listAll()));
+    IOUtils.deleteFiles(dir1, List.of(dir1.listAll()));
+    IOUtils.close(commit1, dir1);
+  }
+
+  public void testAlwaysRefreshDirectoryTaxonomyReader() throws IOException {
+    testAlwaysRefreshDirectoryTaxonomyReader(
+        (dir) -> {
+          try {
+            return new DirectoryTaxonomyReader(dir);
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+          return null;
+        },
+        ArrayIndexOutOfBoundsException.class);
+    testAlwaysRefreshDirectoryTaxonomyReader(
+        (dir) -> {
+          try {
+            return new AlwaysRefreshDirectoryTaxonomyReader(dir);
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+          return null;
+        },
+        null);
+  }
+
+  /**
+   * A modified DirectoryTaxonomyReader that always recreates a new {@link
+   * AlwaysRefreshDirectoryTaxonomyReader} instance when {@link
+   * AlwaysRefreshDirectoryTaxonomyReader#doOpenIfChanged()} is called. This enables us to easily go
+   * forward or backward in time by re-computing the ordinal space during each refresh. This results
+   * in an always O(#facet_label) taxonomy array construction time when refresh is called.
+   */
+  private class AlwaysRefreshDirectoryTaxonomyReader extends DirectoryTaxonomyReader {
+
+    AlwaysRefreshDirectoryTaxonomyReader(Directory directory) throws IOException {
+      super(directory);
+    }
+
+    AlwaysRefreshDirectoryTaxonomyReader(DirectoryReader indexReader) throws IOException {
+      super(indexReader, null, null, null, null);
+    }
+
+    @Override
+    protected DirectoryTaxonomyReader doOpenIfChanged() throws IOException {
+      boolean success = false;
+
+      // the getInternalIndexReader() function performs the ensureOpen() check
+      final DirectoryReader reader = DirectoryReader.openIfChanged(super.getInternalIndexReader());
+      if (reader == null) {
+        return null; // no changes in the directory at all, nothing to do
+      }
+
+      try {
+        // It is important that we create an AlwaysRefreshDirectoryTaxonomyReader here and not a
+        // DirectoryTaxonomyReader.
+        // Returning a AlwaysRefreshDirectoryTaxonomyReader ensures that the recreated taxonomy
+        // reader also uses the overridden doOpenIfChanged
+        // method (that always recomputes values).
+        final AlwaysRefreshDirectoryTaxonomyReader newTaxonomyReader =
+            new AlwaysRefreshDirectoryTaxonomyReader(reader);
+        success = true;
+        return newTaxonomyReader;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(reader);
+        }
+      }
+    }
+  }
+}