You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2014/04/30 22:29:17 UTC

svn commit: r1591474 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/

Author: shaie
Date: Wed Apr 30 20:29:17 2014
New Revision: 1591474

URL: http://svn.apache.org/r1591474
Log:
LUCENE-5591: pass proper IOContext when writing DocValues updates

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Wed Apr 30 20:29:17 2014
@@ -52,6 +52,9 @@ Optimizations
   
 * LUCENE-5599: HttpReplicator did not properly delegate bulk read() to wrapped
   InputStream. (Christoph Kaser via Shai Erera)
+  
+* LUCENE-5591: pass an IOContext with estimated flush size when applying DV
+  updates. (Shai Erera)
 
 Bug fixes
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java Wed Apr 30 20:29:17 2014
@@ -3,6 +3,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.InPlaceMergeSorter;
@@ -110,13 +111,15 @@ class BinaryDocValuesFieldUpdates extend
   private PagedGrowableWriter offsets, lengths;
   private BytesRef values;
   private int size;
+  private final int bitsPerValue;
   
   public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
     super(field, Type.BINARY);
     docsWithField = new FixedBitSet(64);
-    docs = new PagedMutable(1, 1024, PackedInts.bitsRequired(maxDoc - 1), PackedInts.COMPACT);
-    offsets = new PagedGrowableWriter(1, 1024, 1, PackedInts.FAST);
-    lengths = new PagedGrowableWriter(1, 1024, 1, PackedInts.FAST);
+    bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
+    docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
+    offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
+    lengths = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
     values = new BytesRef(16); // start small
     size = 0;
   }
@@ -222,12 +225,27 @@ class BinaryDocValuesFieldUpdates extend
       lengths.set(size, otherUpdates.lengths.get(i));
       ++size;
     }
-    values.append(otherUpdates.values);
+    int newLen = values.length + otherUpdates.values.length;
+    if (values.bytes.length < newLen) {
+      values.bytes = ArrayUtil.grow(values.bytes, newLen);
+    }
+    System.arraycopy(otherUpdates.values.bytes, otherUpdates.values.offset, values.bytes, values.length, otherUpdates.values.length);
+    values.length = newLen;
   }
 
   @Override
   public boolean any() {
     return size > 0;
   }
-  
+
+  @Override
+  public long ramBytesPerDoc() {
+    long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue + 1 /* docsWithField */) / 8); // docs
+    final int capacity = estimateCapacity(size);
+    bytesPerDoc += (long) Math.ceil((double) offsets.ramBytesUsed() / capacity); // offsets
+    bytesPerDoc += (long) Math.ceil((double) lengths.ramBytesUsed() / capacity); // lengths
+    bytesPerDoc += (long) Math.ceil((double) values.length / size); // values
+    return bytesPerDoc;
+  }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java Wed Apr 30 20:29:17 2014
@@ -5,6 +5,7 @@ import java.util.Map;
 
 import org.apache.lucene.index.NumericDocValuesFieldUpdates;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.packed.PagedGrowableWriter;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -30,6 +31,8 @@ import org.apache.lucene.search.DocIdSet
  */
 abstract class DocValuesFieldUpdates {
   
+  protected static final int PAGE_SIZE = 1024;
+
   static enum Type { NUMERIC, BINARY }
   
   /**
@@ -86,6 +89,17 @@ abstract class DocValuesFieldUpdates {
       return numericDVUpdates.size() + binaryDVUpdates.size();
     }
     
+    long ramBytesPerDoc() {
+      long ramBytesPerDoc = 0;
+      for (NumericDocValuesFieldUpdates updates : numericDVUpdates.values()) {
+        ramBytesPerDoc += updates.ramBytesPerDoc();
+      }
+      for (BinaryDocValuesFieldUpdates updates : binaryDVUpdates.values()) {
+        ramBytesPerDoc += updates.ramBytesPerDoc();
+      }
+      return ramBytesPerDoc;
+    }
+    
     DocValuesFieldUpdates getUpdates(String field, Type type) {
       switch (type) {
         case NUMERIC:
@@ -129,6 +143,14 @@ abstract class DocValuesFieldUpdates {
   }
   
   /**
+   * Returns the estimated capacity of a {@link PagedGrowableWriter} given the
+   * actual number of stored elements.
+   */
+  protected static int estimateCapacity(int size) {
+    return (int) Math.ceil((double) size / PAGE_SIZE) * PAGE_SIZE;
+  }
+  
+  /**
    * Add an update to a document. For unsetting a value you should pass
    * {@code null}.
    */
@@ -147,8 +169,10 @@ abstract class DocValuesFieldUpdates {
    */
   public abstract void merge(DocValuesFieldUpdates other);
 
-  /** Returns true if this instance contains any updates. 
-   * @return TODO*/
+  /** Returns true if this instance contains any updates. */
   public abstract boolean any();
   
+  /** Returns approximate RAM bytes used per document. */
+  public abstract long ramBytesPerDoc();
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java Wed Apr 30 20:29:17 2014
@@ -88,6 +88,7 @@ class NumericDocValuesFieldUpdates exten
     }
   }
 
+  private final int bitsPerValue;
   private FixedBitSet docsWithField;
   private PagedMutable docs;
   private PagedGrowableWriter values;
@@ -96,8 +97,9 @@ class NumericDocValuesFieldUpdates exten
   public NumericDocValuesFieldUpdates(String field, int maxDoc) {
     super(field, Type.NUMERIC);
     docsWithField = new FixedBitSet(64);
-    docs = new PagedMutable(1, 1024, PackedInts.bitsRequired(maxDoc - 1), PackedInts.COMPACT);
-    values = new PagedGrowableWriter(1, 1024, 1, PackedInts.FAST);
+    bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
+    docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
+    values = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
     size = 0;
   }
   
@@ -198,4 +200,12 @@ class NumericDocValuesFieldUpdates exten
     return size > 0;
   }
 
+  @Override
+  public long ramBytesPerDoc() {
+    long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue + 1 /* docsWithField */) / 8);
+    final int capacity = estimateCapacity(size);
+    bytesPerDoc += (long) Math.ceil((double) values.ramBytesUsed() / capacity); // values
+    return bytesPerDoc;
+  }
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Wed Apr 30 20:29:17 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.codecs.LiveDocs
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Bits;
@@ -395,7 +396,9 @@ class ReadersAndUpdates {
         fieldInfos = builder.finish();
         final long nextFieldInfosGen = info.getNextFieldInfosGen();
         final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
-        final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, writer.getConfig().getTermIndexInterval(), null, IOContext.DEFAULT, segmentSuffix);
+        final long estUpdatesSize = dvUpdates.ramBytesPerDoc() * info.info.getDocCount();
+        final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
+        final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, writer.getConfig().getTermIndexInterval(), null, updatesContext, segmentSuffix);
         final DocValuesFormat docValuesFormat = codec.docValuesFormat();
         final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
         boolean fieldsConsumerSuccess = false;
@@ -517,9 +520,14 @@ class ReadersAndUpdates {
             }
           });
         }
-
-          codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, IOContext.DEFAULT);
-          fieldsConsumerSuccess = true;
+        
+        // we write approximately that many bytes (based on Lucene46DVF):
+        // HEADER + FOOTER: 40
+        // 90 bytes per-field (over estimating long name and attributes map)
+        final long estInfosSize = 40 + 90 * fieldInfos.size();
+        final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
+        codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
+        fieldsConsumerSuccess = true;
         } finally {
           if (fieldsConsumerSuccess) {
             fieldsConsumer.close();

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java Wed Apr 30 20:29:17 2014
@@ -27,12 +27,14 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
+import org.junit.Test;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
@@ -54,6 +56,7 @@ import com.carrotsearch.randomizedtestin
  */
 
 @SuppressCodecs({"Appending","Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45"})
+@SuppressWarnings("resource")
 public class TestBinaryDocValuesUpdates extends LuceneTestCase {
 
   static long getValue(BinaryDocValues bdv, int idx, BytesRef scratch) {
@@ -1447,5 +1450,31 @@ public class TestBinaryDocValuesUpdates 
     
     dir.close();
   }
+
+  @Test
+  public void testIOContext() throws Exception {
+    // LUCENE-5591: make sure we pass an IOContext with an approximate
+    // segmentSize in FlushInfo
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    // we want a single large enough segment so that a doc-values update writes a large file
+    conf.setMergePolicy(NoMergePolicy.INSTANCE);
+    conf.setMaxBufferedDocs(Integer.MAX_VALUE); // manually flush
+    conf.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
+    for (int i = 0; i < 100; i++) {
+      writer.addDocument(doc(i));
+    }
+    writer.commit();
+    writer.close();
+    
+    NRTCachingDirectory cachingDir = new NRTCachingDirectory(dir, 100, 1/(1024.*1024.));
+    writer = new IndexWriter(cachingDir, conf.clone());
+    writer.updateBinaryDocValue(new Term("id", "doc-0"), "val", toBytes(100L));
+    DirectoryReader reader = DirectoryReader.open(writer, true); // flush
+    assertEquals(0, cachingDir.listCachedFiles().length);
+    
+    IOUtils.close(reader, writer, cachingDir);
+  }
   
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java?rev=1591474&r1=1591473&r2=1591474&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java Wed Apr 30 20:29:17 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -55,6 +56,7 @@ import com.carrotsearch.randomizedtestin
  */
 
 @SuppressCodecs({"Appending","Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45"})
+@SuppressWarnings("resource")
 public class TestNumericDocValuesUpdates extends LuceneTestCase {
   
   private Document doc(int id) {
@@ -1428,5 +1430,32 @@ public class TestNumericDocValuesUpdates
     
     dir.close();
   }
+
+  @Test
+  public void testIOContext() throws Exception {
+    // LUCENE-5591: make sure we pass an IOContext with an approximate
+    // segmentSize in FlushInfo
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    // we want a single large enough segment so that a doc-values update writes a large file
+    conf.setMergePolicy(NoMergePolicy.INSTANCE);
+    conf.setMaxBufferedDocs(Integer.MAX_VALUE); // manually flush
+    conf.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
+    for (int i = 0; i < 100; i++) {
+      writer.addDocument(doc(i));
+    }
+    writer.commit();
+    writer.close();
+    
+    NRTCachingDirectory cachingDir = new NRTCachingDirectory(dir, 100, 1/(1024.*1024.));
+    writer = new IndexWriter(cachingDir, conf.clone());
+    writer.updateNumericDocValue(new Term("id", "doc-0"), "val", 100L);
+    DirectoryReader reader = DirectoryReader.open(writer, true); // flush
+    assertEquals(0, cachingDir.listCachedFiles().length);
+    for (String f : cachingDir.listAll()) System.out.println(f + " " + cachingDir.fileLength(f));
+    
+    IOUtils.close(reader, writer, cachingDir);
+  }
   
 }