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 2013/02/14 00:19:08 UTC

svn commit: r1445978 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/test/org/apache/lucene/index/

Author: mikemccand
Date: Wed Feb 13 23:19:08 2013
New Revision: 1445978

URL: http://svn.apache.org/r1445978
Log:
LUCENE-4775: nuke SegmentInfo.sizeInBytes; fix OneMerge.totalBytesSize thread safety

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Feb 13 23:19:08 2013
@@ -216,6 +216,10 @@ Bug Fixes
   cases, for example if you had an index with more than 260M documents and a 
   VAR_INT field.  (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
 
+* LUCENE-4775: Remove SegmentInfo.sizeInBytes() and make
+  MergePolicy.OneMerge.totalBytesSize thread safe (Josh Bronson via
+  Robert Muir, Mike McCandless)
+
 Documentation
 
 * LUCENE-4718: Fixed documentation of oal.queryparser.classic.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed Feb 13 23:19:08 2013
@@ -520,7 +520,7 @@ class DocumentsWriterPerThread {
       }
 
       if (infoStream.isEnabled("DWPT")) {
-        final double newSegmentSize = segmentInfo.sizeInBytes()/1024./1024.;
+        final double newSegmentSize = segmentInfoPerCommit.sizeInBytes()/1024./1024.;
         infoStream.message("DWPT", "flushed: segment=" + segmentInfo.name + 
                 " ramUsed=" + nf.format(startMBUsed) + " MB" +
                 " newFlushedSize(includes docstores)=" + nf.format(newSegmentSize) + " MB" +
@@ -557,7 +557,7 @@ class DocumentsWriterPerThread {
 
     IndexWriter.setDiagnostics(newSegment.info, "flush");
     
-    IOContext context = new IOContext(new FlushInfo(newSegment.info.getDocCount(), newSegment.info.sizeInBytes()));
+    IOContext context = new IOContext(new FlushInfo(newSegment.info.getDocCount(), newSegment.sizeInBytes()));
 
     boolean success = false;
     try {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Wed Feb 13 23:19:08 2013
@@ -2301,7 +2301,7 @@ public class IndexWriter implements Clos
               infoStream.message("IW", "addIndexes: process segment origName=" + info.info.name + " newName=" + newSegName + " info=" + info);
             }
 
-            IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.info.sizeInBytes(), true, -1));
+            IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.sizeInBytes(), true, -1));
 
             for(FieldInfo fi : getFieldInfos(info.info)) {
               globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType());
@@ -3458,7 +3458,8 @@ public class IndexWriter implements Clos
         final int delCount = numDeletedDocs(info);
         assert delCount <= info.info.getDocCount();
         final double delRatio = ((double) delCount)/info.info.getDocCount();
-        merge.estimatedMergeBytes += info.info.sizeInBytes() * (1.0 - delRatio);
+        merge.estimatedMergeBytes += info.sizeInBytes() * (1.0 - delRatio);
+        merge.totalMergeBytes += info.sizeInBytes();
       }
     }
   }
@@ -3759,7 +3760,7 @@ public class IndexWriter implements Clos
       // lost... 
 
       if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", String.format(Locale.ROOT, "merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.info.sizeInBytes()/1024./1024., merge.estimatedMergeBytes/1024/1024.));
+        infoStream.message("IW", String.format(Locale.ROOT, "merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes()/1024./1024., merge.estimatedMergeBytes/1024/1024.));
       }
 
       final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Wed Feb 13 23:19:08 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -74,7 +73,11 @@ public abstract class MergePolicy implem
     int maxNumSegments = -1;        // used by IndexWriter
 
     /** Estimated size in bytes of the merged segment. */
-    public long estimatedMergeBytes;       // used by IndexWriter
+    public volatile long estimatedMergeBytes;       // used by IndexWriter
+
+    // Sum of sizeInBytes of all SegmentInfos; set by IW.mergeInit
+    volatile long totalMergeBytes;
+
     List<SegmentReader> readers;        // used by IndexWriter
 
     /** Segments to be merged. */
@@ -187,14 +190,12 @@ public abstract class MergePolicy implem
     
     /**
      * Returns the total size in bytes of this merge. Note that this does not
-     * indicate the size of the merged segment, but the input total size.
-     * */
+     * indicate the size of the merged segment, but the
+     * input total size. This is only set once the merge is
+     * initialized by IndexWriter.
+     */
     public long totalBytesSize() throws IOException {
-      long total = 0;
-      for (SegmentInfoPerCommit info : segments) {
-        total += info.info.sizeInBytes();
-      }
-      return total;
+      return totalMergeBytes;
     }
 
     /**

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Wed Feb 13 23:19:08 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 
-import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -57,8 +56,6 @@ public final class SegmentInfo {
 
   private boolean isCompoundFile;
 
-  private volatile long sizeInBytes = -1;         // total byte size of all files (computed on demand)
-
   private Codec codec;
 
   private Map<String,String> diagnostics;
@@ -101,23 +98,6 @@ public final class SegmentInfo {
   }
 
   /**
-   * Returns total size in bytes of all of files used by
-   * this segment.  Note that this will not include any live
-   * docs for the segment; to include that use {@link
-   * SegmentInfoPerCommit#sizeInBytes()} instead.
-   */
-  public long sizeInBytes() throws IOException {
-    if (sizeInBytes == -1) {
-      long sum = 0;
-      for (final String fileName : files()) {
-        sum += dir.fileLength(fileName);
-      }
-      sizeInBytes = sum;
-    }
-    return sizeInBytes;
-  }
-
-  /**
    * Mark whether this segment is stored as a compound file.
    *
    * @param isCompoundFile true if this is a compound file;
@@ -254,7 +234,6 @@ public final class SegmentInfo {
   public void setFiles(Set<String> files) {
     checkFileNames(files);
     setFiles = files;
-    sizeInBytes = -1;
   }
 
   /** Add these files to the set of files written for this
@@ -262,7 +241,6 @@ public final class SegmentInfo {
   public void addFiles(Collection<String> files) {
     checkFileNames(files);
     setFiles.addAll(files);
-    sizeInBytes = -1;
   }
 
   /** Add this file to the set of files written for this
@@ -270,7 +248,6 @@ public final class SegmentInfo {
   public void addFile(String file) {
     checkFileNames(Collections.singleton(file));
     setFiles.add(file);
-    sizeInBytes = -1;
   }
   
   private void checkFileNames(Collection<String> files) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java Wed Feb 13 23:19:08 2013
@@ -374,7 +374,7 @@ public class TieredMergePolicy extends M
       for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
         final SegmentInfoPerCommit info = infosSorted.get(idx);
         if (merging.contains(info)) {
-          mergingBytes += info.info.sizeInBytes();
+          mergingBytes += info.sizeInBytes();
         } else if (!toBeMerged.contains(info)) {
           eligible.add(info);
         }
@@ -470,7 +470,7 @@ public class TieredMergePolicy extends M
       final long segBytes = size(info);
       totAfterMergeBytes += segBytes;
       totAfterMergeBytesFloored += floorSize(segBytes);
-      totBeforeMergeBytes += info.info.sizeInBytes();
+      totBeforeMergeBytes += info.sizeInBytes();
     }
 
     // Measure "skew" of the merge, which can range
@@ -670,7 +670,7 @@ public class TieredMergePolicy extends M
 
   // Segment size in bytes, pro-rated by % deleted
   private long size(SegmentInfoPerCommit info) throws IOException {
-    final long byteSize = info.info.sizeInBytes();    
+    final long byteSize = info.sizeInBytes();    
     final int delCount = writer.get().numDeletedDocs(info);
     final double delRatio = (info.info.getDocCount() <= 0 ? 0.0f : ((double)delCount / (double)info.info.getDocCount()));    
     assert delRatio <= 1.0;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1445978&r1=1445977&r2=1445978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Wed Feb 13 23:19:08 2013
@@ -324,4 +324,40 @@ public class TestConcurrentMergeSchedule
     w.close(false);
     dir.close();
   }
+
+
+  private static class TrackingCMS extends ConcurrentMergeScheduler {
+    long totMergedBytes;
+
+    public TrackingCMS() {
+      setMaxMergeCount(5);
+      setMaxThreadCount(5);
+    }
+
+    @Override
+    public void doMerge(MergePolicy.OneMerge merge) throws IOException {
+      totMergedBytes += merge.totalBytesSize();
+      super.doMerge(merge);
+    }
+  }
+
+  public void testTotalBytesSize() throws Exception {
+    Directory d = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMaxBufferedDocs(5);
+    iwc.setMergeScheduler(new TrackingCMS());
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    for(int i=0;i<100000;i++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+i, Field.Store.NO));
+      doc.add(newTextField("field", "here is some text", Field.Store.NO));
+      w.addDocument(doc);
+
+      if (random().nextBoolean()) {
+        w.deleteDocuments(new Term("id", ""+random().nextInt(i+1)));
+      }
+    }
+    w.close();
+    d.close();
+  }
 }