From 85c00e77efdf53f30da6eaffd38c2b016a7805bc Mon Sep 17 00:00:00 2001 From: Dawid Weiss Date: Tue, 8 May 2018 10:32:11 +0200 Subject: [PATCH] LUCENE-8267: removed references to memory codecs. --- lucene/CHANGES.txt | 3 + lucene/MIGRATE.txt | 4 + .../memory/MemoryDocValuesConsumer.java | 539 --------- .../codecs/memory/MemoryDocValuesFormat.java | 72 -- .../memory/MemoryDocValuesProducer.java | 877 -------------- .../codecs/memory/MemoryPostingsFormat.java | 1037 ----------------- .../org.apache.lucene.codecs.DocValuesFormat | 1 - .../org.apache.lucene.codecs.PostingsFormat | 2 +- .../memory/TestMemoryDocValuesFormat.java | 39 - .../memory/TestMemoryPostingsFormat.java | 35 - .../org/apache/lucene/TestExternalCodecs.java | 5 +- .../perfield/TestPerFieldDocValuesFormat.java | 2 +- .../perfield/TestPerFieldPostingsFormat2.java | 24 +- .../lucene/index/Test2BBinaryDocValues.java | 2 +- .../org/apache/lucene/index/Test2BDocs.java | 2 +- .../lucene/index/Test2BNumericDocValues.java | 2 +- .../org/apache/lucene/index/Test2BPoints.java | 2 +- .../apache/lucene/index/Test2BPositions.java | 2 +- .../apache/lucene/index/Test2BPostings.java | 2 +- .../lucene/index/Test2BPostingsBytes.java | 2 +- .../Test2BSortedDocValuesFixedSorted.java | 2 +- .../index/Test2BSortedDocValuesOrds.java | 2 +- .../org/apache/lucene/index/Test2BTerms.java | 2 +- .../apache/lucene/index/TestAddIndexes.java | 7 +- .../lucene/index/TestBagOfPositions.java | 2 +- .../lucene/index/TestBagOfPostings.java | 2 +- .../index/TestDuelingCodecsAtNight.java | 2 +- .../lucene/index/TestIndexWriterCommit.java | 1 - .../lucene/index/TestIndexWriterDelete.java | 1 - .../lucene/index/TestIndexWriterMaxDocs.java | 2 +- .../index/TestIndexWriterOnDiskFull.java | 1 - .../lucene/index/TestLazyProxSkipping.java | 1 - .../apache/lucene/index/TestLongPostings.java | 2 +- .../apache/lucene/index/TestNRTThreads.java | 2 +- .../org/apache/lucene/index/TestNorms.java | 2 +- .../lucene/index/TestRollingUpdates.java | 6 +- .../apache/lucene/index/TestTermsEnum.java | 2 +- .../TestControlledRealTimeReopenThread.java | 2 +- .../lucene/search/TestSearchWithThreads.java | 2 +- .../lucene/search/TestSearcherManager.java | 2 +- .../lucene/search/TestShardSearching.java | 2 +- .../lucene/search/TestSortedSetSelector.java | 2 +- .../org/apache/lucene/util/fst/TestFSTs.java | 2 +- .../search/TestInetAddressRangeQueries.java | 2 +- .../lucene/replicator/nrt/SimpleServer.java | 2 +- .../replicator/nrt/TestNRTReplication.java | 2 +- .../nrt/TestStressNRTReplication.java | 2 +- .../analysis/BaseTokenStreamTestCase.java | 4 +- .../lucene/geo/BaseGeoPointTestCase.java | 1 - .../org/apache/lucene/index/RandomCodec.java | 7 +- .../java/org/apache/lucene/util/TestUtil.java | 2 +- solr/CHANGES.txt | 4 + .../solr/collection1/conf/schema_codec.xml | 5 +- .../apache/solr/core/TestCodecSupport.java | 9 +- .../TestMinMaxOnMultiValuedField.java | 2 +- .../function/TestSortByMinMaxFunction.java | 2 +- .../TestDocTermOrdsUninvertLimit.java | 1 - .../TestFieldCacheVsDocValues.java | 111 +- 58 files changed, 62 insertions(+), 2799 deletions(-) delete mode 100644 lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java delete mode 100644 lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java delete mode 100644 lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java delete mode 100644 lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java delete mode 100644 lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java delete mode 100644 lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index ec68882864e..f6477f3d192 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -41,6 +41,9 @@ API Changes * LUCENE-8242: Deprecated method IndexSearcher#createNormalizedWeight() has been removed (Alan Woodward) +* LUCENE-8267: Memory codecs removed from the codebase (MemoryPostings, + MemoryDocValues). (Dawid Weiss) + Changes in Runtime Behavior * LUCENE-7837: Indices that were created before the previous major version diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt index 2825bcaced5..0f9fcf68cab 100644 --- a/lucene/MIGRATE.txt +++ b/lucene/MIGRATE.txt @@ -43,3 +43,7 @@ the same index options for that field. Instead use IndexSearcher.createWeight(), rewriting the query first, and using a boost of 1f. + +## Memory codecs removed (LUCENE-8267) ## + +Memory codecs have been removed from the codebase (MemoryPostings, MemoryDocValues). diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java deleted file mode 100644 index eb7d0969779..00000000000 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java +++ /dev/null @@ -1,539 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - - -import java.io.IOException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.NoSuchElementException; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.DocValuesConsumer; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.LegacyDocValuesIterables; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.store.ByteArrayDataOutput; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.IntsRefBuilder; -import org.apache.lucene.util.MathUtil; -import org.apache.lucene.util.fst.Builder; -import org.apache.lucene.util.fst.FST.INPUT_TYPE; -import org.apache.lucene.util.fst.FST; -import org.apache.lucene.util.fst.PositiveIntOutputs; -import org.apache.lucene.util.fst.Util; -import org.apache.lucene.util.packed.BlockPackedWriter; -import org.apache.lucene.util.packed.MonotonicBlockPackedWriter; -import org.apache.lucene.util.packed.PackedInts.FormatAndBits; -import org.apache.lucene.util.packed.PackedInts; - -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_COMPRESSED; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_SIZE; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BYTES; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.DELTA_COMPRESSED; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC_SINGLETON; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET_SINGLETON; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.TABLE_COMPRESSED; -import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.VERSION_CURRENT; - -/** - * Writer for {@link MemoryDocValuesFormat} - */ -class MemoryDocValuesConsumer extends DocValuesConsumer { - IndexOutput data, meta; - final int maxDoc; - final float acceptableOverheadRatio; - - MemoryDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException { - this.acceptableOverheadRatio = acceptableOverheadRatio; - maxDoc = state.segmentInfo.maxDoc(); - boolean success = false; - try { - String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); - data = state.directory.createOutput(dataName, state.context); - CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); - meta = state.directory.createOutput(metaName, state.context); - CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - success = true; - } finally { - if (!success) { - IOUtils.closeWhileHandlingException(this); - } - } - } - - @Override - public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), true); - } - - void addNumericField(FieldInfo field, Iterable values, boolean optimizeStorage) throws IOException { - meta.writeVInt(field.number); - meta.writeByte(NUMBER); - meta.writeLong(data.getFilePointer()); - long minValue = Long.MAX_VALUE; - long maxValue = Long.MIN_VALUE; - long blockSum = 0; - long gcd = 0; - boolean missing = false; - // TODO: more efficient? - HashSet uniqueValues = null; - long count = 0; - - if (optimizeStorage) { - uniqueValues = new HashSet<>(); - - long currentBlockMin = Long.MAX_VALUE; - long currentBlockMax = Long.MIN_VALUE; - for (Number nv : values) { - final long v; - if (nv == null) { - v = 0; - missing = true; - } else { - v = nv.longValue(); - } - - if (gcd != 1) { - if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) { - // in that case v - minValue might overflow and make the GCD computation return - // wrong results. Since these extreme values are unlikely, we just discard - // GCD computation for them - gcd = 1; - } else if (count != 0) { // minValue needs to be set first - gcd = MathUtil.gcd(gcd, v - minValue); - } - } - - currentBlockMin = Math.min(minValue, v); - currentBlockMax = Math.max(maxValue, v); - - minValue = Math.min(minValue, v); - maxValue = Math.max(maxValue, v); - - if (uniqueValues != null) { - if (uniqueValues.add(v)) { - if (uniqueValues.size() > 256) { - uniqueValues = null; - } - } - } - - ++count; - if (count % BLOCK_SIZE == 0) { - final long blockDelta = currentBlockMax - currentBlockMin; - final int blockDeltaRequired = PackedInts.unsignedBitsRequired(blockDelta); - final int blockBPV = PackedInts.fastestFormatAndBits(BLOCK_SIZE, blockDeltaRequired, acceptableOverheadRatio).bitsPerValue; - blockSum += blockBPV; - currentBlockMax = Long.MIN_VALUE; - currentBlockMin = Long.MAX_VALUE; - } - } - } else { - for (Number nv : values) { - long v = nv.longValue(); - maxValue = Math.max(v, maxValue); - minValue = Math.min(v, minValue); - count++; - } - } - - if (missing) { - long start = data.getFilePointer(); - writeMissingBitset(values); - meta.writeLong(start); - meta.writeLong(data.getFilePointer() - start); - } else { - meta.writeLong(-1L); - } - - final long delta = maxValue - minValue; - final int deltaRequired = delta < 0 ? 64 : PackedInts.bitsRequired(delta); - final FormatAndBits deltaBPV = PackedInts.fastestFormatAndBits(maxDoc, deltaRequired, acceptableOverheadRatio); - - final FormatAndBits tableBPV; - if (count < Integer.MAX_VALUE && uniqueValues != null) { - tableBPV = PackedInts.fastestFormatAndBits(maxDoc, PackedInts.bitsRequired(uniqueValues.size()-1), acceptableOverheadRatio); - } else { - tableBPV = null; - } - - final FormatAndBits gcdBPV; - if (count < Integer.MAX_VALUE && gcd != 0 && gcd != 1) { - final long gcdDelta = (maxValue - minValue) / gcd; - final int gcdRequired = gcdDelta < 0 ? 64 : PackedInts.bitsRequired(gcdDelta); - gcdBPV = PackedInts.fastestFormatAndBits(maxDoc, gcdRequired, acceptableOverheadRatio); - } else { - gcdBPV = null; - } - - boolean doBlock = false; - if (blockSum != 0) { - int numBlocks = maxDoc / BLOCK_SIZE; - float avgBPV = blockSum / (float)numBlocks; - // just a heuristic, with tiny amounts of blocks our estimate is skewed as we ignore the final "incomplete" block. - // with at least 4 blocks it's pretty accurate. The difference must also be significant (according to acceptable overhead). - if (numBlocks >= 4 && (avgBPV+avgBPV*acceptableOverheadRatio) < deltaBPV.bitsPerValue) { - doBlock = true; - } - } - // blockpackedreader allows us to read in huge streams of ints - if (count >= Integer.MAX_VALUE) { - doBlock = true; - } - - if (tableBPV != null && (tableBPV.bitsPerValue+tableBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) { - // small number of unique values - meta.writeByte(TABLE_COMPRESSED); // table-compressed - Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]); - final HashMap encode = new HashMap<>(); - int length = 1 << tableBPV.bitsPerValue; - data.writeVInt(length); - for (int i = 0; i < decode.length; i++) { - data.writeLong(decode[i]); - encode.put(decode[i], i); - } - for (int i = decode.length; i < length; i++) { - data.writeLong(0); - } - - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeLong(count); - data.writeVInt(tableBPV.format.getId()); - data.writeVInt(tableBPV.bitsPerValue); - - final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, tableBPV.format, (int)count, tableBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); - for(Number nv : values) { - writer.add(encode.get(nv == null ? 0 : nv.longValue())); - } - writer.finish(); - } else if (gcdBPV != null && (gcdBPV.bitsPerValue+gcdBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) { - meta.writeByte(GCD_COMPRESSED); - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeLong(count); - data.writeLong(minValue); - data.writeLong(gcd); - data.writeVInt(gcdBPV.format.getId()); - data.writeVInt(gcdBPV.bitsPerValue); - - final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, gcdBPV.format, (int)count, gcdBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); - for (Number nv : values) { - long value = nv == null ? 0 : nv.longValue(); - writer.add((value - minValue) / gcd); - } - writer.finish(); - } else if (doBlock) { - meta.writeByte(BLOCK_COMPRESSED); // block delta-compressed - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeLong(count); - data.writeVInt(BLOCK_SIZE); - final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE); - for (Number nv : values) { - writer.add(nv == null ? 0 : nv.longValue()); - } - writer.finish(); - } else { - meta.writeByte(DELTA_COMPRESSED); // delta-compressed - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeLong(count); - final long minDelta = deltaBPV.bitsPerValue == 64 ? 0 : minValue; - data.writeLong(minDelta); - data.writeVInt(deltaBPV.format.getId()); - data.writeVInt(deltaBPV.bitsPerValue); - - final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, deltaBPV.format, (int)count, deltaBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE); - for (Number nv : values) { - long v = nv == null ? 0 : nv.longValue(); - writer.add(v - minDelta); - } - writer.finish(); - } - } - - @Override - public void close() throws IOException { - boolean success = false; - try { - if (meta != null) { - meta.writeVInt(-1); // write EOF marker - CodecUtil.writeFooter(meta); // write checksum - } - if (data != null) { - CodecUtil.writeFooter(data); - } - success = true; - } finally { - if (success) { - IOUtils.close(data, meta); - } else { - IOUtils.closeWhileHandlingException(data, meta); - } - data = meta = null; - } - } - - @Override - public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc)); - } - - private void addBinaryField(FieldInfo field, final Iterable values) throws IOException { - // write the byte[] data - meta.writeVInt(field.number); - meta.writeByte(BYTES); - int minLength = Integer.MAX_VALUE; - int maxLength = Integer.MIN_VALUE; - final long startFP = data.getFilePointer(); - boolean missing = false; - int upto = 0; - for(BytesRef v : values) { - final int length; - if (v == null) { - length = 0; - missing = true; - } else { - length = v.length; - } - if (length > MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH) { - throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH + " but got length=" + length + " v=" + v + "; upto=" + upto + " values=" + values); - } - upto++; - minLength = Math.min(minLength, length); - maxLength = Math.max(maxLength, length); - if (v != null) { - data.writeBytes(v.bytes, v.offset, v.length); - } - } - meta.writeLong(startFP); - meta.writeLong(data.getFilePointer() - startFP); - if (missing) { - long start = data.getFilePointer(); - writeMissingBitset(values); - meta.writeLong(start); - meta.writeLong(data.getFilePointer() - start); - } else { - meta.writeLong(-1L); - } - meta.writeVInt(minLength); - meta.writeVInt(maxLength); - - // if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit) - // otherwise, we need to record the length fields... - if (minLength != maxLength) { - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeVInt(BLOCK_SIZE); - - final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE); - long addr = 0; - for (BytesRef v : values) { - if (v != null) { - addr += v.length; - } - writer.add(addr); - } - writer.finish(); - } - } - - private void writeFST(FieldInfo field, Iterable values) throws IOException { - meta.writeVInt(field.number); - meta.writeByte(FST); - meta.writeLong(data.getFilePointer()); - PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(); - Builder builder = new Builder<>(INPUT_TYPE.BYTE1, outputs); - IntsRefBuilder scratch = new IntsRefBuilder(); - long ord = 0; - for (BytesRef v : values) { - builder.add(Util.toIntsRef(v, scratch), ord); - ord++; - } - FST fst = builder.finish(); - if (fst != null) { - fst.save(data); - } - meta.writeVLong(ord); - } - - // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on, - // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode) - void writeMissingBitset(Iterable values) throws IOException { - long bits = 0; - int count = 0; - for (Object v : values) { - if (count == 64) { - data.writeLong(bits); - count = 0; - bits = 0; - } - if (v != null) { - bits |= 1L << (count & 0x3f); - } - count++; - } - if (count > 0) { - data.writeLong(bits); - } - } - - @Override - public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - addSortedField(field, - LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)), - LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc)); - } - - private void addSortedField(FieldInfo field, Iterable values, Iterable docToOrd) throws IOException { - // write the ordinals as numerics - addNumericField(field, docToOrd, false); - - // write the values as FST - writeFST(field, values); - } - - @Override - public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { - - final Iterable docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc); - final Iterable values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field); - - meta.writeVInt(field.number); - - if (isSingleValued(docToValueCount)) { - meta.writeByte(SORTED_NUMERIC_SINGLETON); - addNumericField(field, singletonView(docToValueCount, values, null), true); - } else { - meta.writeByte(SORTED_NUMERIC); - - // write the addresses: - meta.writeVInt(PackedInts.VERSION_CURRENT); - meta.writeVInt(BLOCK_SIZE); - meta.writeLong(data.getFilePointer()); - final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE); - long addr = 0; - writer.add(addr); - for (Number v : docToValueCount) { - addr += v.longValue(); - writer.add(addr); - } - writer.finish(); - long valueCount = writer.ord(); - meta.writeLong(valueCount); - - // write the values - addNumericField(field, values, true); - } - } - - // note: this might not be the most efficient... but it's fairly simple - @Override - public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - Iterable values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field)); - Iterable docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc); - Iterable ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field); - meta.writeVInt(field.number); - - if (isSingleValued(docToOrdCount)) { - meta.writeByte(SORTED_SET_SINGLETON); - addSortedField(field, values, singletonView(docToOrdCount, ords, -1L)); - } else { - meta.writeByte(SORTED_SET); - // write the ordinals as a binary field - addBinaryField(field, new Iterable() { - @Override - public Iterator iterator() { - return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator()); - } - }); - - // write the values as FST - writeFST(field, values); - } - } - - // per-document vint-encoded byte[] - static class SortedSetIterator implements Iterator { - byte[] buffer = new byte[10]; - ByteArrayDataOutput out = new ByteArrayDataOutput(); - BytesRef ref = new BytesRef(); - - final Iterator counts; - final Iterator ords; - - SortedSetIterator(Iterator counts, Iterator ords) { - this.counts = counts; - this.ords = ords; - } - - @Override - public boolean hasNext() { - return counts.hasNext(); - } - - @Override - public BytesRef next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - int count = counts.next().intValue(); - int maxSize = count*9; // worst case - if (maxSize > buffer.length) { - buffer = ArrayUtil.grow(buffer, maxSize); - } - - try { - encodeValues(count); - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - - ref.bytes = buffer; - ref.offset = 0; - ref.length = out.getPosition(); - - return ref; - } - - // encodes count values to buffer - private void encodeValues(int count) throws IOException { - out.reset(buffer); - long lastOrd = 0; - for (int i = 0; i < count; i++) { - long ord = ords.next().longValue(); - out.writeVLong(ord - lastOrd); - lastOrd = ord; - } - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - } -} diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java deleted file mode 100644 index 4624ec01b05..00000000000 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - - -import java.io.IOException; - -import org.apache.lucene.codecs.DocValuesConsumer; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.DocValuesFormat; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.util.packed.PackedInts; - -/** In-memory docvalues format */ -public class MemoryDocValuesFormat extends DocValuesFormat { - - /** Maximum length for each binary doc values field. */ - public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2; - - final float acceptableOverheadRatio; - - /** - * Calls {@link #MemoryDocValuesFormat(float) - * MemoryDocValuesFormat(PackedInts.DEFAULT)} - */ - public MemoryDocValuesFormat() { - this(PackedInts.DEFAULT); - } - - /** - * Creates a new MemoryDocValuesFormat with the specified - * acceptableOverheadRatio for NumericDocValues. - * @param acceptableOverheadRatio compression parameter for numerics. - * Currently this is only used when the number of unique values is small. - * - * @lucene.experimental - */ - public MemoryDocValuesFormat(float acceptableOverheadRatio) { - super("Memory"); - this.acceptableOverheadRatio = acceptableOverheadRatio; - } - - @Override - public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return new MemoryDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio); - } - - @Override - public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException { - return new MemoryDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); - } - - static final String DATA_CODEC = "MemoryDocValuesData"; - static final String DATA_EXTENSION = "mdvd"; - static final String METADATA_CODEC = "MemoryDocValuesMetadata"; - static final String METADATA_EXTENSION = "mdvm"; -} diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java deleted file mode 100644 index ac81360c5d6..00000000000 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java +++ /dev/null @@ -1,877 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.index.*; -import org.apache.lucene.store.ByteArrayDataInput; -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.util.Accountable; -import org.apache.lucene.util.Accountables; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.IntsRef; -import org.apache.lucene.util.IntsRefBuilder; -import org.apache.lucene.util.PagedBytes; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.fst.BytesRefFSTEnum; -import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput; -import org.apache.lucene.util.fst.FST; -import org.apache.lucene.util.fst.FST.Arc; -import org.apache.lucene.util.fst.FST.BytesReader; -import org.apache.lucene.util.fst.PositiveIntOutputs; -import org.apache.lucene.util.fst.Util; -import org.apache.lucene.util.packed.BlockPackedReader; -import org.apache.lucene.util.packed.MonotonicBlockPackedReader; -import org.apache.lucene.util.packed.PackedInts; - -/** - * Reader for {@link MemoryDocValuesFormat} - */ -class MemoryDocValuesProducer extends DocValuesProducer { - // metadata maps (just file pointers and minimal stuff) - private final Map numerics = new HashMap<>(); - private final Map binaries = new HashMap<>(); - private final Map fsts = new HashMap<>(); - private final Map sortedSets = new HashMap<>(); - private final Map sortedNumerics = new HashMap<>(); - private final IndexInput data; - - // ram instances we have already loaded - private final Map numericInstances = new HashMap<>(); - private final Map pagedBytesInstances = new HashMap<>(); - private final Map> fstInstances = new HashMap<>(); - private final Map docsWithFieldInstances = new HashMap<>(); - private final Map addresses = new HashMap<>(); - - private final Map numericInfo = new HashMap<>(); - - private final int numEntries; - private final int maxDoc; - private final AtomicLong ramBytesUsed; - private final int version; - - private final boolean merging; - - static final byte NUMBER = 0; - static final byte BYTES = 1; - static final byte FST = 2; - static final byte SORTED_SET = 4; - static final byte SORTED_SET_SINGLETON = 5; - static final byte SORTED_NUMERIC = 6; - static final byte SORTED_NUMERIC_SINGLETON = 7; - - static final int BLOCK_SIZE = 4096; - - static final byte DELTA_COMPRESSED = 0; - static final byte TABLE_COMPRESSED = 1; - static final byte BLOCK_COMPRESSED = 2; - static final byte GCD_COMPRESSED = 3; - - static final int VERSION_START = 4; - static final int VERSION_CURRENT = VERSION_START; - - // clone for merge: when merging we don't do any instances.put()s - MemoryDocValuesProducer(MemoryDocValuesProducer original) throws IOException { - assert Thread.holdsLock(original); - numerics.putAll(original.numerics); - binaries.putAll(original.binaries); - fsts.putAll(original.fsts); - sortedSets.putAll(original.sortedSets); - sortedNumerics.putAll(original.sortedNumerics); - data = original.data.clone(); - - numericInstances.putAll(original.numericInstances); - pagedBytesInstances.putAll(original.pagedBytesInstances); - fstInstances.putAll(original.fstInstances); - docsWithFieldInstances.putAll(original.docsWithFieldInstances); - addresses.putAll(original.addresses); - - numericInfo.putAll(original.numericInfo); - - numEntries = original.numEntries; - maxDoc = original.maxDoc; - ramBytesUsed = new AtomicLong(original.ramBytesUsed.get()); - version = original.version; - merging = true; - } - - MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { - maxDoc = state.segmentInfo.maxDoc(); - merging = false; - String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); - // read in the entries from the metadata file. - ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context); - boolean success = false; - try { - version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, - state.segmentInfo.getId(), state.segmentSuffix); - numEntries = readFields(in, state.fieldInfos); - CodecUtil.checkFooter(in); - ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass())); - success = true; - } finally { - if (success) { - IOUtils.close(in); - } else { - IOUtils.closeWhileHandlingException(in); - } - } - - String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); - this.data = state.directory.openInput(dataName, state.context); - success = false; - try { - final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, - state.segmentInfo.getId(), state.segmentSuffix); - if (version != version2) { - throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data); - } - - // NOTE: data file is too costly to verify checksum against all the bytes on open, - // but for now we at least verify proper structure of the checksum footer: which looks - // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption - // such as file truncation. - CodecUtil.retrieveChecksum(data); - - success = true; - } finally { - if (!success) { - IOUtils.closeWhileHandlingException(this.data); - } - } - } - - private NumericEntry readNumericEntry(IndexInput meta) throws IOException { - NumericEntry entry = new NumericEntry(); - entry.offset = meta.readLong(); - entry.missingOffset = meta.readLong(); - if (entry.missingOffset != -1) { - entry.missingBytes = meta.readLong(); - } else { - entry.missingBytes = 0; - } - entry.format = meta.readByte(); - switch(entry.format) { - case DELTA_COMPRESSED: - case TABLE_COMPRESSED: - case BLOCK_COMPRESSED: - case GCD_COMPRESSED: - break; - default: - throw new CorruptIndexException("Unknown format: " + entry.format, meta); - } - entry.packedIntsVersion = meta.readVInt(); - entry.count = meta.readLong(); - return entry; - } - - private BinaryEntry readBinaryEntry(IndexInput meta) throws IOException { - BinaryEntry entry = new BinaryEntry(); - entry.offset = meta.readLong(); - entry.numBytes = meta.readLong(); - entry.missingOffset = meta.readLong(); - if (entry.missingOffset != -1) { - entry.missingBytes = meta.readLong(); - } else { - entry.missingBytes = 0; - } - entry.minLength = meta.readVInt(); - entry.maxLength = meta.readVInt(); - if (entry.minLength != entry.maxLength) { - entry.packedIntsVersion = meta.readVInt(); - entry.blockSize = meta.readVInt(); - } - return entry; - } - - private FSTEntry readFSTEntry(IndexInput meta) throws IOException { - FSTEntry entry = new FSTEntry(); - entry.offset = meta.readLong(); - entry.numOrds = meta.readVLong(); - return entry; - } - - private int readFields(IndexInput meta, FieldInfos infos) throws IOException { - int numEntries = 0; - int fieldNumber = meta.readVInt(); - while (fieldNumber != -1) { - numEntries++; - FieldInfo info = infos.fieldInfo(fieldNumber); - if (info == null) { - throw new CorruptIndexException("invalid field number: " + fieldNumber, meta); - } - int fieldType = meta.readByte(); - if (fieldType == NUMBER) { - numerics.put(info.name, readNumericEntry(meta)); - } else if (fieldType == BYTES) { - binaries.put(info.name, readBinaryEntry(meta)); - } else if (fieldType == FST) { - fsts.put(info.name,readFSTEntry(meta)); - } else if (fieldType == SORTED_SET) { - SortedSetEntry entry = new SortedSetEntry(); - entry.singleton = false; - sortedSets.put(info.name, entry); - } else if (fieldType == SORTED_SET_SINGLETON) { - SortedSetEntry entry = new SortedSetEntry(); - entry.singleton = true; - sortedSets.put(info.name, entry); - } else if (fieldType == SORTED_NUMERIC) { - SortedNumericEntry entry = new SortedNumericEntry(); - entry.singleton = false; - entry.packedIntsVersion = meta.readVInt(); - entry.blockSize = meta.readVInt(); - entry.addressOffset = meta.readLong(); - entry.valueCount = meta.readLong(); - sortedNumerics.put(info.name, entry); - } else if (fieldType == SORTED_NUMERIC_SINGLETON) { - SortedNumericEntry entry = new SortedNumericEntry(); - entry.singleton = true; - sortedNumerics.put(info.name, entry); - } else { - throw new CorruptIndexException("invalid entry type: " + fieldType + ", fieldName=" + info.name, meta); - } - fieldNumber = meta.readVInt(); - } - return numEntries; - } - - @Override - public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException { - LegacyNumericDocValues instance = numericInstances.get(field.name); - if (instance == null) { - instance = loadNumeric(field); - if (!merging) { - numericInstances.put(field.name, instance); - } - } - NumericEntry ne = numerics.get(field.name); - return new LegacyNumericDocValuesWrapper(getMissingBits(field, ne.missingOffset, ne.missingBytes), instance); - } - - private synchronized LegacyNumericDocValues getNumericNonIterator(FieldInfo field) throws IOException { - LegacyNumericDocValues instance = numericInstances.get(field.name); - if (instance == null) { - instance = loadNumeric(field); - if (!merging) { - numericInstances.put(field.name, instance); - } - } - return instance; - } - - @Override - public long ramBytesUsed() { - return ramBytesUsed.get(); - } - - @Override - public synchronized Collection getChildResources() { - List resources = new ArrayList<>(); - resources.addAll(Accountables.namedAccountables("numeric field", numericInfo)); - resources.addAll(Accountables.namedAccountables("pagedbytes field", pagedBytesInstances)); - resources.addAll(Accountables.namedAccountables("term dict field", fstInstances)); - resources.addAll(Accountables.namedAccountables("missing bitset field", docsWithFieldInstances)); - resources.addAll(Accountables.namedAccountables("addresses field", addresses)); - return Collections.unmodifiableList(resources); - } - - @Override - public void checkIntegrity() throws IOException { - CodecUtil.checksumEntireFile(data.clone()); - } - - @Override - public synchronized DocValuesProducer getMergeInstance() throws IOException { - return new MemoryDocValuesProducer(this); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(entries=" + numEntries + ")"; - } - - private LegacyNumericDocValues loadNumeric(FieldInfo field) throws IOException { - NumericEntry entry = numerics.get(field.name); - IndexInput data = this.data.clone(); - data.seek(entry.offset + entry.missingBytes); - switch (entry.format) { - case TABLE_COMPRESSED: - int size = data.readVInt(); - if (size > 256) { - throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + size, data); - } - final long decode[] = new long[size]; - for (int i = 0; i < decode.length; i++) { - decode[i] = data.readLong(); - } - final int formatID = data.readVInt(); - final int bitsPerValue = data.readVInt(); - final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, (int)entry.count, bitsPerValue); - if (!merging) { - ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed()); - numericInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader)); - } - return new LegacyNumericDocValues() { - @Override - public long get(int docID) { - return decode[(int)ordsReader.get(docID)]; - } - }; - case DELTA_COMPRESSED: - final long minDelta = data.readLong(); - final int formatIDDelta = data.readVInt(); - final int bitsPerValueDelta = data.readVInt(); - final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta); - if (!merging) { - ramBytesUsed.addAndGet(deltaReader.ramBytesUsed()); - numericInfo.put(field.name, Accountables.namedAccountable("delta compressed", deltaReader)); - } - return new LegacyNumericDocValues() { - @Override - public long get(int docID) { - return minDelta + deltaReader.get(docID); - } - }; - case BLOCK_COMPRESSED: - final int blockSize = data.readVInt(); - final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false); - if (!merging) { - ramBytesUsed.addAndGet(reader.ramBytesUsed()); - numericInfo.put(field.name, Accountables.namedAccountable("block compressed", reader)); - } - return new LegacyNumericDocValues() { - @Override - public long get(int docID) { - return reader.get(docID); - } - }; - case GCD_COMPRESSED: - final long min = data.readLong(); - final long mult = data.readLong(); - final int formatIDGCD = data.readVInt(); - final int bitsPerValueGCD = data.readVInt(); - final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD); - if (!merging) { - ramBytesUsed.addAndGet(quotientReader.ramBytesUsed()); - numericInfo.put(field.name, Accountables.namedAccountable("gcd compressed", quotientReader)); - } - return new LegacyNumericDocValues() { - @Override - public long get(int docID) { - return min + mult * quotientReader.get(docID); - } - }; - default: - throw new AssertionError(); - } - } - - private LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException { - BinaryEntry entry = binaries.get(field.name); - - BytesAndAddresses instance; - synchronized (this) { - instance = pagedBytesInstances.get(field.name); - if (instance == null) { - instance = loadBinary(field); - if (!merging) { - pagedBytesInstances.put(field.name, instance); - } - } - } - final PagedBytes.Reader bytesReader = instance.reader; - final MonotonicBlockPackedReader addresses = instance.addresses; - - if (addresses == null) { - assert entry.minLength == entry.maxLength; - final int fixedLength = entry.minLength; - return new LegacyBinaryDocValues() { - final BytesRef term = new BytesRef(); - - @Override - public BytesRef get(int docID) { - bytesReader.fillSlice(term, fixedLength * (long)docID, fixedLength); - return term; - } - }; - } else { - return new LegacyBinaryDocValues() { - final BytesRef term = new BytesRef(); - - @Override - public BytesRef get(int docID) { - long startAddress = docID == 0 ? 0 : addresses.get(docID-1); - long endAddress = addresses.get(docID); - bytesReader.fillSlice(term, startAddress, (int) (endAddress - startAddress)); - return term; - } - }; - } - } - - @Override - public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException { - BinaryEntry be = binaries.get(field.name); - return new LegacyBinaryDocValuesWrapper(getMissingBits(field, be.missingOffset, be.missingBytes), getLegacyBinary(field)); - } - - private BytesAndAddresses loadBinary(FieldInfo field) throws IOException { - BytesAndAddresses bytesAndAddresses = new BytesAndAddresses(); - BinaryEntry entry = binaries.get(field.name); - IndexInput data = this.data.clone(); - data.seek(entry.offset); - PagedBytes bytes = new PagedBytes(16); - bytes.copy(data, entry.numBytes); - bytesAndAddresses.reader = bytes.freeze(true); - if (!merging) { - ramBytesUsed.addAndGet(bytesAndAddresses.reader.ramBytesUsed()); - } - if (entry.minLength != entry.maxLength) { - data.seek(data.getFilePointer() + entry.missingBytes); - bytesAndAddresses.addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false); - if (!merging) { - ramBytesUsed.addAndGet(bytesAndAddresses.addresses.ramBytesUsed()); - } - } - return bytesAndAddresses; - } - - @Override - public SortedDocValues getSorted(FieldInfo field) throws IOException { - return new LegacySortedDocValuesWrapper(getSortedNonIterator(field), maxDoc); - } - - private LegacySortedDocValues getSortedNonIterator(FieldInfo field) throws IOException { - final FSTEntry entry = fsts.get(field.name); - if (entry.numOrds == 0) { - return DocValues.emptyLegacySorted(); - } - FST instance; - synchronized(this) { - instance = fstInstances.get(field.name); - if (instance == null) { - IndexInput data = this.data.clone(); - data.seek(entry.offset); - instance = new FST<>(data, PositiveIntOutputs.getSingleton()); - if (!merging) { - ramBytesUsed.addAndGet(instance.ramBytesUsed()); - fstInstances.put(field.name, instance); - } - } - } - final LegacyNumericDocValues docToOrd = getNumericNonIterator(field); - final FST fst = instance; - - // per-thread resources - final BytesReader in = fst.getBytesReader(); - final Arc firstArc = new Arc<>(); - final Arc scratchArc = new Arc<>(); - final IntsRefBuilder scratchInts = new IntsRefBuilder(); - final BytesRefFSTEnum fstEnum = new BytesRefFSTEnum<>(fst); - - return new LegacySortedDocValues() { - final BytesRefBuilder term = new BytesRefBuilder(); - - @Override - public int getOrd(int docID) { - return (int) docToOrd.get(docID); - } - - @Override - public BytesRef lookupOrd(int ord) { - try { - in.setPosition(0); - fst.getFirstArc(firstArc); - IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts); - return Util.toBytesRef(output, term); - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - @Override - public int lookupTerm(BytesRef key) { - try { - InputOutput o = fstEnum.seekCeil(key); - if (o == null) { - return -getValueCount()-1; - } else if (o.input.equals(key)) { - return o.output.intValue(); - } else { - return (int) -o.output-1; - } - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - @Override - public int getValueCount() { - return (int)entry.numOrds; - } - - @Override - public TermsEnum termsEnum() { - return new FSTTermsEnum(fst); - } - }; - } - - @Override - public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - SortedNumericEntry entry = sortedNumerics.get(field.name); - if (entry.singleton) { - LegacyNumericDocValues values = getNumericNonIterator(field); - NumericEntry ne = numerics.get(field.name); - Bits docsWithField = getMissingBits(field, ne.missingOffset, ne.missingBytes); - return DocValues.singleton(new LegacyNumericDocValuesWrapper(docsWithField, values)); - } else { - final LegacyNumericDocValues values = getNumericNonIterator(field); - final MonotonicBlockPackedReader addr; - synchronized (this) { - MonotonicBlockPackedReader res = addresses.get(field.name); - if (res == null) { - IndexInput data = this.data.clone(); - data.seek(entry.addressOffset); - res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false); - if (!merging) { - addresses.put(field.name, res); - ramBytesUsed.addAndGet(res.ramBytesUsed()); - } - } - addr = res; - } - return new LegacySortedNumericDocValuesWrapper(new LegacySortedNumericDocValues() { - int startOffset; - int endOffset; - - @Override - public void setDocument(int doc) { - startOffset = (int) addr.get(doc); - endOffset = (int) addr.get(doc+1); - } - - @Override - public long valueAt(int index) { - return values.get(startOffset + index); - } - - @Override - public int count() { - return (endOffset - startOffset); - } - }, maxDoc); - } - } - - @Override - public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - SortedSetEntry sortedSetEntry = sortedSets.get(field.name); - if (sortedSetEntry.singleton) { - return DocValues.singleton(getSorted(field)); - } - - final FSTEntry entry = fsts.get(field.name); - if (entry.numOrds == 0) { - return DocValues.emptySortedSet(); // empty FST! - } - FST instance; - synchronized(this) { - instance = fstInstances.get(field.name); - if (instance == null) { - IndexInput data = this.data.clone(); - data.seek(entry.offset); - instance = new FST<>(data, PositiveIntOutputs.getSingleton()); - if (!merging) { - ramBytesUsed.addAndGet(instance.ramBytesUsed()); - fstInstances.put(field.name, instance); - } - } - } - final LegacyBinaryDocValues docToOrds = getLegacyBinary(field); - final FST fst = instance; - - // per-thread resources - final BytesReader in = fst.getBytesReader(); - final Arc firstArc = new Arc<>(); - final Arc scratchArc = new Arc<>(); - final IntsRefBuilder scratchInts = new IntsRefBuilder(); - final BytesRefFSTEnum fstEnum = new BytesRefFSTEnum<>(fst); - final ByteArrayDataInput input = new ByteArrayDataInput(); - return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() { - final BytesRefBuilder term = new BytesRefBuilder(); - BytesRef ref; - long currentOrd; - - @Override - public long nextOrd() { - if (input.eof()) { - return NO_MORE_ORDS; - } else { - currentOrd += input.readVLong(); - return currentOrd; - } - } - - @Override - public void setDocument(int docID) { - ref = docToOrds.get(docID); - input.reset(ref.bytes, ref.offset, ref.length); - currentOrd = 0; - } - - @Override - public BytesRef lookupOrd(long ord) { - try { - in.setPosition(0); - fst.getFirstArc(firstArc); - IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts); - return Util.toBytesRef(output, term); - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - @Override - public long lookupTerm(BytesRef key) { - try { - InputOutput o = fstEnum.seekCeil(key); - if (o == null) { - return -getValueCount()-1; - } else if (o.input.equals(key)) { - return o.output.intValue(); - } else { - return -o.output-1; - } - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - @Override - public long getValueCount() { - return entry.numOrds; - } - - @Override - public TermsEnum termsEnum() { - return new FSTTermsEnum(fst); - } - }, maxDoc); - } - - private Bits getMissingBits(FieldInfo field, final long offset, final long length) throws IOException { - if (offset == -1) { - return new Bits.MatchAllBits(maxDoc); - } else { - FixedBitSet instance; - synchronized(this) { - instance = docsWithFieldInstances.get(field.name); - if (instance == null) { - IndexInput data = this.data.clone(); - data.seek(offset); - assert length % 8 == 0; - long bits[] = new long[(int) length >> 3]; - for (int i = 0; i < bits.length; i++) { - bits[i] = data.readLong(); - } - instance = new FixedBitSet(bits, maxDoc); - if (!merging) { - docsWithFieldInstances.put(field.name, instance); - ramBytesUsed.addAndGet(instance.ramBytesUsed()); - } - } - } - return instance; - } - } - - @Override - public void close() throws IOException { - data.close(); - } - - static class NumericEntry { - long offset; - long count; - long missingOffset; - long missingBytes; - byte format; - int packedIntsVersion; - } - - static class BinaryEntry { - long offset; - long missingOffset; - long missingBytes; - long numBytes; - int minLength; - int maxLength; - int packedIntsVersion; - int blockSize; - } - - static class FSTEntry { - long offset; - long numOrds; - } - - static class SortedSetEntry { - boolean singleton; - } - - static class SortedNumericEntry { - boolean singleton; - long addressOffset; - int packedIntsVersion; - int blockSize; - long valueCount; - } - - static class BytesAndAddresses implements Accountable { - PagedBytes.Reader reader; - MonotonicBlockPackedReader addresses; - - @Override - public long ramBytesUsed() { - long bytesUsed = reader.ramBytesUsed(); - if (addresses != null) { - bytesUsed += addresses.ramBytesUsed(); - } - return bytesUsed; - } - - @Override - public Collection getChildResources() { - List resources = new ArrayList<>(); - if (addresses != null) { - resources.add(Accountables.namedAccountable("addresses", addresses)); - } - resources.add(Accountables.namedAccountable("term bytes", reader)); - return Collections.unmodifiableList(resources); - } - } - - // exposes FSTEnum directly as a TermsEnum: avoids binary-search next() - static class FSTTermsEnum extends TermsEnum { - final BytesRefFSTEnum in; - - // this is all for the complicated seek(ord)... - // maybe we should add a FSTEnum that supports this operation? - final FST fst; - final FST.BytesReader bytesReader; - final Arc firstArc = new Arc<>(); - final Arc scratchArc = new Arc<>(); - final IntsRefBuilder scratchInts = new IntsRefBuilder(); - final BytesRefBuilder scratchBytes = new BytesRefBuilder(); - - FSTTermsEnum(FST fst) { - this.fst = fst; - in = new BytesRefFSTEnum<>(fst); - bytesReader = fst.getBytesReader(); - } - - @Override - public BytesRef next() throws IOException { - InputOutput io = in.next(); - if (io == null) { - return null; - } else { - return io.input; - } - } - - @Override - public SeekStatus seekCeil(BytesRef text) throws IOException { - if (in.seekCeil(text) == null) { - return SeekStatus.END; - } else if (term().equals(text)) { - // TODO: add SeekStatus to FSTEnum like in https://issues.apache.org/jira/browse/LUCENE-3729 - // to remove this comparision? - return SeekStatus.FOUND; - } else { - return SeekStatus.NOT_FOUND; - } - } - - @Override - public boolean seekExact(BytesRef text) throws IOException { - if (in.seekExact(text) == null) { - return false; - } else { - return true; - } - } - - @Override - public void seekExact(long ord) throws IOException { - // TODO: would be better to make this simpler and faster. - // but we dont want to introduce a bug that corrupts our enum state! - bytesReader.setPosition(0); - fst.getFirstArc(firstArc); - IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts); - // TODO: we could do this lazily, better to try to push into FSTEnum though? - in.seekExact(Util.toBytesRef(output, new BytesRefBuilder())); - } - - @Override - public BytesRef term() throws IOException { - return in.current().input; - } - - @Override - public long ord() throws IOException { - return in.current().output; - } - - @Override - public int docFreq() throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public long totalTermFreq() throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public ImpactsEnum impacts(int flags) throws IOException { - throw new UnsupportedOperationException(); - } - } -} diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java deleted file mode 100644 index 2433de7a271..00000000000 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java +++ /dev/null @@ -1,1037 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.FieldsProducer; -import org.apache.lucene.codecs.NormsProducer; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.Fields; -import org.apache.lucene.index.ImpactsEnum; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.SlowImpactsEnum; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.store.ByteArrayDataInput; -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.util.Accountable; -import org.apache.lucene.util.Accountables; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.IntsRefBuilder; -import org.apache.lucene.util.fst.Builder; -import org.apache.lucene.util.fst.ByteSequenceOutputs; -import org.apache.lucene.util.fst.BytesRefFSTEnum; -import org.apache.lucene.util.fst.FST; -import org.apache.lucene.util.fst.Util; -import org.apache.lucene.util.packed.PackedInts; - -// TODO: would be nice to somehow allow this to act like -// InstantiatedIndex, by never writing to disk; ie you write -// to this Codec in RAM only and then when you open a reader -// it pulls the FST directly from what you wrote w/o going -// to disk. - -/** Stores terms and postings (docs, positions, payloads) in - * RAM, using an FST. - * - *

Note that this codec implements advance as a linear - * scan! This means if you store large fields in here, - * queries that rely on advance will (AND BooleanQuery, - * PhraseQuery) will be relatively slow! - * - * @lucene.experimental */ - -// TODO: Maybe name this 'Cached' or something to reflect -// the reality that it is actually written to disk, but -// loads itself in ram? -public final class MemoryPostingsFormat extends PostingsFormat { - - public MemoryPostingsFormat() { - this(false, PackedInts.DEFAULT); - } - - /** - * Create MemoryPostingsFormat, specifying advanced FST options. - * @param doPackFST true if a packed FST should be built. - * NOTE: packed FSTs are limited to ~2.1 GB of postings. - * @param acceptableOverheadRatio allowable overhead for packed ints - * during FST construction. - */ - public MemoryPostingsFormat(boolean doPackFST, float acceptableOverheadRatio) { - super("Memory"); - } - - @Override - public String toString() { - return "PostingsFormat(name=" + getName() + ")"; - } - - private final static class TermsWriter { - private final IndexOutput out; - private final FieldInfo field; - private final Builder builder; - private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton(); - private int termCount; - - public TermsWriter(IndexOutput out, FieldInfo field) { - this.out = out; - this.field = field; - builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15); - } - - private class PostingsWriter { - private int lastDocID; - private int lastPos; - private int lastPayloadLen; - - // NOTE: not private so we don't pay access check at runtime: - int docCount; - RAMOutputStream buffer = new RAMOutputStream(); - - int lastOffsetLength; - int lastOffset; - - public void startDoc(int docID, int termDocFreq) throws IOException { - //System.out.println(" startDoc docID=" + docID + " freq=" + termDocFreq); - final int delta = docID - lastDocID; - assert docID == 0 || delta > 0; - lastDocID = docID; - docCount++; - - if (field.getIndexOptions() == IndexOptions.DOCS) { - buffer.writeVInt(delta); - } else if (termDocFreq == 1) { - buffer.writeVInt((delta<<1) | 1); - } else { - buffer.writeVInt(delta<<1); - assert termDocFreq > 0; - buffer.writeVInt(termDocFreq); - } - - lastPos = 0; - lastOffset = 0; - } - - public void addPosition(int pos, BytesRef payload, int startOffset, int endOffset) throws IOException { - assert payload == null || field.hasPayloads(); - - //System.out.println(" addPos pos=" + pos + " payload=" + payload); - - final int delta = pos - lastPos; - assert delta >= 0; - lastPos = pos; - - int payloadLen = 0; - - if (field.hasPayloads()) { - payloadLen = payload == null ? 0 : payload.length; - if (payloadLen != lastPayloadLen) { - lastPayloadLen = payloadLen; - buffer.writeVInt((delta<<1)|1); - buffer.writeVInt(payloadLen); - } else { - buffer.writeVInt(delta<<1); - } - } else { - buffer.writeVInt(delta); - } - - if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) { - // don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms, - // and the numbers aren't that much smaller anyways. - int offsetDelta = startOffset - lastOffset; - int offsetLength = endOffset - startOffset; - if (offsetLength != lastOffsetLength) { - buffer.writeVInt(offsetDelta << 1 | 1); - buffer.writeVInt(offsetLength); - } else { - buffer.writeVInt(offsetDelta << 1); - } - lastOffset = startOffset; - lastOffsetLength = offsetLength; - } - - if (payloadLen > 0) { - buffer.writeBytes(payload.bytes, payload.offset, payloadLen); - } - } - - public PostingsWriter reset() { - assert buffer.getFilePointer() == 0; - lastDocID = 0; - docCount = 0; - lastPayloadLen = 0; - // force first offset to write its length - lastOffsetLength = -1; - return this; - } - } - - final PostingsWriter postingsWriter = new PostingsWriter(); - - private final RAMOutputStream buffer2 = new RAMOutputStream(); - private final BytesRef spare = new BytesRef(); - private byte[] finalBuffer = new byte[128]; - - private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder(); - - private void finishTerm(BytesRef text, TermStats stats) throws IOException { - - if (stats.docFreq == 0) { - return; - } - assert postingsWriter.docCount == stats.docFreq; - - assert buffer2.getFilePointer() == 0; - - buffer2.writeVInt(stats.docFreq); - if (field.getIndexOptions() != IndexOptions.DOCS) { - buffer2.writeVLong(stats.totalTermFreq-stats.docFreq); - } - int pos = (int) buffer2.getFilePointer(); - buffer2.writeTo(finalBuffer, 0); - buffer2.reset(); - - final int totalBytes = pos + (int) postingsWriter.buffer.getFilePointer(); - if (totalBytes > finalBuffer.length) { - finalBuffer = ArrayUtil.grow(finalBuffer, totalBytes); - } - postingsWriter.buffer.writeTo(finalBuffer, pos); - postingsWriter.buffer.reset(); - - spare.bytes = finalBuffer; - spare.length = totalBytes; - - //System.out.println(" finishTerm term=" + text.utf8ToString() + " " + totalBytes + " bytes totalTF=" + stats.totalTermFreq); - //for(int i=0;i 0) { - out.writeVInt(termCount); - out.writeVInt(field.number); - if (field.getIndexOptions() != IndexOptions.DOCS) { - out.writeVLong(sumTotalTermFreq); - } - out.writeVLong(sumDocFreq); - out.writeVInt(docCount); - FST fst = builder.finish(); - fst.save(out); - //System.out.println("finish field=" + field.name + " fp=" + out.getFilePointer()); - } - } - } - - private static String EXTENSION = "ram"; - private static final String CODEC_NAME = "MemoryPostings"; - private static final int VERSION_START = 1; - private static final int VERSION_CURRENT = VERSION_START; - - private static class MemoryFieldsConsumer extends FieldsConsumer { - private final SegmentWriteState state; - private final IndexOutput out; - - private MemoryFieldsConsumer(SegmentWriteState state) throws IOException { - final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION); - out = state.directory.createOutput(fileName, state.context); - boolean success = false; - try { - CodecUtil.writeIndexHeader(out, CODEC_NAME, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - success = true; - } finally { - if (!success) { - IOUtils.closeWhileHandlingException(out); - } - } - this.state = state; - } - - @Override - public void write(Fields fields, NormsProducer norms) throws IOException { - for(String field : fields) { - - Terms terms = fields.terms(field); - if (terms == null) { - continue; - } - - TermsEnum termsEnum = terms.iterator(); - - FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field); - TermsWriter termsWriter = new TermsWriter(out, fieldInfo); - - FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.maxDoc()); - long sumTotalTermFreq = 0; - long sumDocFreq = 0; - PostingsEnum postingsEnum = null; - PostingsEnum posEnum = null; - int enumFlags; - - IndexOptions indexOptions = fieldInfo.getIndexOptions(); - boolean writeFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - boolean writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - boolean writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - boolean writePayloads = fieldInfo.hasPayloads(); - - if (writeFreqs == false) { - enumFlags = 0; - } else if (writePositions == false) { - enumFlags = PostingsEnum.FREQS; - } else if (writeOffsets == false) { - if (writePayloads) { - enumFlags = PostingsEnum.PAYLOADS; - } - else { - enumFlags = PostingsEnum.POSITIONS; - } - } else { - if (writePayloads) { - enumFlags = PostingsEnum.PAYLOADS | PostingsEnum.OFFSETS; - } else { - enumFlags = PostingsEnum.OFFSETS; - } - } - - while (true) { - BytesRef term = termsEnum.next(); - if (term == null) { - break; - } - termsWriter.postingsWriter.reset(); - - if (writePositions) { - posEnum = termsEnum.postings(posEnum, enumFlags); - postingsEnum = posEnum; - } else { - postingsEnum = termsEnum.postings(postingsEnum, enumFlags); - posEnum = null; - } - - int docFreq = 0; - long totalTermFreq = 0; - while (true) { - int docID = postingsEnum.nextDoc(); - if (docID == PostingsEnum.NO_MORE_DOCS) { - break; - } - docsSeen.set(docID); - docFreq++; - - int freq; - if (writeFreqs) { - freq = postingsEnum.freq(); - totalTermFreq += freq; - } else { - freq = -1; - } - - termsWriter.postingsWriter.startDoc(docID, freq); - if (writePositions) { - for (int i=0;i 0; - if (buffer.length < bufferIn.length) { - buffer = ArrayUtil.grow(buffer, bufferIn.length); - } - in.reset(buffer, 0, bufferIn.length); - System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length); - docID = -1; - accum = 0; - docUpto = 0; - freq = 1; - payloadLen = 0; - this.numDocs = numDocs; - return this; - } - - @Override - public int nextDoc() { - //System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this); - if (docUpto == numDocs) { - // System.out.println(" END"); - return docID = NO_MORE_DOCS; - } - docUpto++; - if (indexOptions == IndexOptions.DOCS) { - accum += in.readVInt(); - } else { - final int code = in.readVInt(); - accum += code >>> 1; - //System.out.println(" docID=" + accum + " code=" + code); - if ((code & 1) != 0) { - freq = 1; - } else { - freq = in.readVInt(); - assert freq > 0; - } - - if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { - // Skip positions/payloads - for(int posUpto=0;posUpto 0; - - // System.out.println("D&P reset bytes this=" + this); - // for(int i=bufferIn.offset;i 0) { - nextPosition(); - } - while(true) { - //System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this); - if (docUpto == numDocs) { - //System.out.println(" END"); - return docID = NO_MORE_DOCS; - } - docUpto++; - - final int code = in.readVInt(); - accum += code >>> 1; - if ((code & 1) != 0) { - freq = 1; - } else { - freq = in.readVInt(); - assert freq > 0; - } - - pos = 0; - startOffset = storeOffsets ? 0 : -1; - posPending = freq; - //System.out.println(" return docID=" + accum + " freq=" + freq); - return (docID = accum); - } - } - - @Override - public int nextPosition() { - //System.out.println(" nextPos storePayloads=" + storePayloads + " this=" + this); - assert posPending > 0; - posPending--; - if (!storePayloads) { - pos += in.readVInt(); - } else { - final int code = in.readVInt(); - pos += code >>> 1; - if ((code & 1) != 0) { - payloadLength = in.readVInt(); - //System.out.println(" new payloadLen=" + payloadLength); - //} else { - //System.out.println(" same payloadLen=" + payloadLength); - } - } - - if (storeOffsets) { - int offsetCode = in.readVInt(); - if ((offsetCode & 1) != 0) { - // new offset length - offsetLength = in.readVInt(); - } - startOffset += offsetCode >>> 1; - } - - if (storePayloads) { - payload.offset = in.getPosition(); - in.skipBytes(payloadLength); - payload.length = payloadLength; - } - - //System.out.println(" pos=" + pos + " payload=" + payload + " fp=" + in.getPosition()); - return pos; - } - - @Override - public int startOffset() { - return startOffset; - } - - @Override - public int endOffset() { - return startOffset + offsetLength; - } - - @Override - public BytesRef getPayload() { - return payload.length > 0 ? payload : null; - } - - @Override - public int docID() { - return docID; - } - - @Override - public int advance(int target) throws IOException { - // TODO: we could make more efficient version, but, it - // should be rare that this will matter in practice - // since usually apps will not store "big" fields in - // this codec! - return slowAdvance(target); - } - - @Override - public int freq() { - return freq; - } - - @Override - public long cost() { - return numDocs; - } - } - - private final static class FSTTermsEnum extends TermsEnum { - private final FieldInfo field; - private final BytesRefFSTEnum fstEnum; - private final ByteArrayDataInput buffer = new ByteArrayDataInput(); - private boolean didDecode; - - private int docFreq; - private long totalTermFreq; - private BytesRefFSTEnum.InputOutput current; - private BytesRef postingsSpare = new BytesRef(); - - public FSTTermsEnum(FieldInfo field, FST fst) { - this.field = field; - fstEnum = new BytesRefFSTEnum<>(fst); - } - - private void decodeMetaData() { - if (!didDecode) { - buffer.reset(current.output.bytes, current.output.offset, current.output.length); - docFreq = buffer.readVInt(); - if (field.getIndexOptions() == IndexOptions.DOCS) { - totalTermFreq = docFreq; - } else { - totalTermFreq = docFreq + buffer.readVLong(); - } - postingsSpare.bytes = current.output.bytes; - postingsSpare.offset = buffer.getPosition(); - postingsSpare.length = current.output.length - (buffer.getPosition() - current.output.offset); - //System.out.println(" df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length); - didDecode = true; - } - } - - @Override - public boolean seekExact(BytesRef text) throws IOException { - //System.out.println("te.seekExact text=" + field.name + ":" + text.utf8ToString() + " this=" + this); - current = fstEnum.seekExact(text); - didDecode = false; - return current != null; - } - - @Override - public SeekStatus seekCeil(BytesRef text) throws IOException { - //System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this); - current = fstEnum.seekCeil(text); - if (current == null) { - return SeekStatus.END; - } else { - - // System.out.println(" got term=" + current.input.utf8ToString()); - // for(int i=0;i= 0; - if (hasPositions && PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) { - boolean hasOffsets = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - decodeMetaData(); - FSTPostingsEnum docsAndPositionsEnum; - if (reuse == null || !(reuse instanceof FSTPostingsEnum)) { - docsAndPositionsEnum = new FSTPostingsEnum(field.hasPayloads(), hasOffsets); - } else { - docsAndPositionsEnum = (FSTPostingsEnum) reuse; - if (!docsAndPositionsEnum.canReuse(field.hasPayloads(), hasOffsets)) { - docsAndPositionsEnum = new FSTPostingsEnum(field.hasPayloads(), hasOffsets); - } - } - //System.out.println("D&P reset this=" + this); - return docsAndPositionsEnum.reset(postingsSpare, docFreq); - } - - decodeMetaData(); - FSTDocsEnum docsEnum; - - if (reuse == null || !(reuse instanceof FSTDocsEnum)) { - docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads()); - } else { - docsEnum = (FSTDocsEnum) reuse; - if (!docsEnum.canReuse(field.getIndexOptions(), field.hasPayloads())) { - docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads()); - } - } - return docsEnum.reset(this.postingsSpare, docFreq); - } - - @Override - public ImpactsEnum impacts(int flags) throws IOException { - return new SlowImpactsEnum(postings(null, flags)); - } - - @Override - public BytesRef term() { - return current.input; - } - - @Override - public BytesRef next() throws IOException { - //System.out.println("te.next"); - current = fstEnum.next(); - if (current == null) { - //System.out.println(" END"); - return null; - } - didDecode = false; - //System.out.println(" term=" + field.name + ":" + current.input.utf8ToString()); - return current.input; - } - - @Override - public int docFreq() { - decodeMetaData(); - return docFreq; - } - - @Override - public long totalTermFreq() { - decodeMetaData(); - return totalTermFreq; - } - - @Override - public void seekExact(long ord) { - // NOTE: we could add this... - throw new UnsupportedOperationException(); - } - - @Override - public long ord() { - // NOTE: we could add this... - throw new UnsupportedOperationException(); - } - } - - private final static class TermsReader extends Terms implements Accountable { - - private final long sumTotalTermFreq; - private final long sumDocFreq; - private final int docCount; - private final int termCount; - private FST fst; - private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton(); - private final FieldInfo field; - - public TermsReader(FieldInfos fieldInfos, IndexInput in, int termCount) throws IOException { - this.termCount = termCount; - final int fieldNumber = in.readVInt(); - field = fieldInfos.fieldInfo(fieldNumber); - if (field == null) { - throw new CorruptIndexException("invalid field number: " + fieldNumber, in); - } else { - sumTotalTermFreq = in.readVLong(); - } - // if frequencies are omitted, sumDocFreq = sumTotalTermFreq and we only write one value. - if (field.getIndexOptions() == IndexOptions.DOCS) { - sumDocFreq = sumTotalTermFreq; - } else { - sumDocFreq = in.readVLong(); - } - docCount = in.readVInt(); - - fst = new FST<>(in, outputs); - } - - @Override - public long getSumTotalTermFreq() { - return sumTotalTermFreq; - } - - @Override - public long getSumDocFreq() { - return sumDocFreq; - } - - @Override - public int getDocCount() { - return docCount; - } - - @Override - public long size() { - return termCount; - } - - @Override - public TermsEnum iterator() { - return new FSTTermsEnum(field, fst); - } - - @Override - public boolean hasFreqs() { - return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - } - - @Override - public boolean hasOffsets() { - return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - } - - @Override - public boolean hasPositions() { - return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - } - - @Override - public boolean hasPayloads() { - return field.hasPayloads(); - } - - @Override - public long ramBytesUsed() { - return ((fst!=null) ? fst.ramBytesUsed() : 0); - } - - @Override - public Collection getChildResources() { - if (fst == null) { - return Collections.emptyList(); - } else { - return Collections.singletonList(Accountables.namedAccountable("terms", fst)); - } - } - - @Override - public String toString() { - return "MemoryTerms(terms=" + termCount + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")"; - } - } - - @Override - public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION); - - final SortedMap fields = new TreeMap<>(); - - try (ChecksumIndexInput in = state.directory.openChecksumInput(fileName, IOContext.READONCE)) { - Throwable priorE = null; - try { - CodecUtil.checkIndexHeader(in, CODEC_NAME, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - while(true) { - final int termCount = in.readVInt(); - if (termCount == 0) { - break; - } - final TermsReader termsReader = new TermsReader(state.fieldInfos, in, termCount); - // System.out.println("load field=" + termsReader.field.name); - fields.put(termsReader.field.name, termsReader); - } - } catch (Throwable exception) { - priorE = exception; - } finally { - CodecUtil.checkFooter(in, priorE); - } - } - - return new FieldsProducer() { - @Override - public Iterator iterator() { - return Collections.unmodifiableSet(fields.keySet()).iterator(); - } - - @Override - public Terms terms(String field) { - return fields.get(field); - } - - @Override - public int size() { - return fields.size(); - } - - @Override - public void close() { - // Drop ref to FST: - for(TermsReader termsReader : fields.values()) { - termsReader.fst = null; - } - } - - @Override - public long ramBytesUsed() { - long sizeInBytes = 0; - for(Map.Entry entry: fields.entrySet()) { - sizeInBytes += (entry.getKey().length() * Character.BYTES); - sizeInBytes += entry.getValue().ramBytesUsed(); - } - return sizeInBytes; - } - - @Override - public Collection getChildResources() { - return Accountables.namedAccountables("field", fields); - } - - @Override - public String toString() { - return "MemoryPostings(fields=" + fields.size() + ")"; - } - - @Override - public void checkIntegrity() throws IOException {} - }; - } -} diff --git a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat index daef7c58536..33c8fadb8cc 100644 --- a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat +++ b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.memory.MemoryDocValuesFormat org.apache.lucene.codecs.memory.DirectDocValuesFormat diff --git a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 710c6050c67..1c048f8f63f 100644 --- a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -18,4 +18,4 @@ org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat org.apache.lucene.codecs.memory.DirectPostingsFormat org.apache.lucene.codecs.memory.FSTOrdPostingsFormat org.apache.lucene.codecs.memory.FSTPostingsFormat -org.apache.lucene.codecs.memory.MemoryPostingsFormat + diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java deleted file mode 100644 index 1c83eec0a81..00000000000 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - - -import org.apache.lucene.codecs.Codec; -import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase; -import org.apache.lucene.util.TestUtil; - -/** - * Tests MemoryDocValuesFormat - */ -public class TestMemoryDocValuesFormat extends BaseCompressingDocValuesFormatTestCase { - private final Codec codec = TestUtil.alwaysDocValuesFormat(new MemoryDocValuesFormat()); - - @Override - protected Codec getCodec() { - return codec; - } - - @Override - protected boolean codecAcceptsHugeBinaryValues(String field) { - return false; - } -} diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java deleted file mode 100644 index 3a56bf56744..00000000000 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.codecs.memory; - - -import org.apache.lucene.codecs.Codec; -import org.apache.lucene.index.BasePostingsFormatTestCase; -import org.apache.lucene.util.TestUtil; - -/** - * Tests MemoryPostingsFormat - */ -public class TestMemoryPostingsFormat extends BasePostingsFormatTestCase { - // TODO: randomize doPack - private final Codec codec = TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat()); - - @Override - protected Codec getCodec() { - return codec; - } -} diff --git a/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java b/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java index 2795679d0f3..47b69218aef 100644 --- a/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java +++ b/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java @@ -42,13 +42,10 @@ public class TestExternalCodecs extends LuceneTestCase { private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly"); private final PostingsFormat defaultFormat = TestUtil.getDefaultPostingsFormat(); - private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory"); @Override public PostingsFormat getPostingsFormatForField(String field) { - if (field.equals("field2") || field.equals("id")) { - return memoryFormat; - } else if (field.equals("field1")) { + if (field.equals("field2") || field.equals("field1") || field.equals("id")) { return defaultFormat; } else { return ramFormat; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java index 578b2189ba7..df94a90ea16 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java @@ -89,7 +89,7 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase { // we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1 IndexWriterConfig iwc = newIndexWriterConfig(analyzer); final DocValuesFormat fast = TestUtil.getDefaultDocValuesFormat(); - final DocValuesFormat slow = DocValuesFormat.forName("Memory"); + final DocValuesFormat slow = DocValuesFormat.forName("Direct"); iwc.setCodec(new AssertingCodec() { @Override public DocValuesFormat getDocValuesFormatForField(String field) { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java index 84544bc86ee..f72ffcc055d 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java @@ -33,7 +33,6 @@ import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.asserting.AssertingCodec; import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval; import org.apache.lucene.codecs.memory.DirectPostingsFormat; -import org.apache.lucene.codecs.memory.MemoryPostingsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FieldType; @@ -165,7 +164,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase { //((LogMergePolicy) iwconf.getMergePolicy()).setMergeFactor(10); iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH); - iwconf.setCodec(new MockCodec2()); // uses standard for field content + iwconf.setCodec(new MockCodec()); // uses standard for field content writer = newWriter(dir, iwconf); // swap in new codec for currently written segments if (VERBOSE) { @@ -218,24 +217,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase { public static class MockCodec extends AssertingCodec { final PostingsFormat luceneDefault = TestUtil.getDefaultPostingsFormat(); final PostingsFormat direct = new DirectPostingsFormat(); - final PostingsFormat memory = new MemoryPostingsFormat(); - - @Override - public PostingsFormat getPostingsFormatForField(String field) { - if (field.equals("id")) { - return direct; - } else if (field.equals("content")) { - return memory; - } else { - return luceneDefault; - } - } - } - public static class MockCodec2 extends AssertingCodec { - final PostingsFormat luceneDefault = TestUtil.getDefaultPostingsFormat(); - final PostingsFormat direct = new DirectPostingsFormat(); - @Override public PostingsFormat getPostingsFormatForField(String field) { if (field.equals("id")) { @@ -287,9 +269,9 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase { @Override public PostingsFormat getPostingsFormatForField(String field) { if ("id".equals(field)) { - return new MemoryPostingsFormat(); + return new DirectPostingsFormat(); } else if ("date".equals(field)) { - return new MemoryPostingsFormat(); + return new DirectPostingsFormat(); } else { return super.getPostingsFormatForField(field); } diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java b/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java index 466248027b4..e0d5bd3464d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java @@ -33,7 +33,7 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TimeUnits; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; -@SuppressCodecs({"SimpleText", "Memory", "Direct"}) +@SuppressCodecs({"SimpleText", "Direct"}) @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit // The six hour time was achieved on a Linux 3.13 system with these specs: // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA. diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java b/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java index dbbe2717927..91406c6fd67 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java @@ -35,7 +35,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TimeUnits; -@SuppressCodecs({"SimpleText", "Memory", "Direct"}) +@SuppressCodecs({"SimpleText", "Direct"}) @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit @Monster("Takes ~30min") @SuppressSysoutChecks(bugUrl = "Stuff gets printed") diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java b/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java index 046ac6fe71b..8587f829fcd 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java @@ -31,7 +31,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; -@SuppressCodecs({"SimpleText", "Memory", "Direct"}) +@SuppressCodecs({"SimpleText", "Direct"}) @TimeoutSuite(millis = 8 * TimeUnits.HOUR) // The two hour time was achieved on a Linux 3.13 system with these specs: // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA. diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java index 2e327732ec6..a12f31c6462 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java @@ -35,7 +35,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; // // or: python -u /l/util/src/python/repeatLuceneTest.py -heap 6g -once -nolog -tmpDir /b/tmp -logDir /l/logs Test2BPoints.test2D -verbose -@SuppressCodecs({ "SimpleText", "Memory", "Direct", "Compressing" }) +@SuppressCodecs({ "SimpleText", "Direct", "Compressing" }) @TimeoutSuite(millis = 365 * 24 * TimeUnits.HOUR) // hopefully ~1 year is long enough ;) @Monster("takes at least 4 hours and consumes many GB of temp disk space") public class Test2BPoints extends LuceneTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java index d65c95c4e49..d6687bfb973 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java @@ -36,7 +36,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; * Test indexes ~82M docs with 52 positions each, so you get > Integer.MAX_VALUE positions * @lucene.experimental */ -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) @Monster("uses lots of space and takes a few minutes") public class Test2BPositions extends LuceneTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java index 15a931fceec..07449710ff7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java @@ -35,7 +35,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; * Test indexes ~82M docs with 26 terms each, so you get > Integer.MAX_VALUE terms/docs pairs * @lucene.experimental */ -@SuppressCodecs({ "SimpleText", "Memory", "Direct", "Compressing" }) +@SuppressCodecs({ "SimpleText", "Direct", "Compressing" }) @TimeoutSuite(millis = 4 * TimeUnits.HOUR) public class Test2BPostings extends LuceneTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java index ae935d68a2e..7e8731a6c5d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java @@ -42,7 +42,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; * so you get > Integer.MAX_VALUE postings data for the term * @lucene.experimental */ -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) @Monster("takes ~20GB-30GB of space and 10 minutes") public class Test2BPostingsBytes extends LuceneTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java index 988e3f7324f..a41dda9f7f5 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java @@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; -@SuppressCodecs({"SimpleText", "Memory", "Direct"}) +@SuppressCodecs({"SimpleText", "Direct"}) @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit // The six hour time was achieved on a Linux 3.13 system with these specs: // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA. diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java index 744873899f0..11837283293 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java @@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; -@SuppressCodecs({"SimpleText", "Memory", "Direct"}) +@SuppressCodecs({"SimpleText", "Direct"}) @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit // The six hour time was achieved on a Linux 3.13 system with these specs: // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA. diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java index 2e962b68db7..4b6a456a4bb 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java @@ -55,7 +55,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; // // ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene62 -Dtestcase=Test2BTerms // -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) @Monster("very slow, use 5g minimum heap") @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit @SuppressSysoutChecks(bugUrl = "Stuff gets printed") diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java index 48a28e2e791..0df7ac8c74a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java @@ -28,7 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.asserting.AssertingCodec; -import org.apache.lucene.codecs.memory.MemoryPostingsFormat; +import org.apache.lucene.codecs.memory.DirectPostingsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FieldType; @@ -1088,14 +1088,11 @@ public class TestAddIndexes extends LuceneTestCase { private static final class CustomPerFieldCodec extends AssertingCodec { private final PostingsFormat directFormat = PostingsFormat.forName("Direct"); private final PostingsFormat defaultFormat = TestUtil.getDefaultPostingsFormat(); - private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory"); @Override public PostingsFormat getPostingsFormatForField(String field) { if (field.equals("id")) { return directFormat; - } else if (field.equals("content")) { - return memoryFormat; } else { return defaultFormat; } @@ -1164,7 +1161,7 @@ public class TestAddIndexes extends LuceneTestCase { { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); - conf.setCodec(TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat())); + conf.setCodec(TestUtil.alwaysPostingsFormat(new DirectPostingsFormat())); IndexWriter w = new IndexWriter(dir, conf); expectThrows(IllegalArgumentException.class, () -> { w.addIndexes(toAdd); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java b/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java index 6214e4cde1d..28165094766 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java @@ -40,7 +40,7 @@ import org.apache.lucene.util.TestUtil; * totalTermFreq of its integer value, and checks that the totalTermFreq is correct. */ // TODO: somehow factor this with BagOfPostings? it's almost the same -@SuppressCodecs({"Direct", "Memory"}) // at night this makes like 200k/300k docs and will make Direct's heart beat! +@SuppressCodecs({"Direct"}) // at night this makes like 200k/300k docs and will make Direct's heart beat! public class TestBagOfPositions extends LuceneTestCase { public void test() throws Exception { List postingsList = new ArrayList<>(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBagOfPostings.java b/lucene/core/src/test/org/apache/lucene/index/TestBagOfPostings.java index 8cd25519567..18e011cd11d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestBagOfPostings.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestBagOfPostings.java @@ -38,7 +38,7 @@ import org.apache.lucene.util.TestUtil; * Simple test that adds numeric terms, where each term has the * docFreq of its integer value, and checks that the docFreq is correct. */ -@SuppressCodecs({"Direct", "Memory"}) // at night this makes like 200k/300k docs and will make Direct's heart beat! +@SuppressCodecs({"Direct"}) // at night this makes like 200k/300k docs and will make Direct's heart beat! public class TestBagOfPostings extends LuceneTestCase { public void test() throws Exception { List postingsList = new ArrayList<>(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecsAtNight.java b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecsAtNight.java index 41143b286e7..1839c225fc1 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecsAtNight.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecsAtNight.java @@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; * Just like TestDuelingCodecs, only with a lot more documents. */ @Nightly -@SuppressCodecs({"Memory", "Direct"}) // it can be too much for these codecs +@SuppressCodecs({"Direct"}) // it can be too much for these codecs public class TestDuelingCodecsAtNight extends TestDuelingCodecs { // use a big number of documents diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java index e94621a34c9..a7c2c9bbab9 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java @@ -177,7 +177,6 @@ public class TestIndexWriterCommit extends LuceneTestCase { final String idFormat = TestUtil.getPostingsFormat("id"); final String contentFormat = TestUtil.getPostingsFormat("content"); - assumeFalse("This test cannot run with Memory codec", idFormat.equals("Memory") || contentFormat.equals("Memory")); MockDirectoryWrapper dir = newMockDirectory(); Analyzer analyzer; if (random().nextBoolean()) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java index fe951f39d2d..660f88be18a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java @@ -994,7 +994,6 @@ public class TestIndexWriterDelete extends LuceneTestCase { // TODO: move this test to its own class and just @SuppressCodecs? // TODO: is it enough to just use newFSDirectory? final String fieldFormat = TestUtil.getPostingsFormat("field"); - assumeFalse("This test cannot run with Memory codec", fieldFormat.equals("Memory")); assumeFalse("This test cannot run with SimpleText codec", fieldFormat.equals("SimpleText")); assumeFalse("This test cannot run with Direct codec", fieldFormat.equals("Direct")); final Random r = random(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java index 4fd7550d5b0..237ad382f81 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java @@ -39,7 +39,7 @@ import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TimeUnits; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) @TimeoutSuite(millis = 8 * TimeUnits.HOUR) public class TestIndexWriterMaxDocs extends LuceneTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java index d225f43f2c7..3629721f484 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java @@ -162,7 +162,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase { final String idFormat = TestUtil.getPostingsFormat("id"); final String contentFormat = TestUtil.getPostingsFormat("content"); - assumeFalse("This test cannot run with Memory codec", idFormat.equals("Memory") || contentFormat.equals("Memory")); int START_COUNT = 57; int NUM_DIR = TEST_NIGHTLY ? 50 : 5; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java index ff79e5e4113..3673887f88f 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java @@ -130,7 +130,6 @@ public class TestLazyProxSkipping extends LuceneTestCase { public void testLazySkipping() throws IOException { final String fieldFormat = TestUtil.getPostingsFormat(this.field); - assumeFalse("This test cannot run with Memory postings format", fieldFormat.equals("Memory")); assumeFalse("This test cannot run with Direct postings format", fieldFormat.equals("Direct")); assumeFalse("This test cannot run with SimpleText postings format", fieldFormat.equals("SimpleText")); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java b/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java index bafea384c2d..20c7f90e45d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java @@ -35,7 +35,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestLongPostings extends LuceneTestCase { // Produces a realistic unicode random string that diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java b/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java index 1b55653db0e..d8c0b49ac3a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestNRTThreads.java @@ -30,7 +30,7 @@ import org.junit.Before; // - mix in forceMerge, addIndexes // - randomoly mix in non-congruent docs -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestNRTThreads extends ThreadedIndexingAndSearchingTestCase { private boolean useNonNrtReaders = true; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java index 805c7e58474..4100885a791 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java @@ -43,7 +43,7 @@ import org.apache.lucene.util.TestUtil; * Test that norms info is preserved during index life - including * separate norms, addDocument, addIndexes, forceMerge. */ -@SuppressCodecs({ "Memory", "Direct", "SimpleText" }) +@SuppressCodecs({ "Direct", "SimpleText" }) @Slow public class TestNorms extends LuceneTestCase { static final String BYTE_TEST_FIELD = "normsTestByte"; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java index 5a2c82fec4a..10e714ed753 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java @@ -21,7 +21,7 @@ import java.util.Random; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.memory.MemoryPostingsFormat; +import org.apache.lucene.codecs.memory.DirectPostingsFormat; import org.apache.lucene.document.*; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; @@ -42,9 +42,9 @@ public class TestRollingUpdates extends LuceneTestCase { final LineFileDocs docs = new LineFileDocs(random); - //provider.register(new MemoryCodec()); if (random().nextBoolean()) { - Codec.setDefault(TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat(random().nextBoolean(), random.nextFloat()))); + Codec.setDefault(TestUtil.alwaysPostingsFormat( + new DirectPostingsFormat())); } MockAnalyzer analyzer = new MockAnalyzer(random()); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java index 96c42f3dcc4..31fe18a022d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java @@ -46,7 +46,7 @@ import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.RegExp; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestTermsEnum extends LuceneTestCase { public void test() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java index 7a7b0ac6cb6..c1157fff55b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java @@ -47,7 +47,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.ThreadInterruptedException; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearchingTestCase { // Not guaranteed to reflect deletes: diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java b/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java index 0008dc71022..ec73368eb1c 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java @@ -29,7 +29,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestSearchWithThreads extends LuceneTestCase { int NUM_DOCS; static final int NUM_SEARCH_THREADS = 5; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java index d74d5ac1fc1..b70784c0f63 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java @@ -50,7 +50,7 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.NamedThreadFactory; import org.apache.lucene.util.TestUtil; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase { boolean warmCalled; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java b/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java index 768ee0e1713..ebc706273b3 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java @@ -41,7 +41,7 @@ import org.apache.lucene.util.TestUtil; // - test pulling docs in 2nd round trip... // - filter too -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) public class TestShardSearching extends ShardSearchingTestBase { private static class PreviousSearchState { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSelector.java b/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSelector.java index 82abf12f9ac..16daf2b2629 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSelector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSelector.java @@ -29,7 +29,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; /** Tests for SortedSetSortField selectors other than MIN, * these require optional codec support (random access to ordinals) */ -@SuppressCodecs({"Memory", "SimpleText"}) // TODO: fix MemoryDV to support random access to ordinals +@SuppressCodecs({"SimpleText"}) public class TestSortedSetSelector extends LuceneTestCase { public void testMax() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java index 6f704c94f5b..f3109e201f7 100644 --- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java +++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java @@ -81,7 +81,7 @@ import static org.apache.lucene.util.fst.FSTTester.getRandomString; import static org.apache.lucene.util.fst.FSTTester.simpleRandomString; import static org.apache.lucene.util.fst.FSTTester.toIntsRef; -@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) +@SuppressCodecs({ "SimpleText", "Direct" }) @Slow public class TestFSTs extends LuceneTestCase { diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java index 8f0c4ca7e15..b7932bd9738 100644 --- a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java +++ b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java @@ -28,7 +28,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; /** * Random testing for {@link InetAddressRange} */ -@SuppressCodecs({"Direct", "Memory"}) +@SuppressCodecs({"Direct"}) public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase { private static final String FIELD_NAME = "ipRangeField"; diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java index 20c3f613f23..6f95c5b4237 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java @@ -51,7 +51,7 @@ import org.apache.lucene.util.TestUtil; /** Child process with silly naive TCP socket server to handle * between-node commands, launched for each node by TestNRTReplication. */ -@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"}) +@SuppressCodecs({"MockRandom", "Direct", "SimpleText"}) @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure") @SuppressForbidden(reason = "We need Unsafe to actually crush :-)") public class SimpleServer extends LuceneTestCase { diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java index a2e7cf9d0df..05f02cce8c2 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java @@ -44,7 +44,7 @@ import org.apache.lucene.util.TestUtil; import com.carrotsearch.randomizedtesting.SeedUtils; // MockRandom's .sd file has no index header/footer: -@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"}) +@SuppressCodecs({"MockRandom", "Direct", "SimpleText"}) @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure") public class TestNRTReplication extends LuceneTestCase { diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java index 045bb38953a..d11b22bc58a 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -112,7 +112,7 @@ import com.carrotsearch.randomizedtesting.SeedUtils; */ // MockRandom's .sd file has no index header/footer: -@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"}) +@SuppressCodecs({"MockRandom", "Direct", "SimpleText"}) @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure") public class TestStressNRTReplication extends LuceneTestCase { diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java index a87148f5518..e4897c59f00 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java @@ -549,9 +549,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase { Directory dir = null; RandomIndexWriter iw = null; final String postingsFormat = TestUtil.getPostingsFormat("dummy"); - boolean codecOk = iterations * maxWordLength < 100000 || - !(postingsFormat.equals("Memory") || - postingsFormat.equals("SimpleText")); + boolean codecOk = iterations * maxWordLength < 100000 || !(postingsFormat.equals("SimpleText")); if (rarely(random) && codecOk) { dir = newFSDirectory(createTempDir("bttc")); iw = new RandomIndexWriter(new Random(seed), dir, a); diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java index 6b3993042fa..ea62178bdd1 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java @@ -627,7 +627,6 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase { @Nightly public void testRandomBig() throws Exception { assumeFalse("Direct codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Direct")); - assumeFalse("Memory codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Memory")); doTestRandom(200000); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java index df28105d33c..3588a8eb649 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java @@ -46,8 +46,6 @@ import org.apache.lucene.codecs.memory.DirectDocValuesFormat; import org.apache.lucene.codecs.memory.DirectPostingsFormat; import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat; import org.apache.lucene.codecs.memory.FSTPostingsFormat; -import org.apache.lucene.codecs.memory.MemoryDocValuesFormat; -import org.apache.lucene.codecs.memory.MemoryPostingsFormat; import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat; import org.apache.lucene.index.PointValues.IntersectVisitor; import org.apache.lucene.store.Directory; @@ -205,14 +203,11 @@ public class RandomCodec extends AssertingCodec { new LuceneVarGapFixedInterval(TestUtil.nextInt(random, 1, 1000)), new LuceneVarGapDocFreqInterval(TestUtil.nextInt(random, 1, 100), TestUtil.nextInt(random, 1, 1000)), TestUtil.getDefaultPostingsFormat(), - new AssertingPostingsFormat(), - new MemoryPostingsFormat(true, random.nextFloat()), - new MemoryPostingsFormat(false, random.nextFloat())); + new AssertingPostingsFormat()); addDocValues(avoidCodecs, TestUtil.getDefaultDocValuesFormat(), new DirectDocValuesFormat(), // maybe not a great idea... - new MemoryDocValuesFormat(), TestUtil.getDefaultDocValuesFormat(), new AssertingDocValuesFormat()); diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java index 29697889e5f..d8960a1e3bb 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java @@ -980,7 +980,7 @@ public final class TestUtil { // TODO: remove this, push this test to Lucene40/Lucene42 codec tests public static boolean fieldSupportsHugeBinaryDocValues(String field) { String dvFormat = getDocValuesFormat(field); - if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42") || dvFormat.equals("Memory")) { + if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42")) { return false; } return true; diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index a51a22e950b..8b443f77556 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -41,6 +41,10 @@ Upgrade Notes method signatures was required to fix it. Third party components that use this API need to be updated. (Eros Taborelli, Erick Erickson, ab) +* LUCENE-8267: Memory codecs have been removed from the codebase (MemoryPostings, + MemoryDocValues). If you used postingsFormat="Memory" or docValuesFormat="Memory" + switch to "Direct" instead. (Dawid Weiss) + ================== 7.4.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml index a90cf9cee03..e2597401a19 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml @@ -16,11 +16,10 @@ limitations under the License. --> - + - @@ -39,7 +38,6 @@ - @@ -48,7 +46,6 @@ - string_f diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java index bdef535fc84..146a0f04437 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java +++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java @@ -63,9 +63,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 { SchemaField schemaField = fields.get("string_disk_f"); PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat(); assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName()); - schemaField = fields.get("string_memory_f"); - assertEquals("Memory", - format.getDocValuesFormatForField(schemaField.getName()).getName()); + schemaField = fields.get("string_direct_f"); + assertEquals("Direct", format.getDocValuesFormatForField(schemaField.getName()).getName()); schemaField = fields.get("string_f"); assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName()); @@ -87,8 +86,8 @@ public class TestCodecSupport extends SolrTestCaseJ4 { assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("foo_disk").getName()); assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("bar_disk").getName()); - assertEquals("Memory", format.getDocValuesFormatForField("foo_memory").getName()); - assertEquals("Memory", format.getDocValuesFormatForField("bar_memory").getName()); + assertEquals("Direct", format.getDocValuesFormatForField("foo_direct").getName()); + assertEquals("Direct", format.getDocValuesFormatForField("bar_direct").getName()); } private void reloadCoreAndRecreateIndex() { diff --git a/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java b/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java index 29fd8dc54b7..f273ba59852 100644 --- a/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java +++ b/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java @@ -40,7 +40,7 @@ import org.junit.BeforeClass; * Tests the behavior of field(foo,min|max) on numerious types of multivalued 'foo' fields, * as well as the beahvior of sorting on foo asc|desc to implicitly choose the min|max. */ -@SuppressCodecs({"Memory", "SimpleText"}) // see TestSortedSetSelector +@SuppressCodecs({"SimpleText"}) // see TestSortedSetSelector public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 { /** Initializes core and does some sanity checking of schema */ diff --git a/solr/core/src/test/org/apache/solr/search/function/TestSortByMinMaxFunction.java b/solr/core/src/test/org/apache/solr/search/function/TestSortByMinMaxFunction.java index 5475b597f78..bf898f2f8c9 100644 --- a/solr/core/src/test/org/apache/solr/search/function/TestSortByMinMaxFunction.java +++ b/solr/core/src/test/org/apache/solr/search/function/TestSortByMinMaxFunction.java @@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; * * @see SortByFunctionTest **/ -@SuppressCodecs({"Memory", "SimpleText"}) // see TestSortedSetSelector +@SuppressCodecs({"SimpleText"}) // see TestSortedSetSelector public class TestSortByMinMaxFunction extends SortByFunctionTest { @Override diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java index af66b424212..a9678a8ef52 100644 --- a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java +++ b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java @@ -30,7 +30,6 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; import org.apache.solr.index.SlowCompositeReaderWrapper; -@LuceneTestCase.SuppressCodecs({"Memory"}) public class TestDocTermOrdsUninvertLimit extends LuceneTestCase { /* UnInvertedField had a reference block limitation of 2^24. This unit test triggered it. diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheVsDocValues.java b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheVsDocValues.java index 9bacab64976..0377dff5022 100644 --- a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheVsDocValues.java +++ b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheVsDocValues.java @@ -165,38 +165,11 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase { b.length = bytes.length; doc.add(new BinaryDocValuesField("field", b)); doc.add(new StringField("id", "" + docID, Field.Store.YES)); - try { - w.addDocument(doc); - } catch (IllegalArgumentException iae) { - if (iae.getMessage().indexOf("is too large") == -1) { - throw iae; - } else { - // OK: some codecs can't handle binary DV > 32K - assertFalse(codecAcceptsHugeBinaryValues("field")); - w.rollback(); - d.close(); - return; - } - } + w.addDocument(doc); } - DirectoryReader r; - try { - r = DirectoryReader.open(w); - } catch (IllegalArgumentException iae) { - if (iae.getMessage().indexOf("is too large") == -1) { - throw iae; - } else { - assertFalse(codecAcceptsHugeBinaryValues("field")); - - // OK: some codecs can't handle binary DV > 32K - w.rollback(); - d.close(); - return; - } - } - + DirectoryReader r = DirectoryReader.open(w); try (LeafReader ar = SlowCompositeReaderWrapper.wrap(r)) { TestUtil.checkReader(ar); @@ -210,86 +183,11 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase { assertEquals(expected.length, bytes.length); assertEquals(new BytesRef(expected), bytes); } - - assertTrue(codecAcceptsHugeBinaryValues("field")); } } } } - private static final int LARGE_BINARY_FIELD_LENGTH = (1 << 15) - 2; - - // TODO: get this out of here and into the deprecated codecs (4.0, 4.2) - public void testHugeBinaryValueLimit() throws Exception { - // We only test DVFormats that have a limit - assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field")); - Analyzer analyzer = new MockAnalyzer(random()); - // FSDirectory because SimpleText will consume gobbs of - // space when storing big binary values: - Directory d = newFSDirectory(createTempDir("hugeBinaryValues")); - boolean doFixed = random().nextBoolean(); - int numDocs; - int fixedLength = 0; - if (doFixed) { - // Sometimes make all values fixed length since some - // codecs have different code paths for this: - numDocs = TestUtil.nextInt(random(), 10, 20); - fixedLength = LARGE_BINARY_FIELD_LENGTH; - } else { - numDocs = TestUtil.nextInt(random(), 100, 200); - } - IndexWriter w = new IndexWriter(d, newIndexWriterConfig(analyzer)); - List docBytes = new ArrayList<>(); - long totalBytes = 0; - for(int docID=0;docID 64KB in size to ensure more than 2 pages in - // PagedBytes would be needed: - int numBytes; - if (doFixed) { - numBytes = fixedLength; - } else if (docID == 0 || random().nextInt(5) == 3) { - numBytes = LARGE_BINARY_FIELD_LENGTH; - } else { - numBytes = TestUtil.nextInt(random(), 1, LARGE_BINARY_FIELD_LENGTH); - } - totalBytes += numBytes; - if (totalBytes > 5 * 1024*1024) { - break; - } - byte[] bytes = new byte[numBytes]; - random().nextBytes(bytes); - docBytes.add(bytes); - Document doc = new Document(); - BytesRef b = new BytesRef(bytes); - b.length = bytes.length; - doc.add(new BinaryDocValuesField("field", b)); - doc.add(new StringField("id", ""+docID, Field.Store.YES)); - w.addDocument(doc); - } - - DirectoryReader r = DirectoryReader.open(w); - w.close(); - - LeafReader ar = SlowCompositeReaderWrapper.wrap(r); - TestUtil.checkReader(ar); - - BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field"); - for(int docID=0;docID