You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/10/04 07:21:18 UTC

[lucene] branch main updated: LUCENE-10139: ExternalRefSorter returns a covariant with a subtype of BytesRefIterator that is Closeable. (#340)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 2e57a40  LUCENE-10139: ExternalRefSorter returns a covariant with a subtype of BytesRefIterator that is Closeable. (#340)
2e57a40 is described below

commit 2e57a405463435570548475464140452ea0a6f83
Author: Dawid Weiss <da...@carrotsearch.com>
AuthorDate: Mon Oct 4 09:21:09 2021 +0200

    LUCENE-10139: ExternalRefSorter returns a covariant with a subtype of BytesRefIterator that is Closeable. (#340)
---
 lucene/CHANGES.txt                                 |  3 +
 .../matchhighlight/TestMatchHighlighter.java       |  3 +-
 .../search/suggest/fst/ExternalRefSorter.java      | 65 ++++++++++++++--------
 .../search/suggest/fst/TestBytesRefSorters.java    | 34 +++++++++--
 4 files changed, 77 insertions(+), 28 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4fce34b..c39a7d4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -157,6 +157,9 @@ API Changes
 
 Improvements
 
+* LUCENE-10139: ExternalRefSorter returns a covariant with a subtype of BytesRefIterator
+  that is Closeable. (Dawid Weiss).
+
 * LUCENE-10135: Correct passage selector behavior for long matching snippets (Dawid Weiss).
 
 * LUCENE-9960: Avoid unnecessary top element replacement for equal elements in PriorityQueue. (Dawid Weiss)
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/TestMatchHighlighter.java b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/TestMatchHighlighter.java
index 9b3ebfc..bafb75d 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/TestMatchHighlighter.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/TestMatchHighlighter.java
@@ -57,6 +57,7 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.hamcrest.MatcherAssert;
 import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Test;
@@ -116,7 +117,7 @@ public class TestMatchHighlighter extends LuceneTestCase {
   static SynonymMap buildSynonymMap(String[][] synonyms) throws IOException {
     SynonymMap.Builder builder = new SynonymMap.Builder();
     for (String[] pair : synonyms) {
-      assertThat(pair.length, Matchers.equalTo(2));
+      MatcherAssert.assertThat(pair.length, Matchers.equalTo(2));
       builder.add(new CharsRef(pair[0]), new CharsRef(pair[1]), true);
     }
     return builder.build();
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
index 08940d8..998a2a2 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
@@ -28,7 +28,9 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OfflineSorter;
 
 /**
- * Builds and iterates over sequences stored on disk.
+ * An implementation of a {@link BytesRefSorter} that allows appending {@link BytesRef}s to an
+ * {@link OfflineSorter} and returns a {@link Closeable} {@link ByteSequenceIterator} that iterates
+ * over sequences stored on disk.
  *
  * @lucene.experimental
  * @lucene.internal
@@ -36,76 +38,87 @@ import org.apache.lucene.util.OfflineSorter;
 public class ExternalRefSorter implements BytesRefSorter, Closeable {
   private final OfflineSorter sorter;
   private OfflineSorter.ByteSequencesWriter writer;
-  private IndexOutput input;
-  private String sortedFileName;
+  private IndexOutput tempOutput;
+  private String sortedOutput;
 
   /** Will buffer all sequences to a temporary file and then sort (all on-disk). */
   public ExternalRefSorter(OfflineSorter sorter) throws IOException {
     this.sorter = sorter;
-    this.input =
+    this.tempOutput =
         sorter
             .getDirectory()
             .createTempOutput(sorter.getTempFileNamePrefix(), "RefSorterRaw", IOContext.DEFAULT);
-    this.writer = new OfflineSorter.ByteSequencesWriter(this.input);
+    this.writer = new OfflineSorter.ByteSequencesWriter(this.tempOutput);
   }
 
   @Override
   public void add(BytesRef utf8) throws IOException {
     if (writer == null) {
-      throw new IllegalStateException();
+      throw new IllegalStateException(
+          "Can't append after iterator() has been called and all the input sorted.");
     }
     writer.write(utf8);
   }
 
+  /**
+   * @return Returns a {@link ByteSequenceIterator} that implements {@link BytesRefIterator} but is
+   *     also {@link Closeable}, ensuring any temporary resources are cleaned up if the iterator is
+   *     either exhausted or closed.
+   */
   @Override
-  public BytesRefIterator iterator() throws IOException {
-    if (sortedFileName == null) {
+  public ByteSequenceIterator iterator() throws IOException {
+    if (sortedOutput == null) {
+      if (tempOutput == null) {
+        throw new IOException("The sorter has been closed.");
+      }
+
       closeWriter();
 
       boolean success = false;
       try {
-        sortedFileName = sorter.sort(input.getName());
+        sortedOutput = sorter.sort(tempOutput.getName());
         success = true;
       } finally {
         if (success) {
-          sorter.getDirectory().deleteFile(input.getName());
+          sorter.getDirectory().deleteFile(tempOutput.getName());
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorter.getDirectory(), input.getName());
+          IOUtils.deleteFilesIgnoringExceptions(sorter.getDirectory(), tempOutput.getName());
         }
       }
 
-      input = null;
+      tempOutput = null;
     }
 
     return new ByteSequenceIterator(
         new OfflineSorter.ByteSequencesReader(
-            sorter.getDirectory().openChecksumInput(sortedFileName, IOContext.READONCE),
-            sortedFileName));
+            sorter.getDirectory().openChecksumInput(sortedOutput, IOContext.READONCE),
+            sortedOutput));
   }
 
+  /** Close the writer but leave any sorted output for iteration. */
   private void closeWriter() throws IOException {
     if (writer != null) {
-      CodecUtil.writeFooter(input);
+      CodecUtil.writeFooter(tempOutput);
       writer.close();
       writer = null;
     }
   }
 
-  /** Removes any written temporary files. */
+  /** Close the writer and remove any temporary files. */
   @Override
   public void close() throws IOException {
     try {
       closeWriter();
     } finally {
       IOUtils.deleteFilesIgnoringExceptions(
-          sorter.getDirectory(), input == null ? null : input.getName(), sortedFileName);
+          sorter.getDirectory(), tempOutput == null ? null : tempOutput.getName(), sortedOutput);
     }
   }
 
-  /** Iterate over byte refs in a file. */
-  // TODO: this class is a bit silly ... sole purpose is to "remove" Closeable from what #iterator
-  // returns:
-  static class ByteSequenceIterator implements BytesRefIterator {
+  /**
+   * Iterates over {@link BytesRef}s in a file, closes the reader when the iterator is exhausted.
+   */
+  static class ByteSequenceIterator implements BytesRefIterator, Closeable {
     private final OfflineSorter.ByteSequencesReader reader;
     private BytesRef scratch;
 
@@ -119,7 +132,7 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable {
       try {
         scratch = reader.next();
         if (scratch == null) {
-          reader.close();
+          close();
         }
         success = true;
         return scratch;
@@ -129,8 +142,16 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable {
         }
       }
     }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
   }
 
+  /**
+   * @return Return the {@link Comparator} of the {@link OfflineSorter} used to sort byte sequences.
+   */
   @Override
   public Comparator<BytesRef> getComparator() {
     return sorter.getComparator();
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestBytesRefSorters.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestBytesRefSorters.java
index 2af2eb0..80b7687 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestBytesRefSorters.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestBytesRefSorters.java
@@ -16,7 +16,12 @@
  */
 package org.apache.lucene.search.suggest.fst;
 
+import com.carrotsearch.randomizedtesting.RandomizedContext;
+import com.carrotsearch.randomizedtesting.generators.RandomBytes;
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import java.io.IOException;
 import java.util.Comparator;
+import java.util.Random;
 import org.apache.lucene.search.suggest.InMemorySorter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -36,16 +41,28 @@ public class TestBytesRefSorters extends LuceneTestCase {
   }
 
   @Test
+  public void testExternalRefSortersIteratorIsCloseable() throws Exception {
+    try (Directory tempDir = newDirectory();
+        ExternalRefSorter s = new ExternalRefSorter(new OfflineSorter(tempDir, "temp"))) {
+      appendRandomSequences(s);
+
+      // Sometimes iterate over a subset of the entries in the sequence iterator, then
+      // close it before exhausting the iterator.
+      try (ExternalRefSorter.ByteSequenceIterator it = s.iterator()) {
+        for (int i = 0; i < 5 && it.next() != null; i++) {
+          // Empty.
+        }
+      }
+    }
+  }
+
+  @Test
   public void testInMemorySorter() throws Exception {
     check(new InMemorySorter(Comparator.naturalOrder()));
   }
 
   private void check(BytesRefSorter sorter) throws Exception {
-    for (int i = 0; i < 100; i++) {
-      byte[] current = new byte[random().nextInt(256)];
-      random().nextBytes(current);
-      sorter.add(new BytesRef(current));
-    }
+    appendRandomSequences(sorter);
 
     // Create two iterators and check that they're aligned with each other.
     BytesRefIterator i1 = sorter.iterator();
@@ -67,4 +84,11 @@ public class TestBytesRefSorters extends LuceneTestCase {
       }
     }
   }
+
+  private void appendRandomSequences(BytesRefSorter sorter) throws IOException {
+    Random rnd = new Random(RandomizedContext.current().getRandom().nextLong());
+    for (int i = 0; i < RandomNumbers.randomIntBetween(rnd, 10, 100); i++) {
+      sorter.add(new BytesRef(RandomBytes.randomBytesOfLengthBetween(rnd, 1, 256)));
+    }
+  }
 }