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

lucene-solr:master: LUCENE-7779: don't call BytesSequencesReader.next again after it already returned null

Repository: lucene-solr
Updated Branches:
  refs/heads/master 0bcd88b18 -> b954f220e


LUCENE-7779: don't call BytesSequencesReader.next again after it already returned null


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

Branch: refs/heads/master
Commit: b954f220e0321a31b9dc3d6a7e19a552dd58e0b1
Parents: 0bcd88b
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Apr 12 17:17:30 2017 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Apr 12 17:17:30 2017 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/util/OfflineSorter.java   | 15 +++++--
 .../apache/lucene/util/TestOfflineSorter.java   | 42 ++++++++++++++++++++
 2 files changed, 53 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b954f220/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index fa22320..ef2d819 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -249,9 +249,14 @@ public class OfflineSorter {
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
 
     boolean success = false;
+    boolean[] isExhausted = new boolean[1];
     try (ByteSequencesReader is = getReader(dir.openChecksumInput(inputFileName, IOContext.READONCE), inputFileName)) {
-      int lineCount;
-      while ((lineCount = readPartition(is)) > 0) {
+      while (isExhausted[0] == false) {
+        int lineCount = readPartition(is, isExhausted);
+        if (lineCount == 0) {
+          assert isExhausted[0];
+          break;
+        }
         segments.add(sortPartition(trackingDir));
         sortInfo.tempMergeFiles++;
         sortInfo.lineCount += lineCount;
@@ -420,8 +425,8 @@ public class OfflineSorter {
     sortInfo.tempMergeFiles++;
   }
 
-  /** Read in a single partition of data */
-  int readPartition(ByteSequencesReader reader) throws IOException {
+  /** Read in a single partition of data, setting isExhausted[0] to true if there are no more items. */
+  int readPartition(ByteSequencesReader reader, boolean[] isExhausted) throws IOException {
     long start = System.currentTimeMillis();
     if (valueLength != -1) {
       int limit = ramBufferSize.bytes / valueLength;
@@ -433,6 +438,7 @@ public class OfflineSorter {
           verifyChecksum(t, reader);
         }
         if (item == null) {
+          isExhausted[0] = true;
           break;
         }
         buffer.append(item);
@@ -446,6 +452,7 @@ public class OfflineSorter {
           verifyChecksum(t, reader);
         }
         if (item == null) {
+          isExhausted[0] = true;
           break;
         }
         buffer.append(item);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b954f220/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
index 49ed110..839f103 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.CorruptingIndexOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FilterDirectory;
@@ -455,6 +456,47 @@ public class TestOfflineSorter extends LuceneTestCase {
     dir.close();
   }
 
+  // OfflineSorter should not call my BytesSequencesReader.next() again after it already returned null:
+  public void testOverNexting() throws Exception {
+    Directory dir = newDirectory();
+    IndexOutput out = dir.createTempOutput("unsorted", "tmp", IOContext.DEFAULT);
+    try (ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(out)) {
+      byte[] bytes = new byte[Integer.BYTES];
+      random().nextBytes(bytes);
+      w.write(bytes);
+      CodecUtil.writeFooter(out);
+    }
+
+    new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Integer.BYTES) {
+      @Override
+      protected ByteSequencesReader getReader(ChecksumIndexInput in, String name) throws IOException {
+        ByteSequencesReader other = super.getReader(in, name);
+
+        return new ByteSequencesReader(in, name) {
+
+          private boolean alreadyEnded;
+              
+          @Override
+          public BytesRef next() throws IOException {
+            // if we returned null already, OfflineSorter should not call next() again
+            assertFalse(alreadyEnded);
+            BytesRef result = other.next();
+            if (result == null) {
+              alreadyEnded = true;
+            }
+            return result;
+          }
+
+          @Override
+          public void close() throws IOException {
+            other.close();
+          }
+        };
+      }
+    }.sort(out.getName());
+    dir.close();
+  }
+
   public void testInvalidFixedLength() throws Exception {
     IllegalArgumentException e;
     e = expectThrows(IllegalArgumentException.class,