From ad8934b205eeb5e0265831ecbb6bfb4a4e8ca687 Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Mon, 28 Jan 2013 02:48:46 +0000 Subject: [PATCH] clear up some nocommits git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1439232 13f79535-47bb-0310-9956-ffa450edef68 --- .../lucene40/Lucene40DocValuesReader.java | 77 ++++++++----------- .../lucene42/Lucene42DocValuesConsumer.java | 3 - .../lucene42/Lucene42DocValuesProducer.java | 15 ++-- .../lucene/index/BinaryDocValuesWriter.java | 4 +- .../org/apache/lucene/index/CheckIndex.java | 2 - .../lucene/index/NumericDocValuesWriter.java | 2 - .../org/apache/lucene/util/ByteBlockPool.java | 8 +- .../org/apache/lucene/util/BytesRefArray.java | 18 ----- .../org/apache/lucene/util/BytesRefHash.java | 15 ++-- .../org/apache/lucene/util/PagedBytes.java | 12 +-- .../apache/lucene/util/TestBytesRefArray.java | 4 +- .../grouping/DistinctValuesCollectorTest.java | 3 - .../BufferingTermFreqIteratorWrapper.java | 3 +- .../search/suggest/fst/InMemorySorter.java | 3 +- .../lucene/search/suggest/fst/Sort.java | 5 +- .../apache/solr/core/SchemaCodecFactory.java | 2 +- 16 files changed, 67 insertions(+), 109 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java index b25dba644b2..bd61680ff9a 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java @@ -36,6 +36,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.PagedBytes; import org.apache.lucene.util.packed.PackedInts; /** @@ -308,9 +309,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer { Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_START, Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_CURRENT); final int fixedLength = input.readInt(); - // nocommit? can the current impl even handle > 2G? - final byte bytes[] = new byte[state.segmentInfo.getDocCount() * fixedLength]; - input.readBytes(bytes, 0, bytes.length); + PagedBytes bytes = new PagedBytes(16); + bytes.copy(input, fixedLength * (long)state.segmentInfo.getDocCount()); + final PagedBytes.Reader bytesReader = bytes.freeze(true); if (input.getFilePointer() != input.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); } @@ -318,9 +319,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer { return new BinaryDocValues() { @Override public void get(int docID, BytesRef result) { - result.bytes = bytes; - result.offset = docID * fixedLength; - result.length = fixedLength; + bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength); } }; } finally { @@ -347,10 +346,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer { CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_IDX, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_CURRENT); - // nocommit? can the current impl even handle > 2G? long totalBytes = index.readVLong(); - final byte bytes[] = new byte[(int)totalBytes]; - data.readBytes(bytes, 0, bytes.length); + PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, totalBytes); + final PagedBytes.Reader bytesReader = bytes.freeze(true); final PackedInts.Reader reader = PackedInts.getReader(index); if (data.getFilePointer() != data.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")"); @@ -364,9 +363,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer { public void get(int docID, BytesRef result) { long startAddress = reader.get(docID); long endAddress = reader.get(docID+1); - result.bytes = bytes; - result.offset = (int)startAddress; - result.length = (int)(endAddress - startAddress); + bytesReader.fillSlice(result, startAddress, (int)(endAddress - startAddress)); } }; } finally { @@ -396,9 +393,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer { final int fixedLength = data.readInt(); final int valueCount = index.readInt(); - // nocommit? can the current impl even handle > 2G? - final byte bytes[] = new byte[fixedLength * valueCount]; - data.readBytes(bytes, 0, bytes.length); + PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, fixedLength * (long) valueCount); + final PagedBytes.Reader bytesReader = bytes.freeze(true); final PackedInts.Reader reader = PackedInts.getReader(index); if (data.getFilePointer() != data.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")"); @@ -410,10 +407,8 @@ final class Lucene40DocValuesReader extends DocValuesProducer { return new BinaryDocValues() { @Override public void get(int docID, BytesRef result) { - int ord = (int)reader.get(docID); - result.bytes = bytes; - result.offset = ord * fixedLength; - result.length = fixedLength; + final long offset = fixedLength * reader.get(docID); + bytesReader.fillSlice(result, offset, fixedLength); } }; } finally { @@ -442,9 +437,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer { Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_CURRENT); final long totalBytes = index.readLong(); - // nocommit? can the current impl even handle > 2G? - final byte bytes[] = new byte[(int)totalBytes]; - data.readBytes(bytes, 0, bytes.length); + final PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, totalBytes); + final PagedBytes.Reader bytesReader = bytes.freeze(true); final PackedInts.Reader reader = PackedInts.getReader(index); if (data.getFilePointer() != data.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")"); @@ -456,16 +451,17 @@ final class Lucene40DocValuesReader extends DocValuesProducer { return new BinaryDocValues() { @Override public void get(int docID, BytesRef result) { - int startAddress = (int)reader.get(docID); - result.bytes = bytes; - result.offset = startAddress; - if ((bytes[startAddress] & 128) == 0) { + long startAddress = reader.get(docID); + BytesRef lengthBytes = new BytesRef(); + bytesReader.fillSlice(lengthBytes, startAddress, 1); + byte code = lengthBytes.bytes[lengthBytes.offset]; + if ((code & 128) == 0) { // length is 1 byte - result.offset++; - result.length = bytes[startAddress]; + bytesReader.fillSlice(result, startAddress + 1, (int) code); } else { - result.offset += 2; - result.length = ((bytes[startAddress] & 0x7f) << 8) | ((bytes[startAddress+1] & 0xff)); + bytesReader.fillSlice(lengthBytes, startAddress + 1, 1); + int length = ((code & 0x7f) << 8) | (lengthBytes.bytes[lengthBytes.offset] & 0xff); + bytesReader.fillSlice(result, startAddress + 2, length); } } }; @@ -530,9 +526,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer { final int fixedLength = data.readInt(); final int valueCount = index.readInt(); - // nocommit? can the current impl even handle > 2G? - final byte[] bytes = new byte[fixedLength*valueCount]; - data.readBytes(bytes, 0, bytes.length); + PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, fixedLength * (long) valueCount); + final PagedBytes.Reader bytesReader = bytes.freeze(true); final PackedInts.Reader reader = PackedInts.getReader(index); return correctBuggyOrds(new SortedDocValues() { @@ -543,9 +539,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer { @Override public void lookupOrd(int ord, BytesRef result) { - result.bytes = bytes; - result.offset = ord * fixedLength; - result.length = fixedLength; + bytesReader.fillSlice(result, fixedLength * (long) ord, fixedLength); } @Override @@ -564,10 +558,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer { Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_CURRENT); long maxAddress = index.readLong(); - // nocommit? can the current impl even handle > 2G? - final byte[] bytes = new byte[(int)maxAddress]; - data.readBytes(bytes, 0, bytes.length); - + PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, maxAddress); + final PagedBytes.Reader bytesReader = bytes.freeze(true); final PackedInts.Reader addressReader = PackedInts.getReader(index); final PackedInts.Reader ordsReader = PackedInts.getReader(index); @@ -583,9 +576,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer { public void lookupOrd(int ord, BytesRef result) { long startAddress = addressReader.get(ord); long endAddress = addressReader.get(ord+1); - result.bytes = bytes; - result.offset = (int)startAddress; - result.length = (int)(endAddress - startAddress); + bytesReader.fillSlice(result, startAddress, (int)(endAddress - startAddress)); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java index a8aa811c1d5..0bf60fecdae 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java @@ -202,7 +202,4 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer { fst.save(data); meta.writeVInt((int)ord); } - - // nocommit: can/should we make override merge + make it smarter to pull the values - // directly from disk for fields that arent already loaded up in ram? } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java index 6c4ce019a8a..b165aea63dd 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java @@ -35,6 +35,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IntsRef; +import org.apache.lucene.util.PagedBytes; import org.apache.lucene.util.fst.BytesRefFSTEnum; import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput; import org.apache.lucene.util.fst.FST; @@ -181,17 +182,15 @@ class Lucene42DocValuesProducer extends DocValuesProducer { private BinaryDocValues loadBinary(FieldInfo field) throws IOException { BinaryEntry entry = binaries.get(field.number); data.seek(entry.offset); - assert entry.numBytes < Integer.MAX_VALUE; // nocommit - final byte[] bytes = new byte[(int)entry.numBytes]; - data.readBytes(bytes, 0, bytes.length); + PagedBytes bytes = new PagedBytes(16); + bytes.copy(data, entry.numBytes); + final PagedBytes.Reader bytesReader = bytes.freeze(true); if (entry.minLength == entry.maxLength) { final int fixedLength = entry.minLength; return new BinaryDocValues() { @Override public void get(int docID, BytesRef result) { - result.bytes = bytes; - result.offset = docID * fixedLength; - result.length = fixedLength; + bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength); } }; } else { @@ -201,9 +200,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer { public void get(int docID, BytesRef result) { long startAddress = docID == 0 ? 0 : addresses.get(docID-1); long endAddress = addresses.get(docID); - result.bytes = bytes; - result.offset = (int) startAddress; - result.length = (int) (endAddress - startAddress); + bytesReader.fillSlice(result, startAddress, (int) (endAddress - startAddress)); } }; } diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java index f51caf4df53..6b92b7e5ae3 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java @@ -37,11 +37,9 @@ class BinaryDocValuesWriter extends DocValuesWriter { private int addedValues = 0; private final BytesRef emptyBytesRef = new BytesRef(); - // nocommit this needs to update bytesUsed? - public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) { this.fieldInfo = fieldInfo; - this.bytesRefArray = new BytesRefArray(iwBytesUsed); // nocommit: test that this thing really accounts correctly + this.bytesRefArray = new BytesRefArray(iwBytesUsed); } public void addValue(int docID, BytesRef value) { diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index cfeb3b6b4f7..ea3ef07a51d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -1293,7 +1293,6 @@ public class CheckIndex { } private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) { - // nocommit what else to check ... BytesRef scratch = new BytesRef(); for (int i = 0; i < reader.maxDoc(); i++) { dv.get(i, scratch); @@ -1360,7 +1359,6 @@ public class CheckIndex { } private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) { - // nocommit what else to check! for (int i = 0; i < reader.maxDoc(); i++) { ndv.get(i); } diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java index 368e67f7f79..7388e6a3d53 100644 --- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java @@ -24,8 +24,6 @@ import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.util.Counter; import org.apache.lucene.util.packed.AppendingLongBuffer; -// nocommit pick numeric or number ... then fix all places ... - /** Buffers up pending long per doc, then flushes when * segment flushes. */ class NumericDocValuesWriter extends DocValuesWriter { diff --git a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java index 4b112867009..265ccea79d4 100644 --- a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java +++ b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java @@ -280,7 +280,7 @@ public final class ByteBlockPool { // Fill in a BytesRef from term's length & bytes encoded in // byte block - public final BytesRef setBytesRef(BytesRef term, int textStart) { + public final void setBytesRef(BytesRef term, int textStart) { final byte[] bytes = term.bytes = buffers[textStart >> BYTE_BLOCK_SHIFT]; int pos = textStart & BYTE_BLOCK_MASK; if ((bytes[pos] & 0x80) == 0) { @@ -293,7 +293,6 @@ public final class ByteBlockPool { term.offset = pos+2; } assert term.length >= 0; - return term; } /** @@ -322,10 +321,10 @@ public final class ByteBlockPool { /** * Copies bytes from the pool starting at the given offset with the given - * length into the given {@link BytesRef} at offset 0 and returns it. + * length into the given {@link BytesRef} at offset 0. *

Note: this method allows to copy across block boundaries.

*/ - public final BytesRef copyFrom(final BytesRef bytes, final int offset, final int length) { + public final void copyFrom(final BytesRef bytes, final int offset, final int length) { bytes.offset = 0; bytes.grow(length); bytes.length = length; @@ -349,7 +348,6 @@ public final class ByteBlockPool { overflow = overflow - BYTE_BLOCK_SIZE; } } while (true); - return bytes; } } diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java index 38ab5b14cc4..c1a616514d7 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java +++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java @@ -38,13 +38,6 @@ public final class BytesRefArray { private int currentOffset = 0; private final Counter bytesUsed; - /** - * Creates a new {@link BytesRefArray} - */ - public BytesRefArray() { - this(Counter.newCounter(false)); - } - /** * Creates a new {@link BytesRefArray} with a counter to track allocated bytes */ @@ -112,17 +105,6 @@ public final class BytesRefArray { } - /** - * Returns the number internally used bytes to hold the appended bytes in - * memory - * - * @return the number internally used bytes to hold the appended bytes in - * memory - */ - public long bytesUsed() { - return bytesUsed.get(); - } - private int[] sort(final Comparator comp) { final int[] orderedEntries = new int[size()]; for (int i = 0; i < orderedEntries.length; i++) { diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java index 9a10c5beb1a..88b24396ed6 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java +++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java @@ -118,7 +118,8 @@ public final class BytesRefHash { public BytesRef get(int ord, BytesRef ref) { assert bytesStart != null : "bytesStart is null - not initialized"; assert ord < bytesStart.length: "ord exceeds byteStart len: " + bytesStart.length; - return pool.setBytesRef(ref, bytesStart[ord]); + pool.setBytesRef(ref, bytesStart[ord]); + return ref; } /** @@ -171,8 +172,9 @@ public final class BytesRefHash { protected int compare(int i, int j) { final int ord1 = compact[i], ord2 = compact[j]; assert bytesStart.length > ord1 && bytesStart.length > ord2; - return comp.compare(pool.setBytesRef(scratch1, bytesStart[ord1]), - pool.setBytesRef(scratch2, bytesStart[ord2])); + pool.setBytesRef(scratch1, bytesStart[ord1]); + pool.setBytesRef(scratch2, bytesStart[ord2]); + return comp.compare(scratch1, scratch2); } @Override @@ -186,8 +188,8 @@ public final class BytesRefHash { protected int comparePivot(int j) { final int ord = compact[j]; assert bytesStart.length > ord; - return comp.compare(pivot, - pool.setBytesRef(scratch2, bytesStart[ord])); + pool.setBytesRef(scratch2, bytesStart[ord]); + return comp.compare(pivot, scratch2); } private final BytesRef pivot = new BytesRef(), @@ -197,7 +199,8 @@ public final class BytesRefHash { } private boolean equals(int ord, BytesRef b) { - return pool.setBytesRef(scratch1, bytesStart[ord]).bytesEquals(b); + pool.setBytesRef(scratch1, bytesStart[ord]); + return scratch1.bytesEquals(b); } private boolean shrink(int targetSize) { diff --git a/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java b/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java index 8aad226bbfe..b5e9ef5a2b7 100644 --- a/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java +++ b/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java @@ -55,7 +55,7 @@ public final class PagedBytes { private final int blockMask; private final int blockSize; - public Reader(PagedBytes pagedBytes) { + private Reader(PagedBytes pagedBytes) { blocks = new byte[pagedBytes.blocks.size()][]; for(int i=0;i * @lucene.internal **/ - public BytesRef fillSlice(BytesRef b, long start, int length) { + public void fillSlice(BytesRef b, long start, int length) { assert length >= 0: "length=" + length; assert length <= blockSize+1; final int index = (int) (start >> blockBits); @@ -95,7 +95,6 @@ public final class PagedBytes { System.arraycopy(blocks[index], offset, b.bytes, 0, blockSize-offset); System.arraycopy(blocks[1+index], 0, b.bytes, blockSize-offset, length-(blockSize-offset)); } - return b; } /** @@ -105,12 +104,10 @@ public final class PagedBytes { * borders. *

* - * @return the given {@link BytesRef} - * * @lucene.internal **/ // nocommit: move this shit and any other vint bogusness to fieldcacheimpl! - public BytesRef fill(BytesRef b, long start) { + public void fill(BytesRef b, long start) { final int index = (int) (start >> blockBits); final int offset = (int) (start & blockMask); final byte[] block = b.bytes = blocks[index]; @@ -123,7 +120,6 @@ public final class PagedBytes { b.offset = offset+2; assert b.length > 0; } - return b; } } @@ -208,7 +204,7 @@ public final class PagedBytes { blockEnd.add(upto); frozen = true; currentBlock = null; - return new Reader(this); + return new PagedBytes.Reader(this); } public long getPointer() { diff --git a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java index f55466c23ee..7136de1ac8e 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java @@ -30,7 +30,7 @@ public class TestBytesRefArray extends LuceneTestCase { public void testAppend() throws IOException { Random random = random(); - BytesRefArray list = new BytesRefArray(); + BytesRefArray list = new BytesRefArray(Counter.newCounter()); List stringList = new ArrayList(); for (int j = 0; j < 2; j++) { if (j > 0 && random.nextBoolean()) { @@ -71,7 +71,7 @@ public class TestBytesRefArray extends LuceneTestCase { public void testSort() throws IOException { Random random = random(); - BytesRefArray list = new BytesRefArray(); + BytesRefArray list = new BytesRefArray(Counter.newCounter()); List stringList = new ArrayList(); for (int j = 0; j < 2; j++) { diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java index 43f843c2a9c..00d3c1c2b40 100644 --- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java +++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java @@ -356,9 +356,6 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase { case NUMERIC: valuesField = new NumericDocValuesField(dvField, Integer.parseInt(value)); break; - /* nocommit: case FLOAT_64: - valuesField = new DoubleDocValuesField(dvField, Double.parseDouble(value)); - break; */ case BINARY: valuesField = new BinaryDocValuesField(dvField, new BytesRef(value)); break; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java index 1bc9d72f24b..f0347cb0c8e 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java @@ -23,6 +23,7 @@ import org.apache.lucene.search.spell.TermFreqIterator; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefArray; +import org.apache.lucene.util.Counter; /** * This wrapper buffers incoming elements. @@ -31,7 +32,7 @@ import org.apache.lucene.util.BytesRefArray; public class BufferingTermFreqIteratorWrapper implements TermFreqIterator { // TODO keep this for now /** buffered term entries */ - protected BytesRefArray entries = new BytesRefArray(); + protected BytesRefArray entries = new BytesRefArray(Counter.newCounter()); /** current buffer position */ protected int curPos = -1; /** buffered weights, parallel with {@link #entries} */ diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java index 757b5178712..adeb5398790 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java @@ -22,6 +22,7 @@ import java.util.Comparator; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; import org.apache.lucene.util.BytesRefArray; +import org.apache.lucene.util.Counter; /** * An {@link BytesRefSorter} that keeps all the entries in memory. @@ -29,7 +30,7 @@ import org.apache.lucene.util.BytesRefArray; * @lucene.internal */ public final class InMemorySorter implements BytesRefSorter { - private final BytesRefArray buffer = new BytesRefArray(); + private final BytesRefArray buffer = new BytesRefArray(Counter.newCounter()); private boolean closed = false; private final Comparator comparator; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java index 815fe9faaae..463cc993186 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java @@ -155,7 +155,8 @@ public final class Sort { private final BufferSize ramBufferSize; private final File tempDirectory; - private final BytesRefArray buffer = new BytesRefArray(); + private final Counter bufferBytesUsed = Counter.newCounter(); + private final BytesRefArray buffer = new BytesRefArray(bufferBytesUsed); private SortInfo sortInfo; private int maxTempFiles; private final Comparator comparator; @@ -396,7 +397,7 @@ public final class Sort { buffer.append(scratch); // Account for the created objects. // (buffer slots do not account to buffer size.) - if (ramBufferSize.bytes < buffer.bytesUsed()) { + if (ramBufferSize.bytes < bufferBytesUsed.get()) { break; } } diff --git a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java index 9f50e0c0980..5e5a81d4e4b 100644 --- a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java +++ b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java @@ -55,7 +55,7 @@ public class SchemaCodecFactory extends CodecFactory implements SchemaAware { } return super.getPostingsFormatForField(field); } - // nocommit: dv too + // TODO: when dv support is added to solr, add it here too }; }