You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/08/19 03:25:15 UTC

svn commit: r1618777 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/lucene410/ lucene/test-framework/ lucene/test-framework/src/java/org/apache/lucene/index/

Author: rmuir
Date: Tue Aug 19 01:25:15 2014
New Revision: 1618777

URL: http://svn.apache.org/r1618777
Log:
LUCENE-5891: clean up Lucene410DVProducer concurrency

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java?rev=1618777&r1=1618776&r2=1618777&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java Tue Aug 19 01:25:15 2014
@@ -420,23 +420,19 @@ class Lucene410DocValuesProducer extends
   }
   
   /** returns an address instance for variable-length binary values. */
-  private MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      addresses = addrInstance;
+  private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
+    if (addresses == null) {
+      data.seek(bytes.addressesOffset);
+      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
+      addressInstances.put(field.number, addresses);
+      ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
     return addresses;
   }
   
   private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses = getAddressInstance(data, field, bytes);
+    final MonotonicBlockPackedReader addresses = getAddressInstance(field, bytes);
 
     final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
     final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
@@ -461,48 +457,39 @@ class Lucene410DocValuesProducer extends
   }
   
   /** returns an address instance for prefix-compressed binary values. */
-  private MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
-        addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        addressInstances.put(field.number, addrInstance);
-        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      addresses = addrInstance;
+  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
+    if (addresses == null) {
+      data.seek(bytes.addressesOffset);
+      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
+      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      addressInstances.put(field.number, addresses);
+      ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
     return addresses;
   }
   
   /** returns a reverse lookup instance for prefix-compressed binary values. */
-  private ReverseTermsIndex getReverseIndexInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
-    final ReverseTermsIndex index;
-    synchronized (reverseIndexInstances) {
-      ReverseTermsIndex instance = reverseIndexInstances.get(field.number);
-      if (instance == null) {
-        instance = new ReverseTermsIndex();
-        data.seek(bytes.reverseIndexOffset);
-        long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
-        instance.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        long dataSize = data.readVLong();
-        PagedBytes pagedBytes = new PagedBytes(15);
-        pagedBytes.copy(data, dataSize);
-        instance.terms = pagedBytes.freeze(true);
-        reverseIndexInstances.put(field.number, instance);
-        ramBytesUsed.addAndGet(instance.termAddresses.ramBytesUsed() + instance.terms.ramBytesUsed());
-      }
-      index = instance;
+  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    ReverseTermsIndex index = reverseIndexInstances.get(field.number);
+    if (index == null) {
+      index = new ReverseTermsIndex();
+      data.seek(bytes.reverseIndexOffset);
+      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
+      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      long dataSize = data.readVLong();
+      PagedBytes pagedBytes = new PagedBytes(15);
+      pagedBytes.copy(data, dataSize);
+      index.terms = pagedBytes.freeze(true);
+      reverseIndexInstances.put(field.number, index);
+      ramBytesUsed.addAndGet(index.termAddresses.ramBytesUsed() + index.terms.ramBytesUsed());
     }
     return index;
   }
 
-
   private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses = getIntervalInstance(data, field, bytes);
-    final ReverseTermsIndex index = getReverseIndexInstance(data, field, bytes);
+    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
+    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
     assert addresses.size() > 0; // we don't have to handle empty case
     IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
     return new CompressedBinaryDocValues(bytes, addresses, index, slice);
@@ -552,19 +539,15 @@ class Lucene410DocValuesProducer extends
   }
   
   /** returns an address instance for sortedset ordinal lists */
-  private MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
-    final MonotonicBlockPackedReader ordIndex;
-    synchronized (ordIndexInstances) {
-      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
-      if (ordIndexInstance == null) {
-        data.seek(entry.offset);
-        ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
-        ordIndexInstances.put(field.number, ordIndexInstance);
-        ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
-      }
-      ordIndex = ordIndexInstance;
+  private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
+    MonotonicBlockPackedReader instance = ordIndexInstances.get(field.number);
+    if (instance == null) {
+      data.seek(entry.offset);
+      instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
+      ordIndexInstances.put(field.number, instance);
+      ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
     }
-    return ordIndex;
+    return instance;
   }
   
   @Override
@@ -576,8 +559,7 @@ class Lucene410DocValuesProducer extends
       final Bits docsWithField = getMissingBits(numericEntry.missingOffset);
       return DocValues.singleton(values, docsWithField);
     } else if (ss.format == SORTED_WITH_ADDRESSES) {
-      final IndexInput data = this.data.clone();
-      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
+      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
       
       return new SortedNumericDocValues() {
         long startOffset;
@@ -614,13 +596,12 @@ class Lucene410DocValuesProducer extends
       throw new AssertionError();
     }
 
-    final IndexInput data = this.data.clone();
     final long valueCount = binaries.get(field.number).count;
     // we keep the byte[]s and list of ords on disk, these could be large
     final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
     final LongValues ordinals = getNumeric(ords.get(field.number));
     // but the addresses to the ord stream are in RAM
-    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
+    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
     
     return new RandomAccessOrds() {
       long startOffset;

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1618777&r1=1618776&r2=1618777&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Tue Aug 19 01:25:15 2014
@@ -19,7 +19,9 @@ package org.apache.lucene.index;
 
 import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -59,6 +61,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -2953,7 +2956,7 @@ public abstract class BaseDocValuesForma
     Field dvNumericField = new NumericDocValuesField("dvNum", 0);
     
     // index some docs
-    int numDocs = atLeast(300);
+    int numDocs = TestUtil.nextInt(random(), 1025, 2047);
     for (int i = 0; i < numDocs; i++) {
       idField.setStringValue(Integer.toString(i));
       int length = TestUtil.nextInt(random(), 0, 8);
@@ -3109,6 +3112,74 @@ public abstract class BaseDocValuesForma
     ir.close();
     dir.close();
   }
+  
+  public void testThreads3() throws Exception {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    assumeTrue("Codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    
+    int numSortedSets = random().nextInt(21);
+    int numBinaries = random().nextInt(21);
+    int numSortedNums = random().nextInt(21);
+    
+    int numDocs = TestUtil.nextInt(random(), 2025, 2047);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      
+      for (int j = 0; j < numSortedSets; j++) {
+        doc.add(new SortedSetDocValuesField("ss" + j, new BytesRef(TestUtil.randomSimpleString(random()))));
+        doc.add(new SortedSetDocValuesField("ss" + j, new BytesRef(TestUtil.randomSimpleString(random()))));
+      }
+      
+      for (int j = 0; j < numBinaries; j++) {
+        doc.add(new BinaryDocValuesField("b" + j, new BytesRef(TestUtil.randomSimpleString(random()))));
+      }
+      
+      for (int j = 0; j < numSortedNums; j++) {
+        doc.add(new SortedNumericDocValuesField("sn" + j, TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE)));
+        doc.add(new SortedNumericDocValuesField("sn" + j, TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE)));
+      }
+      writer.addDocument(doc);
+    }
+    writer.close();
+    
+    // now check with threads
+    for (int i = 0; i < 10; i++) {
+      final DirectoryReader r = DirectoryReader.open(dir);
+      final CountDownLatch startingGun = new CountDownLatch(1);
+      Thread threads[] = new Thread[TestUtil.nextInt(random(), 4, 10)];
+      for (int tid = 0; tid < threads.length; tid++) {
+        threads[tid] = new Thread() {
+          @Override
+          public void run() {
+            try {
+              ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+              PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
+              startingGun.await();
+              for (AtomicReaderContext leaf : r.leaves()) {
+                CheckIndex.testDocValues(leaf.reader(), infoStream, true);
+              }
+            } catch (Exception e) {
+              throw new RuntimeException();
+            }
+          }
+        };
+      }
+      for (int tid = 0; tid < threads.length; tid++) {
+        threads[tid].start();
+      }
+      startingGun.countDown();
+      for (int tid = 0; tid < threads.length; tid++) {
+        threads[tid].join();
+      }
+      r.close();
+    }
+
+    dir.close();
+  }
 
   // LUCENE-5218
   public void testEmptyBinaryValueOnPageSizes() throws Exception {