blob: c9fcb6dcc6653c36a1c2fb753c68ce6699302095 [file] [log] [blame]
Index: lucene/CHANGES.txt
===================================================================
--- lucene/CHANGES.txt (revision 1710832)
+++ lucene/CHANGES.txt (working copy)
@@ -162,6 +162,10 @@
* LUCENE-6850: Optimize BooleanScorer for sparse clauses. (Adrien Grand)
+* LUCENE-6840: Ordinal indexes for SORTED_SET/SORTED_NUMERIC fields and
+ addresses for BINARY fields are now stored on disk instead of in memory.
+ (Adrien Grand)
+
Bug Fixes
* LUCENE-6817: ComplexPhraseQueryParser.ComplexPhraseQuery does not display
Index: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
===================================================================
--- lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java (revision 0)
+++ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java (working copy)
@@ -0,0 +1,658 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
+
+/** writer for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+ IndexOutput data, meta;
+ final int maxDoc;
+
+ /** expert: Creates a new writer */
+ public Lucene50DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ 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, Lucene50DocValuesFormat.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, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+ maxDoc = state.segmentInfo.maxDoc();
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(this);
+ }
+ }
+ }
+
+ @Override
+ public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+ addNumericField(field, values, true);
+ }
+
+ void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
+ long count = 0;
+ long minValue = Long.MAX_VALUE;
+ long maxValue = Long.MIN_VALUE;
+ long gcd = 0;
+ long missingCount = 0;
+ long zeroCount = 0;
+ // TODO: more efficient?
+ HashSet<Long> uniqueValues = null;
+ if (optimizeStorage) {
+ uniqueValues = new HashSet<>();
+
+ for (Number nv : values) {
+ final long v;
+ if (nv == null) {
+ v = 0;
+ missingCount++;
+ zeroCount++;
+ } else {
+ v = nv.longValue();
+ if (v == 0) {
+ zeroCount++;
+ }
+ }
+
+ 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);
+ }
+ }
+
+ minValue = Math.min(minValue, v);
+ maxValue = Math.max(maxValue, v);
+
+ if (uniqueValues != null) {
+ if (uniqueValues.add(v)) {
+ if (uniqueValues.size() > 256) {
+ uniqueValues = null;
+ }
+ }
+ }
+
+ ++count;
+ }
+ } else {
+ for (Number nv : values) {
+ long v = nv.longValue();
+ minValue = Math.min(minValue, v);
+ maxValue = Math.max(maxValue, v);
+ ++count;
+ }
+ }
+
+ final long delta = maxValue - minValue;
+ final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+ final int tableBitsRequired = uniqueValues == null
+ ? Integer.MAX_VALUE
+ : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+ final int format;
+ if (uniqueValues != null
+ && count <= Integer.MAX_VALUE
+ && (uniqueValues.size() == 1
+ || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
+ // either one unique value C or two unique values: "missing" and C
+ format = CONST_COMPRESSED;
+ } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
+ format = TABLE_COMPRESSED;
+ } else if (gcd != 0 && gcd != 1) {
+ final long gcdDelta = (maxValue - minValue) / gcd;
+ final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
+ format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
+ } else {
+ format = DELTA_COMPRESSED;
+ }
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
+ meta.writeVInt(format);
+ if (missingCount == 0) {
+ meta.writeLong(ALL_LIVE);
+ } else if (missingCount == count) {
+ meta.writeLong(ALL_MISSING);
+ } else {
+ meta.writeLong(data.getFilePointer());
+ writeMissingBitset(values);
+ }
+ meta.writeLong(data.getFilePointer());
+ meta.writeVLong(count);
+
+ switch (format) {
+ case CONST_COMPRESSED:
+ // write the constant (nonzero value in the n=2 case, singleton value otherwise)
+ meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
+ break;
+ case GCD_COMPRESSED:
+ meta.writeLong(minValue);
+ meta.writeLong(gcd);
+ final long maxDelta = (maxValue - minValue) / gcd;
+ final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
+ meta.writeVInt(bits);
+ final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
+ for (Number nv : values) {
+ long value = nv == null ? 0 : nv.longValue();
+ quotientWriter.add((value - minValue) / gcd);
+ }
+ quotientWriter.finish();
+ break;
+ case DELTA_COMPRESSED:
+ final long minDelta = delta < 0 ? 0 : minValue;
+ meta.writeLong(minDelta);
+ meta.writeVInt(deltaBitsRequired);
+ final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
+ for (Number nv : values) {
+ long v = nv == null ? 0 : nv.longValue();
+ writer.add(v - minDelta);
+ }
+ writer.finish();
+ break;
+ case TABLE_COMPRESSED:
+ final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+ Arrays.sort(decode);
+ final HashMap<Long,Integer> encode = new HashMap<>();
+ meta.writeVInt(decode.length);
+ for (int i = 0; i < decode.length; i++) {
+ meta.writeLong(decode[i]);
+ encode.put(decode[i], i);
+ }
+ meta.writeVInt(tableBitsRequired);
+ final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
+ for (Number nv : values) {
+ ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
+ }
+ ordsWriter.finish();
+ break;
+ default:
+ throw new AssertionError();
+ }
+ meta.writeLong(data.getFilePointer());
+ }
+
+ // 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 {
+ byte bits = 0;
+ int count = 0;
+ for (Object v : values) {
+ if (count == 8) {
+ data.writeByte(bits);
+ count = 0;
+ bits = 0;
+ }
+ if (v != null) {
+ bits |= 1 << (count & 7);
+ }
+ count++;
+ }
+ if (count > 0) {
+ data.writeByte(bits);
+ }
+ }
+
+ @Override
+ public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+ // write the byte[] data
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.BINARY);
+ int minLength = Integer.MAX_VALUE;
+ int maxLength = Integer.MIN_VALUE;
+ final long startFP = data.getFilePointer();
+ long count = 0;
+ long missingCount = 0;
+ for(BytesRef v : values) {
+ final int length;
+ if (v == null) {
+ length = 0;
+ missingCount++;
+ } else {
+ length = v.length;
+ }
+ minLength = Math.min(minLength, length);
+ maxLength = Math.max(maxLength, length);
+ if (v != null) {
+ data.writeBytes(v.bytes, v.offset, v.length);
+ }
+ count++;
+ }
+ meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
+ if (missingCount == 0) {
+ meta.writeLong(ALL_LIVE);
+ } else if (missingCount == count) {
+ meta.writeLong(ALL_MISSING);
+ } else {
+ meta.writeLong(data.getFilePointer());
+ writeMissingBitset(values);
+ }
+ meta.writeVInt(minLength);
+ meta.writeVInt(maxLength);
+ meta.writeVLong(count);
+ meta.writeLong(startFP);
+
+ // 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.writeLong(data.getFilePointer());
+ meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+ long addr = 0;
+ writer.add(addr);
+ for (BytesRef v : values) {
+ if (v != null) {
+ addr += v.length;
+ }
+ writer.add(addr);
+ }
+ writer.finish();
+ }
+ }
+
+ /** expert: writes a value dictionary for a sorted/sortedset field */
+ private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+ // first check if it's a "fixed-length" terms dict
+ int minLength = Integer.MAX_VALUE;
+ int maxLength = Integer.MIN_VALUE;
+ long numValues = 0;
+ for (BytesRef v : values) {
+ minLength = Math.min(minLength, v.length);
+ maxLength = Math.max(maxLength, v.length);
+ numValues++;
+ }
+ if (minLength == maxLength) {
+ // no index needed: direct addressing by mult
+ addBinaryField(field, values);
+ } else if (numValues < REVERSE_INTERVAL_COUNT) {
+ // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+ addBinaryField(field, values);
+ } else {
+ assert numValues > 0; // we don't have to handle the empty case
+ // header
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.BINARY);
+ meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+ meta.writeLong(-1L);
+ // now write the bytes: sharing prefixes within a block
+ final long startFP = data.getFilePointer();
+ // currently, we have to store the delta from expected for every 1/nth term
+ // we could avoid this, but it's not much and less overall RAM than the previous approach!
+ RAMOutputStream addressBuffer = new RAMOutputStream();
+ MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
+ // buffers up 16 terms
+ RAMOutputStream bytesBuffer = new RAMOutputStream();
+ // buffers up block header
+ RAMOutputStream headerBuffer = new RAMOutputStream();
+ BytesRefBuilder lastTerm = new BytesRefBuilder();
+ lastTerm.grow(maxLength);
+ long count = 0;
+ int suffixDeltas[] = new int[INTERVAL_COUNT];
+ for (BytesRef v : values) {
+ int termPosition = (int) (count & INTERVAL_MASK);
+ if (termPosition == 0) {
+ termAddresses.add(data.getFilePointer() - startFP);
+ // abs-encode first term
+ headerBuffer.writeVInt(v.length);
+ headerBuffer.writeBytes(v.bytes, v.offset, v.length);
+ lastTerm.copyBytes(v);
+ } else {
+ // prefix-code: we only share at most 255 characters, to encode the length as a single
+ // byte and have random access. Larger terms just get less compression.
+ int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
+ bytesBuffer.writeByte((byte) sharedPrefix);
+ bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+ // we can encode one smaller, because terms are unique.
+ suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
+ }
+
+ count++;
+ // flush block
+ if ((count & INTERVAL_MASK) == 0) {
+ flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+ }
+ }
+ // flush trailing crap
+ int leftover = (int) (count & INTERVAL_MASK);
+ if (leftover > 0) {
+ Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
+ flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+ }
+ final long indexStartFP = data.getFilePointer();
+ // write addresses of indexed terms
+ termAddresses.finish();
+ addressBuffer.writeTo(data);
+ addressBuffer = null;
+ termAddresses = null;
+ meta.writeVInt(minLength);
+ meta.writeVInt(maxLength);
+ meta.writeVLong(count);
+ meta.writeLong(startFP);
+ meta.writeLong(indexStartFP);
+ meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+ addReverseTermIndex(field, values, maxLength);
+ }
+ }
+
+ // writes term dictionary "block"
+ // first term is absolute encoded as vint length + bytes.
+ // lengths of subsequent N terms are encoded as either N bytes or N shorts.
+ // in the double-byte case, the first byte is indicated with -1.
+ // subsequent terms are encoded as byte suffixLength + bytes.
+ private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
+ boolean twoByte = false;
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ if (suffixDeltas[i] > 254) {
+ twoByte = true;
+ }
+ }
+ if (twoByte) {
+ headerBuffer.writeByte((byte)255);
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ headerBuffer.writeShort((short) suffixDeltas[i]);
+ }
+ } else {
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ headerBuffer.writeByte((byte) suffixDeltas[i]);
+ }
+ }
+ headerBuffer.writeTo(data);
+ headerBuffer.reset();
+ bytesBuffer.writeTo(data);
+ bytesBuffer.reset();
+ }
+
+ // writes reverse term index: used for binary searching a term into a range of 64 blocks
+ // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
+ // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
+ private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
+ long count = 0;
+ BytesRefBuilder priorTerm = new BytesRefBuilder();
+ priorTerm.grow(maxLength);
+ BytesRef indexTerm = new BytesRef();
+ long startFP = data.getFilePointer();
+ PagedBytes pagedBytes = new PagedBytes(15);
+ MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+
+ for (BytesRef b : values) {
+ int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
+ if (termPosition == 0) {
+ int len = StringHelper.sortKeyLength(priorTerm.get(), b);
+ indexTerm.bytes = b.bytes;
+ indexTerm.offset = b.offset;
+ indexTerm.length = len;
+ addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
+ } else if (termPosition == REVERSE_INTERVAL_MASK) {
+ priorTerm.copyBytes(b);
+ }
+ count++;
+ }
+ addresses.finish();
+ long numBytes = pagedBytes.getPointer();
+ pagedBytes.freeze(true);
+ PagedBytesDataInput in = pagedBytes.getDataInput();
+ meta.writeLong(startFP);
+ data.writeVLong(numBytes);
+ data.copyBytes(in, numBytes);
+ }
+
+ @Override
+ public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.SORTED);
+ addTermsDict(field, values);
+ addNumericField(field, docToOrd, false);
+ }
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.SORTED_NUMERIC);
+ if (isSingleValued(docToValueCount)) {
+ meta.writeVInt(SORTED_SINGLE_VALUED);
+ // The field is single-valued, we can encode it as NUMERIC
+ addNumericField(field, singletonView(docToValueCount, values, null));
+ } else {
+ final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
+ if (uniqueValueSets != null) {
+ meta.writeVInt(SORTED_SET_TABLE);
+
+ // write the set_id -> values mapping
+ writeDictionary(uniqueValueSets);
+
+ // write the doc -> set_id as a numeric field
+ addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), false);
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
+ // write the stream of values as a numeric field
+ addNumericField(field, values, true);
+ // write the doc -> ord count as a absolute index to the stream
+ addAddresses(field, docToValueCount);
+ }
+ }
+ }
+
+ @Override
+ public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.SORTED_SET);
+
+ if (isSingleValued(docToOrdCount)) {
+ meta.writeVInt(SORTED_SINGLE_VALUED);
+ // The field is single-valued, we can encode it as SORTED
+ addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+ } else {
+ final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
+ if (uniqueValueSets != null) {
+ meta.writeVInt(SORTED_SET_TABLE);
+
+ // write the set_id -> ords mapping
+ writeDictionary(uniqueValueSets);
+
+ // write the ord -> byte[] as a binary field
+ addTermsDict(field, values);
+
+ // write the doc -> set_id as a numeric field
+ addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), false);
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+ // write the ord -> byte[] as a binary field
+ addTermsDict(field, values);
+
+ // write the stream of ords as a numeric field
+ // NOTE: we could return an iterator that delta-encodes these within a doc
+ addNumericField(field, ords, false);
+
+ // write the doc -> ord count as a absolute index to the stream
+ addAddresses(field, docToOrdCount);
+ }
+ }
+ }
+
+ private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
+ Set<LongsRef> uniqueValueSet = new HashSet<>();
+ LongsRef docValues = new LongsRef(256);
+
+ Iterator<Number> valueCountIterator = docToValueCount.iterator();
+ Iterator<Number> valueIterator = values.iterator();
+ int totalDictSize = 0;
+ while (valueCountIterator.hasNext()) {
+ docValues.length = valueCountIterator.next().intValue();
+ if (docValues.length > 256) {
+ return null;
+ }
+ for (int i = 0; i < docValues.length; ++i) {
+ docValues.longs[i] = valueIterator.next().longValue();
+ }
+ if (uniqueValueSet.contains(docValues)) {
+ continue;
+ }
+ totalDictSize += docValues.length;
+ if (totalDictSize > 256) {
+ return null;
+ }
+ uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
+ }
+ assert valueIterator.hasNext() == false;
+ return new TreeSet<>(uniqueValueSet);
+ }
+
+ private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
+ int lengthSum = 0;
+ for (LongsRef longs : uniqueValueSets) {
+ lengthSum += longs.length;
+ }
+
+ meta.writeInt(lengthSum);
+ for (LongsRef valueSet : uniqueValueSets) {
+ for (int i = 0; i < valueSet.length; ++i) {
+ meta.writeLong(valueSet.longs[valueSet.offset + i]);
+ }
+ }
+
+ meta.writeInt(uniqueValueSets.size());
+ for (LongsRef valueSet : uniqueValueSets) {
+ meta.writeInt(valueSet.length);
+ }
+ }
+
+ private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
+ final Map<LongsRef, Integer> setIds = new HashMap<>();
+ int i = 0;
+ for (LongsRef set : uniqueValueSets) {
+ setIds.put(set, i++);
+ }
+ assert i == uniqueValueSets.size();
+
+ return new Iterable<Number>() {
+
+ @Override
+ public Iterator<Number> iterator() {
+ final Iterator<Number> valueCountIterator = docToValueCount.iterator();
+ final Iterator<Number> valueIterator = values.iterator();
+ final LongsRef docValues = new LongsRef(256);
+ return new Iterator<Number>() {
+
+ @Override
+ public boolean hasNext() {
+ return valueCountIterator.hasNext();
+ }
+
+ @Override
+ public Number next() {
+ docValues.length = valueCountIterator.next().intValue();
+ for (int i = 0; i < docValues.length; ++i) {
+ docValues.longs[i] = valueIterator.next().longValue();
+ }
+ final Integer id = setIds.get(docValues);
+ assert id != null;
+ return id;
+ }
+
+ };
+
+ }
+ };
+ }
+
+ // writes addressing information as MONOTONIC_COMPRESSED integer
+ private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
+ meta.writeVInt(MONOTONIC_COMPRESSED);
+ meta.writeLong(-1L);
+ meta.writeLong(data.getFilePointer());
+ meta.writeVLong(maxDoc);
+ meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+ long addr = 0;
+ writer.add(addr);
+ for (Number v : values) {
+ addr += v.longValue();
+ writer.add(addr);
+ }
+ writer.finish();
+ meta.writeLong(data.getFilePointer());
+ }
+
+ @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); // write checksum
+ }
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(data, meta);
+ } else {
+ IOUtils.closeWhileHandlingException(data, meta);
+ }
+ meta = data = null;
+ }
+ }
+}
Property changes on: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
===================================================================
--- lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (revision 0)
+++ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (working copy)
@@ -0,0 +1,115 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * Lucene 5.0 Doc values format.
+ * @deprecated Only for reading old 5.0-5.3 segments
+ */
+@Deprecated
+public class Lucene50DocValuesFormat extends DocValuesFormat {
+
+ /** Sole Constructor */
+ public Lucene50DocValuesFormat() {
+ super("Lucene50");
+ }
+
+ @Override
+ public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ @Override
+ public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+ return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ static final String DATA_CODEC = "Lucene50DocValuesData";
+ static final String DATA_EXTENSION = "dvd";
+ static final String META_CODEC = "Lucene50DocValuesMetadata";
+ static final String META_EXTENSION = "dvm";
+ static final int VERSION_START = 0;
+ static final int VERSION_SORTEDSET_TABLE = 1;
+ static final int VERSION_CURRENT = VERSION_SORTEDSET_TABLE;
+
+ // indicates docvalues type
+ static final byte NUMERIC = 0;
+ static final byte BINARY = 1;
+ static final byte SORTED = 2;
+ static final byte SORTED_SET = 3;
+ static final byte SORTED_NUMERIC = 4;
+
+ // address terms in blocks of 16 terms
+ static final int INTERVAL_SHIFT = 4;
+ static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
+ static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
+
+ // build reverse index from every 1024th term
+ static final int REVERSE_INTERVAL_SHIFT = 10;
+ static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
+ static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
+
+ // for conversion from reverse index to block
+ static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
+ static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
+ static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
+
+ /** Compressed using packed blocks of ints. */
+ static final int DELTA_COMPRESSED = 0;
+ /** Compressed by computing the GCD. */
+ static final int GCD_COMPRESSED = 1;
+ /** Compressed by giving IDs to unique values. */
+ static final int TABLE_COMPRESSED = 2;
+ /** Compressed with monotonically increasing values */
+ static final int MONOTONIC_COMPRESSED = 3;
+ /** Compressed with constant value (uses only missing bitset) */
+ static final int CONST_COMPRESSED = 4;
+
+ /** Uncompressed binary, written directly (fixed length). */
+ static final int BINARY_FIXED_UNCOMPRESSED = 0;
+ /** Uncompressed binary, written directly (variable length). */
+ static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+ /** Compressed binary with shared prefixes */
+ static final int BINARY_PREFIX_COMPRESSED = 2;
+
+ /** Standard storage for sorted set values with 1 level of indirection:
+ * {@code docId -> address -> ord}. */
+ static final int SORTED_WITH_ADDRESSES = 0;
+ /** Single-valued sorted set values, encoded as sorted values, so no level
+ * of indirection: {@code docId -> ord}. */
+ static final int SORTED_SINGLE_VALUED = 1;
+ /** Compressed giving IDs to unique sets of values:
+ * {@code docId -> setId -> ords} */
+ static final int SORTED_SET_TABLE = 2;
+
+ /** placeholder for missing offset that means there are no missing values */
+ static final int ALL_LIVE = -1;
+ /** placeholder for missing offset that means all values are missing */
+ static final int ALL_MISSING = -2;
+
+ // addressing uses 16k blocks
+ static final int MONOTONIC_BLOCK_SIZE = 16384;
+}
Property changes on: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
===================================================================
--- lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (revision 0)
+++ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (working copy)
@@ -0,0 +1,1299 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable;
+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.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomAccessOrds;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+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.IOUtils;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
+
+/** reader for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
+ private final Map<String,NumericEntry> numerics = new HashMap<>();
+ private final Map<String,BinaryEntry> binaries = new HashMap<>();
+ private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+ private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+ private final Map<String,NumericEntry> ords = new HashMap<>();
+ private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
+ private final int numFields;
+ private final AtomicLong ramBytesUsed;
+ private final IndexInput data;
+ private final int maxDoc;
+
+ // memory-resident structures
+ private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+ private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
+ private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
+
+ private final boolean merging;
+
+ // clone for merge: when merging we don't do any instances.put()s
+ Lucene50DocValuesProducer(Lucene50DocValuesProducer original) throws IOException {
+ assert Thread.holdsLock(original);
+ numerics.putAll(original.numerics);
+ binaries.putAll(original.binaries);
+ sortedSets.putAll(original.sortedSets);
+ sortedNumerics.putAll(original.sortedNumerics);
+ ords.putAll(original.ords);
+ ordIndexes.putAll(original.ordIndexes);
+ numFields = original.numFields;
+ ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+ data = original.data.clone();
+ maxDoc = original.maxDoc;
+
+ addressInstances.putAll(original.addressInstances);
+ ordIndexInstances.putAll(original.ordIndexInstances);
+ reverseIndexInstances.putAll(original.reverseIndexInstances);
+ merging = true;
+ }
+
+ /** expert: instantiates a new reader */
+ Lucene50DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+ this.maxDoc = state.segmentInfo.maxDoc();
+ merging = false;
+ ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+
+ int version = -1;
+ int numFields = -1;
+
+ // read in the entries from the metadata file.
+ try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+ Throwable priorE = null;
+ try {
+ version = CodecUtil.checkIndexHeader(in, metaCodec,
+ Lucene50DocValuesFormat.VERSION_START,
+ Lucene50DocValuesFormat.VERSION_CURRENT,
+ state.segmentInfo.getId(),
+ state.segmentSuffix);
+ numFields = readFields(in, state.fieldInfos);
+ } catch (Throwable exception) {
+ priorE = exception;
+ } finally {
+ CodecUtil.checkFooter(in, priorE);
+ }
+ }
+
+ this.numFields = numFields;
+ String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+ this.data = state.directory.openInput(dataName, state.context);
+ boolean success = false;
+ try {
+ final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
+ Lucene50DocValuesFormat.VERSION_START,
+ Lucene50DocValuesFormat.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 void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
+ // sorted = binary + numeric
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ BinaryEntry b = readBinaryEntry(meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(meta);
+ ords.put(info.name, n);
+ }
+
+ private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
+ // sortedset = binary + numeric (addresses) + ordIndex
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ BinaryEntry b = readBinaryEntry(meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n1 = readNumericEntry(meta);
+ ords.put(info.name, n1);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n2 = readNumericEntry(meta);
+ ordIndexes.put(info.name, n2);
+ }
+
+ private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
+ // sortedset table = binary + ordset table + ordset index
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+
+ BinaryEntry b = readBinaryEntry(meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(meta);
+ ords.put(info.name, n);
+ }
+
+ private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+ int numFields = 0;
+ int fieldNumber = meta.readVInt();
+ while (fieldNumber != -1) {
+ numFields++;
+ FieldInfo info = infos.fieldInfo(fieldNumber);
+ if (info == null) {
+ // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
+ throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+ }
+ byte type = meta.readByte();
+ if (type == Lucene50DocValuesFormat.NUMERIC) {
+ numerics.put(info.name, readNumericEntry(meta));
+ } else if (type == Lucene50DocValuesFormat.BINARY) {
+ BinaryEntry b = readBinaryEntry(meta);
+ binaries.put(info.name, b);
+ } else if (type == Lucene50DocValuesFormat.SORTED) {
+ readSortedField(info, meta);
+ } else if (type == Lucene50DocValuesFormat.SORTED_SET) {
+ SortedSetEntry ss = readSortedSetEntry(meta);
+ sortedSets.put(info.name, ss);
+ if (ss.format == SORTED_WITH_ADDRESSES) {
+ readSortedSetFieldWithAddresses(info, meta);
+ } else if (ss.format == SORTED_SET_TABLE) {
+ readSortedSetFieldWithTable(info, meta);
+ } else if (ss.format == SORTED_SINGLE_VALUED) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.SORTED) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ readSortedField(info, meta);
+ } else {
+ throw new AssertionError();
+ }
+ } else if (type == Lucene50DocValuesFormat.SORTED_NUMERIC) {
+ SortedSetEntry ss = readSortedSetEntry(meta);
+ sortedNumerics.put(info.name, ss);
+ if (ss.format == SORTED_WITH_ADDRESSES) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ numerics.put(info.name, readNumericEntry(meta));
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry ordIndex = readNumericEntry(meta);
+ ordIndexes.put(info.name, ordIndex);
+ } else if (ss.format == SORTED_SET_TABLE) {
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(meta);
+ ords.put(info.name, n);
+ } else if (ss.format == SORTED_SINGLE_VALUED) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ numerics.put(info.name, readNumericEntry(meta));
+ } else {
+ throw new AssertionError();
+ }
+ } else {
+ throw new CorruptIndexException("invalid type: " + type, meta);
+ }
+ fieldNumber = meta.readVInt();
+ }
+ return numFields;
+ }
+
+ private NumericEntry readNumericEntry(IndexInput meta) throws IOException {
+ NumericEntry entry = new NumericEntry();
+ entry.format = meta.readVInt();
+ entry.missingOffset = meta.readLong();
+ entry.offset = meta.readLong();
+ entry.count = meta.readVLong();
+ switch(entry.format) {
+ case CONST_COMPRESSED:
+ entry.minValue = meta.readLong();
+ if (entry.count > Integer.MAX_VALUE) {
+ // currently just a limitation e.g. of bits interface and so on.
+ throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
+ }
+ break;
+ case GCD_COMPRESSED:
+ entry.minValue = meta.readLong();
+ entry.gcd = meta.readLong();
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case TABLE_COMPRESSED:
+ final int uniqueValues = meta.readVInt();
+ if (uniqueValues > 256) {
+ throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
+ }
+ entry.table = new long[uniqueValues];
+ for (int i = 0; i < uniqueValues; ++i) {
+ entry.table[i] = meta.readLong();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case DELTA_COMPRESSED:
+ entry.minValue = meta.readLong();
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case MONOTONIC_COMPRESSED:
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
+ }
+ entry.endOffset = meta.readLong();
+ return entry;
+ }
+
+ static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
+ BinaryEntry entry = new BinaryEntry();
+ entry.format = meta.readVInt();
+ entry.missingOffset = meta.readLong();
+ entry.minLength = meta.readVInt();
+ entry.maxLength = meta.readVInt();
+ entry.count = meta.readVLong();
+ entry.offset = meta.readLong();
+ switch(entry.format) {
+ case BINARY_FIXED_UNCOMPRESSED:
+ break;
+ case BINARY_PREFIX_COMPRESSED:
+ entry.addressesOffset = meta.readLong();
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ entry.reverseIndexOffset = meta.readLong();
+ break;
+ case BINARY_VARIABLE_UNCOMPRESSED:
+ entry.addressesOffset = meta.readLong();
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+ }
+ return entry;
+ }
+
+ SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+ SortedSetEntry entry = new SortedSetEntry();
+ entry.format = meta.readVInt();
+ if (entry.format == SORTED_SET_TABLE) {
+ final int totalTableLength = meta.readInt();
+ if (totalTableLength > 256) {
+ throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
+ }
+ entry.table = new long[totalTableLength];
+ for (int i = 0; i < totalTableLength; ++i) {
+ entry.table[i] = meta.readLong();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+ final int tableSize = meta.readInt();
+ if (tableSize > totalTableLength + 1) { // +1 because of the empty set
+ throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
+ }
+ entry.tableOffsets = new int[tableSize + 1];
+ for (int i = 1; i < entry.tableOffsets.length; ++i) {
+ entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
+ } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
+ throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+ }
+ return entry;
+ }
+
+ @Override
+ public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+ NumericEntry entry = numerics.get(field.name);
+ return getNumeric(entry);
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return ramBytesUsed.get();
+ }
+
+ @Override
+ public synchronized Collection<Accountable> getChildResources() {
+ List<Accountable> resources = new ArrayList<>();
+ resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
+ resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
+ resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
+ return Collections.unmodifiableList(resources);
+ }
+
+ @Override
+ public void checkIntegrity() throws IOException {
+ CodecUtil.checksumEntireFile(data);
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(fields=" + numFields + ")";
+ }
+
+ LongValues getNumeric(NumericEntry entry) throws IOException {
+ switch (entry.format) {
+ case CONST_COMPRESSED: {
+ final long constant = entry.minValue;
+ final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
+ return new LongValues() {
+ @Override
+ public long get(long index) {
+ return live.get((int)index) ? constant : 0;
+ }
+ };
+ }
+ case DELTA_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long delta = entry.minValue;
+ final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return delta + values.get(id);
+ }
+ };
+ }
+ case GCD_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long min = entry.minValue;
+ final long mult = entry.gcd;
+ final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return min + mult * quotientReader.get(id);
+ }
+ };
+ }
+ case TABLE_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long table[] = entry.table;
+ final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return table[(int) ords.get(id)];
+ }
+ };
+ }
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+ BinaryEntry bytes = binaries.get(field.name);
+ switch(bytes.format) {
+ case BINARY_FIXED_UNCOMPRESSED:
+ return getFixedBinary(field, bytes);
+ case BINARY_VARIABLE_UNCOMPRESSED:
+ return getVariableBinary(field, bytes);
+ case BINARY_PREFIX_COMPRESSED:
+ return getCompressedBinary(field, bytes);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
+
+ final BytesRef term = new BytesRef(bytes.maxLength);
+ final byte[] buffer = term.bytes;
+ final int length = term.length = bytes.maxLength;
+
+ return new LongBinaryDocValues() {
+ @Override
+ public BytesRef get(long id) {
+ try {
+ data.seek(id * length);
+ data.readBytes(buffer, 0, buffer.length);
+ return term;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ }
+
+ /** returns an address instance for variable-length binary values. */
+ private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+ MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+ if (addresses == null) {
+ data.seek(bytes.addressesOffset);
+ addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
+ if (!merging) {
+ addressInstances.put(field.name, addresses);
+ ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+ }
+ }
+ return addresses;
+ }
+
+ private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final MonotonicBlockPackedReader addresses = getAddressInstance(field, bytes);
+
+ final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
+ final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
+ final byte buffer[] = term.bytes;
+
+ return new LongBinaryDocValues() {
+ @Override
+ public BytesRef get(long id) {
+ long startAddress = addresses.get(id);
+ long endAddress = addresses.get(id+1);
+ int length = (int) (endAddress - startAddress);
+ try {
+ data.seek(startAddress);
+ data.readBytes(buffer, 0, length);
+ term.length = length;
+ return term;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ }
+
+ /** returns an address instance for prefix-compressed binary values. */
+ private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+ MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+ if (addresses == null) {
+ data.seek(bytes.addressesOffset);
+ final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
+ addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+ if (!merging) {
+ addressInstances.put(field.name, addresses);
+ ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+ }
+ }
+ return addresses;
+ }
+
+ /** returns a reverse lookup instance for prefix-compressed binary values. */
+ private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+ ReverseTermsIndex index = reverseIndexInstances.get(field.name);
+ if (index == null) {
+ index = new ReverseTermsIndex();
+ data.seek(bytes.reverseIndexOffset);
+ long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
+ index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+ long dataSize = data.readVLong();
+ PagedBytes pagedBytes = new PagedBytes(15);
+ pagedBytes.copy(data, dataSize);
+ index.terms = pagedBytes.freeze(true);
+ if (!merging) {
+ reverseIndexInstances.put(field.name, index);
+ ramBytesUsed.addAndGet(index.ramBytesUsed());
+ }
+ }
+ return index;
+ }
+
+ private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
+ final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
+ assert addresses.size() > 0; // we don't have to handle empty case
+ IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
+ return new CompressedBinaryDocValues(bytes, addresses, index, slice);
+ }
+
+ @Override
+ public SortedDocValues getSorted(FieldInfo field) throws IOException {
+ final int valueCount = (int) binaries.get(field.name).count;
+ final BinaryDocValues binary = getBinary(field);
+ NumericEntry entry = ords.get(field.name);
+ final LongValues ordinals = getNumeric(entry);
+ return new SortedDocValues() {
+
+ @Override
+ public int getOrd(int docID) {
+ return (int) ordinals.get(docID);
+ }
+
+ @Override
+ public BytesRef lookupOrd(int ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public int getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public int lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+ };
+ }
+
+ /** returns an address instance for sortedset ordinal lists */
+ private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
+ MonotonicBlockPackedReader instance = ordIndexInstances.get(field.name);
+ if (instance == null) {
+ data.seek(entry.offset);
+ instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
+ if (!merging) {
+ ordIndexInstances.put(field.name, instance);
+ ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+ }
+ }
+ return instance;
+ }
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedNumerics.get(field.name);
+ if (ss.format == SORTED_SINGLE_VALUED) {
+ NumericEntry numericEntry = numerics.get(field.name);
+ final LongValues values = getNumeric(numericEntry);
+ final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+ return DocValues.singleton(values, docsWithField);
+ } else if (ss.format == SORTED_WITH_ADDRESSES) {
+ NumericEntry numericEntry = numerics.get(field.name);
+ final LongValues values = getNumeric(numericEntry);
+ final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+ return new SortedNumericDocValues() {
+ long startOffset;
+ long endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ startOffset = ordIndex.get(doc);
+ endOffset = ordIndex.get(doc+1L);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values.get(startOffset + index);
+ }
+
+ @Override
+ public int count() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ } else if (ss.format == SORTED_SET_TABLE) {
+ NumericEntry entry = ords.get(field.name);
+ final LongValues ordinals = getNumeric(entry);
+
+ final long[] table = ss.table;
+ final int[] offsets = ss.tableOffsets;
+ return new SortedNumericDocValues() {
+ int startOffset;
+ int endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ final int ord = (int) ordinals.get(doc);
+ startOffset = offsets[ord];
+ endOffset = offsets[ord + 1];
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return table[startOffset + index];
+ }
+
+ @Override
+ public int count() {
+ return endOffset - startOffset;
+ }
+ };
+ } else {
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedSets.get(field.name);
+ switch (ss.format) {
+ case SORTED_SINGLE_VALUED:
+ final SortedDocValues values = getSorted(field);
+ return DocValues.singleton(values);
+ case SORTED_WITH_ADDRESSES:
+ return getSortedSetWithAddresses(field);
+ case SORTED_SET_TABLE:
+ return getSortedSetTable(field, ss);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
+ final long valueCount = binaries.get(field.name).count;
+ // we keep the byte[]s and list of ords on disk, these could be large
+ final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+ final LongValues ordinals = getNumeric(ords.get(field.name));
+ // but the addresses to the ord stream are in RAM
+ final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+ return new RandomAccessOrds() {
+ long startOffset;
+ long offset;
+ long endOffset;
+
+ @Override
+ public long nextOrd() {
+ if (offset == endOffset) {
+ return NO_MORE_ORDS;
+ } else {
+ long ord = ordinals.get(offset);
+ offset++;
+ return ord;
+ }
+ }
+
+ @Override
+ public void setDocument(int docID) {
+ startOffset = offset = ordIndex.get(docID);
+ endOffset = ordIndex.get(docID+1L);
+ }
+
+ @Override
+ public BytesRef lookupOrd(long ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public long getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public long lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+
+ @Override
+ public long ordAt(int index) {
+ return ordinals.get(startOffset + index);
+ }
+
+ @Override
+ public int cardinality() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ }
+
+ private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
+ final long valueCount = binaries.get(field.name).count;
+ final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+ final LongValues ordinals = getNumeric(ords.get(field.name));
+
+ final long[] table = ss.table;
+ final int[] offsets = ss.tableOffsets;
+
+ return new RandomAccessOrds() {
+
+ int offset, startOffset, endOffset;
+
+ @Override
+ public void setDocument(int docID) {
+ final int ord = (int) ordinals.get(docID);
+ offset = startOffset = offsets[ord];
+ endOffset = offsets[ord + 1];
+ }
+
+ @Override
+ public long ordAt(int index) {
+ return table[startOffset + index];
+ }
+
+ @Override
+ public long nextOrd() {
+ if (offset == endOffset) {
+ return NO_MORE_ORDS;
+ } else {
+ return table[offset++];
+ }
+ }
+
+ @Override
+ public int cardinality() {
+ return endOffset - startOffset;
+ }
+
+ @Override
+ public BytesRef lookupOrd(long ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public long getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public long lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues) binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues) binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+
+ };
+ }
+
+ private Bits getLiveBits(final long offset, final int count) throws IOException {
+ if (offset == ALL_MISSING) {
+ return new Bits.MatchNoBits(count);
+ } else if (offset == ALL_LIVE) {
+ return new Bits.MatchAllBits(count);
+ } else {
+ int length = (int) ((count + 7L) >>> 3);
+ final RandomAccessInput in = data.randomAccessSlice(offset, length);
+ return new Bits() {
+ @Override
+ public boolean get(int index) {
+ try {
+ return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public int length() {
+ return count;
+ }
+ };
+ }
+ }
+
+ @Override
+ public Bits getDocsWithField(FieldInfo field) throws IOException {
+ switch(field.getDocValuesType()) {
+ case SORTED_SET:
+ return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+ case SORTED_NUMERIC:
+ return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
+ case SORTED:
+ return DocValues.docsWithValue(getSorted(field), maxDoc);
+ case BINARY:
+ BinaryEntry be = binaries.get(field.name);
+ return getLiveBits(be.missingOffset, maxDoc);
+ case NUMERIC:
+ NumericEntry ne = numerics.get(field.name);
+ return getLiveBits(ne.missingOffset, maxDoc);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public synchronized DocValuesProducer getMergeInstance() throws IOException {
+ return new Lucene50DocValuesProducer(this);
+ }
+
+ @Override
+ public void close() throws IOException {
+ data.close();
+ }
+
+ /** metadata entry for a numeric docvalues field */
+ static class NumericEntry {
+ private NumericEntry() {}
+ /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+ long missingOffset;
+ /** offset to the actual numeric values */
+ public long offset;
+ /** end offset to the actual numeric values */
+ public long endOffset;
+ /** bits per value used to pack the numeric values */
+ public int bitsPerValue;
+
+ int format;
+ /** packed ints version used to encode these numerics */
+ public int packedIntsVersion;
+ /** count of values written */
+ public long count;
+ /** packed ints blocksize */
+ public int blockSize;
+
+ long minValue;
+ long gcd;
+ long table[];
+ }
+
+ /** metadata entry for a binary docvalues field */
+ static class BinaryEntry {
+ private BinaryEntry() {}
+ /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+ long missingOffset;
+ /** offset to the actual binary values */
+ long offset;
+
+ int format;
+ /** count of values written */
+ public long count;
+ int minLength;
+ int maxLength;
+ /** offset to the addressing data that maps a value to its slice of the byte[] */
+ public long addressesOffset;
+ /** offset to the reverse index */
+ public long reverseIndexOffset;
+ /** packed ints version used to encode addressing information */
+ public int packedIntsVersion;
+ /** packed ints blocksize */
+ public int blockSize;
+ }
+
+ /** metadata entry for a sorted-set docvalues field */
+ static class SortedSetEntry {
+ private SortedSetEntry() {}
+ int format;
+
+ long[] table;
+ int[] tableOffsets;
+ }
+
+ // internally we compose complex dv (sorted/sortedset) from other ones
+ static abstract class LongBinaryDocValues extends BinaryDocValues {
+ @Override
+ public final BytesRef get(int docID) {
+ return get((long)docID);
+ }
+
+ abstract BytesRef get(long id);
+ }
+
+ // used for reverse lookup to a small range of blocks
+ static class ReverseTermsIndex implements Accountable {
+ public MonotonicBlockPackedReader termAddresses;
+ public PagedBytes.Reader terms;
+
+ @Override
+ public long ramBytesUsed() {
+ return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+ }
+
+ @Override
+ public Collection<Accountable> getChildResources() {
+ List<Accountable> resources = new ArrayList<>();
+ resources.add(Accountables.namedAccountable("term bytes", terms));
+ resources.add(Accountables.namedAccountable("term addresses", termAddresses));
+ return Collections.unmodifiableList(resources);
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+ }
+ }
+
+ //in the compressed case, we add a few additional operations for
+ //more efficient reverse lookup and enumeration
+ static final class CompressedBinaryDocValues extends LongBinaryDocValues {
+ final long numValues;
+ final long numIndexValues;
+ final int maxTermLength;
+ final MonotonicBlockPackedReader addresses;
+ final IndexInput data;
+ final CompressedBinaryTermsEnum termsEnum;
+ final PagedBytes.Reader reverseTerms;
+ final MonotonicBlockPackedReader reverseAddresses;
+ final long numReverseIndexValues;
+
+ public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
+ this.maxTermLength = bytes.maxLength;
+ this.numValues = bytes.count;
+ this.addresses = addresses;
+ this.numIndexValues = addresses.size();
+ this.data = data;
+ this.reverseTerms = index.terms;
+ this.reverseAddresses = index.termAddresses;
+ this.numReverseIndexValues = reverseAddresses.size();
+ this.termsEnum = getTermsEnum(data);
+ }
+
+ @Override
+ public BytesRef get(long id) {
+ try {
+ termsEnum.seekExact(id);
+ return termsEnum.term();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ long lookupTerm(BytesRef key) {
+ try {
+ switch (termsEnum.seekCeil(key)) {
+ case FOUND: return termsEnum.ord();
+ case NOT_FOUND: return -termsEnum.ord()-1;
+ default: return -numValues-1;
+ }
+ } catch (IOException bogus) {
+ throw new RuntimeException(bogus);
+ }
+ }
+
+ TermsEnum getTermsEnum() {
+ try {
+ return getTermsEnum(data.clone());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
+ return new CompressedBinaryTermsEnum(input);
+ }
+
+ class CompressedBinaryTermsEnum extends TermsEnum {
+ private long currentOrd = -1;
+ // offset to the start of the current block
+ private long currentBlockStart;
+ private final IndexInput input;
+ // delta from currentBlockStart to start of each term
+ private final int offsets[] = new int[INTERVAL_COUNT];
+ private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
+
+ private final BytesRef term = new BytesRef(maxTermLength);
+ private final BytesRef firstTerm = new BytesRef(maxTermLength);
+ private final BytesRef scratch = new BytesRef();
+
+ CompressedBinaryTermsEnum(IndexInput input) throws IOException {
+ this.input = input;
+ input.seek(0);
+ }
+
+ private void readHeader() throws IOException {
+ firstTerm.length = input.readVInt();
+ input.readBytes(firstTerm.bytes, 0, firstTerm.length);
+ input.readBytes(buffer, 0, INTERVAL_COUNT-1);
+ if (buffer[0] == -1) {
+ readShortAddresses();
+ } else {
+ readByteAddresses();
+ }
+ currentBlockStart = input.getFilePointer();
+ }
+
+ // read single byte addresses: each is delta - 2
+ // (shared prefix byte and length > 0 are both implicit)
+ private void readByteAddresses() throws IOException {
+ int addr = 0;
+ for (int i = 1; i < offsets.length; i++) {
+ addr += 2 + (buffer[i-1] & 0xFF);
+ offsets[i] = addr;
+ }
+ }
+
+ // read double byte addresses: each is delta - 2
+ // (shared prefix byte and length > 0 are both implicit)
+ private void readShortAddresses() throws IOException {
+ input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
+ int addr = 0;
+ for (int i = 1; i < offsets.length; i++) {
+ int x = i<<1;
+ addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
+ offsets[i] = addr;
+ }
+ }
+
+ // set term to the first term
+ private void readFirstTerm() throws IOException {
+ term.length = firstTerm.length;
+ System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
+ }
+
+ // read term at offset, delta encoded from first term
+ private void readTerm(int offset) throws IOException {
+ int start = input.readByte() & 0xFF;
+ System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
+ int suffix = offsets[offset] - offsets[offset-1] - 1;
+ input.readBytes(term.bytes, start, suffix);
+ term.length = start + suffix;
+ }
+
+ @Override
+ public BytesRef next() throws IOException {
+ currentOrd++;
+ if (currentOrd >= numValues) {
+ return null;
+ } else {
+ int offset = (int) (currentOrd & INTERVAL_MASK);
+ if (offset == 0) {
+ // switch to next block
+ readHeader();
+ readFirstTerm();
+ } else {
+ readTerm(offset);
+ }
+ return term;
+ }
+ }
+
+ // binary search reverse index to find smaller
+ // range of blocks to search
+ long binarySearchIndex(BytesRef text) throws IOException {
+ long low = 0;
+ long high = numReverseIndexValues - 1;
+ while (low <= high) {
+ long mid = (low + high) >>> 1;
+ reverseTerms.fill(scratch, reverseAddresses.get(mid));
+ int cmp = scratch.compareTo(text);
+
+ if (cmp < 0) {
+ low = mid + 1;
+ } else if (cmp > 0) {
+ high = mid - 1;
+ } else {
+ return mid;
+ }
+ }
+ return high;
+ }
+
+ // binary search against first term in block range
+ // to find term's block
+ long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
+ while (low <= high) {
+ long mid = (low + high) >>> 1;
+ input.seek(addresses.get(mid));
+ term.length = input.readVInt();
+ input.readBytes(term.bytes, 0, term.length);
+ int cmp = term.compareTo(text);
+
+ if (cmp < 0) {
+ low = mid + 1;
+ } else if (cmp > 0) {
+ high = mid - 1;
+ } else {
+ return mid;
+ }
+ }
+ return high;
+ }
+
+ @Override
+ public SeekStatus seekCeil(BytesRef text) throws IOException {
+ // locate block: narrow to block range with index, then search blocks
+ final long block;
+ long indexPos = binarySearchIndex(text);
+ if (indexPos < 0) {
+ block = 0;
+ } else {
+ long low = indexPos << BLOCK_INTERVAL_SHIFT;
+ long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
+ block = Math.max(low, binarySearchBlock(text, low, high));
+ }
+
+ // position before block, then scan to term.
+ input.seek(addresses.get(block));
+ currentOrd = (block << INTERVAL_SHIFT) - 1;
+
+ while (next() != null) {
+ int cmp = term.compareTo(text);
+ if (cmp == 0) {
+ return SeekStatus.FOUND;
+ } else if (cmp > 0) {
+ return SeekStatus.NOT_FOUND;
+ }
+ }
+ return SeekStatus.END;
+ }
+
+ @Override
+ public void seekExact(long ord) throws IOException {
+ long block = ord >>> INTERVAL_SHIFT;
+ if (block != currentOrd >>> INTERVAL_SHIFT) {
+ // switch to different block
+ input.seek(addresses.get(block));
+ readHeader();
+ }
+
+ currentOrd = ord;
+
+ int offset = (int) (ord & INTERVAL_MASK);
+ if (offset == 0) {
+ readFirstTerm();
+ } else {
+ input.seek(currentBlockStart + offsets[offset-1]);
+ readTerm(offset);
+ }
+ }
+
+ @Override
+ public BytesRef term() throws IOException {
+ return term;
+ }
+
+ @Override
+ public long ord() throws IOException {
+ return currentOrd;
+ }
+
+ @Override
+ public int docFreq() throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long totalTermFreq() throws IOException {
+ return -1;
+ }
+
+ @Override
+ public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ }
+ }
+}
Property changes on: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
===================================================================
--- lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (revision 0)
+++ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (working copy)
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.lucene53;
+
+/*
+ * 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.
+ */
+
+import java.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 5.3 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene53 package documentation for file format details.
+ * @deprecated Only for reading old 5.3 segments
+ */
+@Deprecated
+public class Lucene53Codec extends Codec {
+ private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+ private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
+ private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
+ private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+ private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+
+ private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return Lucene53Codec.this.getPostingsFormatForField(field);
+ }
+ };
+
+ private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return Lucene53Codec.this.getDocValuesFormatForField(field);
+ }
+ };
+
+ private final StoredFieldsFormat storedFieldsFormat;
+
+ /**
+ * Instantiates a new codec.
+ */
+ public Lucene53Codec() {
+ this(Mode.BEST_SPEED);
+ }
+
+ /**
+ * Instantiates a new codec, specifying the stored fields compression
+ * mode to use.
+ * @param mode stored fields compression mode to use for newly
+ * flushed/merged segments.
+ */
+ public Lucene53Codec(Mode mode) {
+ super("Lucene53");
+ this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
+ }
+
+ @Override
+ public final StoredFieldsFormat storedFieldsFormat() {
+ return storedFieldsFormat;
+ }
+
+ @Override
+ public final TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public final PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ @Override
+ public final FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
+ public final SegmentInfoFormat segmentInfoFormat() {
+ return segmentInfosFormat;
+ }
+
+ @Override
+ public final LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
+
+ @Override
+ public final CompoundFormat compoundFormat() {
+ return compoundFormat;
+ }
+
+ /** Returns the postings format that should be used for writing
+ * new segments of <code>field</code>.
+ *
+ * The default implementation always returns "Lucene50".
+ * <p>
+ * <b>WARNING:</b> if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return defaultFormat;
+ }
+
+ /** Returns the docvalues format that should be used for writing
+ * new segments of <code>field</code>.
+ *
+ * The default implementation always returns "Lucene50".
+ * <p>
+ * <b>WARNING:</b> if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return defaultDVFormat;
+ }
+
+ @Override
+ public final DocValuesFormat docValuesFormat() {
+ return docValuesFormat;
+ }
+
+ @Override
+ public final DimensionalFormat dimensionalFormat() {
+ return DimensionalFormat.EMPTY;
+ }
+
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+ private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
+
+ private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+ @Override
+ public final NormsFormat normsFormat() {
+ return normsFormat;
+ }
+}
Property changes on: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html
===================================================================
--- lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html (revision 0)
+++ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html (working copy)
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+ <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Lucene 5.3 file format.
+</body>
+</html>
Property changes on: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
===================================================================
--- lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (revision 1710831)
+++ lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (working copy)
@@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene50.Lucene50Codec
+org.apache.lucene.codecs.lucene53.Lucene53Codec
Index: lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
===================================================================
--- lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (revision 1710831)
+++ lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (working copy)
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-
+org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat
Index: lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
===================================================================
--- lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (revision 0)
+++ lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (working copy)
@@ -0,0 +1,281 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene50DocValuesFormat
+ */
+public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+ private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
+
+ @Override
+ protected Codec getCodec() {
+ return codec;
+ }
+
+ // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+ // for now just keep them here, as we want to test this for this format.
+
+ @Slow
+ public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+ }
+ }
+
+ @Nightly
+ public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
+ }
+ }
+
+ @Slow
+ public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedVsStoredFields(atLeast(300), 1, 32766);
+ }
+ }
+
+ @Nightly
+ public void testSortedVariableLengthManyVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
+ }
+ }
+
+ @Slow
+ public void testTermsEnumFixedWidth() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+ }
+ }
+
+ @Slow
+ public void testTermsEnumVariableWidth() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
+ }
+ }
+
+ @Nightly
+ public void testTermsEnumRandomMany() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
+ }
+ }
+
+ // TODO: try to refactor this and some termsenum tests into the base class.
+ // to do this we need to fix the test class to get a DVF not a Codec so we can setup
+ // the postings format correctly.
+ private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
+ Directory dir = newFSDirectory(createTempDir());
+ IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ conf.setMergeScheduler(new SerialMergeScheduler());
+ // set to duel against a codec which has ordinals:
+ final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
+ final DocValuesFormat dv = new Lucene50DocValuesFormat();
+ conf.setCodec(new AssertingCodec() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return pf;
+ }
+
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return dv;
+ }
+ });
+ RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+
+ // index some docs
+ for (int i = 0; i < numDocs; i++) {
+ Document doc = new Document();
+ Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+ doc.add(idField);
+ final int length = TestUtil.nextInt(random(), minLength, maxLength);
+ int numValues = random().nextInt(17);
+ // create a random list of strings
+ List<String> values = new ArrayList<>();
+ for (int v = 0; v < numValues; v++) {
+ values.add(TestUtil.randomSimpleString(random(), minLength, length));
+ }
+
+ // add in any order to the indexed field
+ ArrayList<String> unordered = new ArrayList<>(values);
+ Collections.shuffle(unordered, random());
+ for (String v : values) {
+ doc.add(newStringField("indexed", v, Field.Store.NO));
+ }
+
+ // add in any order to the dv field
+ ArrayList<String> unordered2 = new ArrayList<>(values);
+ Collections.shuffle(unordered2, random());
+ for (String v : unordered2) {
+ doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
+ }
+
+ writer.addDocument(doc);
+ if (random().nextInt(31) == 0) {
+ writer.commit();
+ }
+ }
+
+ // delete some docs
+ int numDeletions = random().nextInt(numDocs/10);
+ for (int i = 0; i < numDeletions; i++) {
+ int id = random().nextInt(numDocs);
+ writer.deleteDocuments(new Term("id", Integer.toString(id)));
+ }
+
+ // compare per-segment
+ DirectoryReader ir = writer.getReader();
+ for (LeafReaderContext context : ir.leaves()) {
+ LeafReader r = context.reader();
+ Terms terms = r.terms("indexed");
+ if (terms != null) {
+ SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
+ assertEquals(terms.size(), ssdv.getValueCount());
+ TermsEnum expected = terms.iterator();
+ TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+ assertEquals(terms.size(), expected, actual);
+
+ doTestSortedSetEnumAdvanceIndependently(ssdv);
+ }
+ }
+ ir.close();
+
+ writer.forceMerge(1);
+
+ // now compare again after the merge
+ ir = writer.getReader();
+ LeafReader ar = getOnlySegmentReader(ir);
+ Terms terms = ar.terms("indexed");
+ if (terms != null) {
+ assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
+ TermsEnum expected = terms.iterator();
+ TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
+ assertEquals(terms.size(), expected, actual);
+ }
+ ir.close();
+
+ writer.close();
+ dir.close();
+ }
+
+ private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
+ BytesRef ref;
+
+ // sequential next() through all terms
+ while ((ref = expected.next()) != null) {
+ assertEquals(ref, actual.next());
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+ assertNull(actual.next());
+
+ // sequential seekExact(ord) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ actual.seekExact(i);
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // sequential seekExact(BytesRef) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ assertTrue(actual.seekExact(expected.term()));
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // sequential seekCeil(BytesRef) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekExact(ord)
+ for (long i = 0; i < numOrds; i++) {
+ long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+ expected.seekExact(randomOrd);
+ actual.seekExact(randomOrd);
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekExact(BytesRef)
+ for (long i = 0; i < numOrds; i++) {
+ long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+ expected.seekExact(randomOrd);
+ actual.seekExact(expected.term());
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekCeil(BytesRef)
+ for (long i = 0; i < numOrds; i++) {
+ BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
+ SeekStatus expectedStatus = expected.seekCeil(target);
+ assertEquals(expectedStatus, actual.seekCeil(target));
+ if (expectedStatus != SeekStatus.END) {
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+ }
+ }
+}
Property changes on: lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
===================================================================
--- lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (revision 1710831)
+++ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (working copy)
@@ -29,7 +29,7 @@
import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
@@ -139,7 +139,7 @@
if (defaultCodec == null && postingsFormat != null) {
try {
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
- iwConf.setCodec(new Lucene53Codec() {
+ iwConf.setCodec(new Lucene54Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormatChosen;
Index: lucene/core/src/java/org/apache/lucene/codecs/Codec.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/Codec.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/Codec.java (working copy)
@@ -57,7 +57,7 @@
}
// TODO: should we use this, or maybe a system property is better?
- static Codec defaultCodec = LOADER.lookup("Lucene53");
+ static Codec defaultCodec = LOADER.lookup("Lucene54");
}
private final String name;
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java (working copy)
@@ -1,658 +0,0 @@
-package org.apache.lucene.codecs.lucene50;
-
-/*
- * 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.
- */
-
-import java.io.Closeable; // javadocs
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LongsRef;
-import org.apache.lucene.util.MathUtil;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
-import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.packed.DirectWriter;
-import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
-import org.apache.lucene.util.packed.PackedInts;
-
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
-
-/** writer for {@link Lucene50DocValuesFormat} */
-class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
-
- IndexOutput data, meta;
- final int maxDoc;
-
- /** expert: Creates a new writer */
- public Lucene50DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
- 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, Lucene50DocValuesFormat.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, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
- maxDoc = state.segmentInfo.maxDoc();
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeWhileHandlingException(this);
- }
- }
- }
-
- @Override
- public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
- addNumericField(field, values, true);
- }
-
- void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
- long count = 0;
- long minValue = Long.MAX_VALUE;
- long maxValue = Long.MIN_VALUE;
- long gcd = 0;
- long missingCount = 0;
- long zeroCount = 0;
- // TODO: more efficient?
- HashSet<Long> uniqueValues = null;
- if (optimizeStorage) {
- uniqueValues = new HashSet<>();
-
- for (Number nv : values) {
- final long v;
- if (nv == null) {
- v = 0;
- missingCount++;
- zeroCount++;
- } else {
- v = nv.longValue();
- if (v == 0) {
- zeroCount++;
- }
- }
-
- 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);
- }
- }
-
- minValue = Math.min(minValue, v);
- maxValue = Math.max(maxValue, v);
-
- if (uniqueValues != null) {
- if (uniqueValues.add(v)) {
- if (uniqueValues.size() > 256) {
- uniqueValues = null;
- }
- }
- }
-
- ++count;
- }
- } else {
- for (Number nv : values) {
- long v = nv.longValue();
- minValue = Math.min(minValue, v);
- maxValue = Math.max(maxValue, v);
- ++count;
- }
- }
-
- final long delta = maxValue - minValue;
- final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
- final int tableBitsRequired = uniqueValues == null
- ? Integer.MAX_VALUE
- : DirectWriter.bitsRequired(uniqueValues.size() - 1);
-
- final int format;
- if (uniqueValues != null
- && count <= Integer.MAX_VALUE
- && (uniqueValues.size() == 1
- || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
- // either one unique value C or two unique values: "missing" and C
- format = CONST_COMPRESSED;
- } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
- format = TABLE_COMPRESSED;
- } else if (gcd != 0 && gcd != 1) {
- final long gcdDelta = (maxValue - minValue) / gcd;
- final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
- format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
- } else {
- format = DELTA_COMPRESSED;
- }
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
- meta.writeVInt(format);
- if (missingCount == 0) {
- meta.writeLong(ALL_LIVE);
- } else if (missingCount == count) {
- meta.writeLong(ALL_MISSING);
- } else {
- meta.writeLong(data.getFilePointer());
- writeMissingBitset(values);
- }
- meta.writeLong(data.getFilePointer());
- meta.writeVLong(count);
-
- switch (format) {
- case CONST_COMPRESSED:
- // write the constant (nonzero value in the n=2 case, singleton value otherwise)
- meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
- break;
- case GCD_COMPRESSED:
- meta.writeLong(minValue);
- meta.writeLong(gcd);
- final long maxDelta = (maxValue - minValue) / gcd;
- final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
- meta.writeVInt(bits);
- final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
- for (Number nv : values) {
- long value = nv == null ? 0 : nv.longValue();
- quotientWriter.add((value - minValue) / gcd);
- }
- quotientWriter.finish();
- break;
- case DELTA_COMPRESSED:
- final long minDelta = delta < 0 ? 0 : minValue;
- meta.writeLong(minDelta);
- meta.writeVInt(deltaBitsRequired);
- final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
- for (Number nv : values) {
- long v = nv == null ? 0 : nv.longValue();
- writer.add(v - minDelta);
- }
- writer.finish();
- break;
- case TABLE_COMPRESSED:
- final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
- Arrays.sort(decode);
- final HashMap<Long,Integer> encode = new HashMap<>();
- meta.writeVInt(decode.length);
- for (int i = 0; i < decode.length; i++) {
- meta.writeLong(decode[i]);
- encode.put(decode[i], i);
- }
- meta.writeVInt(tableBitsRequired);
- final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
- for (Number nv : values) {
- ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
- }
- ordsWriter.finish();
- break;
- default:
- throw new AssertionError();
- }
- meta.writeLong(data.getFilePointer());
- }
-
- // 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 {
- byte bits = 0;
- int count = 0;
- for (Object v : values) {
- if (count == 8) {
- data.writeByte(bits);
- count = 0;
- bits = 0;
- }
- if (v != null) {
- bits |= 1 << (count & 7);
- }
- count++;
- }
- if (count > 0) {
- data.writeByte(bits);
- }
- }
-
- @Override
- public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
- // write the byte[] data
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.BINARY);
- int minLength = Integer.MAX_VALUE;
- int maxLength = Integer.MIN_VALUE;
- final long startFP = data.getFilePointer();
- long count = 0;
- long missingCount = 0;
- for(BytesRef v : values) {
- final int length;
- if (v == null) {
- length = 0;
- missingCount++;
- } else {
- length = v.length;
- }
- minLength = Math.min(minLength, length);
- maxLength = Math.max(maxLength, length);
- if (v != null) {
- data.writeBytes(v.bytes, v.offset, v.length);
- }
- count++;
- }
- meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
- if (missingCount == 0) {
- meta.writeLong(ALL_LIVE);
- } else if (missingCount == count) {
- meta.writeLong(ALL_MISSING);
- } else {
- meta.writeLong(data.getFilePointer());
- writeMissingBitset(values);
- }
- meta.writeVInt(minLength);
- meta.writeVInt(maxLength);
- meta.writeVLong(count);
- meta.writeLong(startFP);
-
- // 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.writeLong(data.getFilePointer());
- meta.writeVInt(PackedInts.VERSION_CURRENT);
- meta.writeVInt(MONOTONIC_BLOCK_SIZE);
-
- final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
- long addr = 0;
- writer.add(addr);
- for (BytesRef v : values) {
- if (v != null) {
- addr += v.length;
- }
- writer.add(addr);
- }
- writer.finish();
- }
- }
-
- /** expert: writes a value dictionary for a sorted/sortedset field */
- private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
- // first check if it's a "fixed-length" terms dict
- int minLength = Integer.MAX_VALUE;
- int maxLength = Integer.MIN_VALUE;
- long numValues = 0;
- for (BytesRef v : values) {
- minLength = Math.min(minLength, v.length);
- maxLength = Math.max(maxLength, v.length);
- numValues++;
- }
- if (minLength == maxLength) {
- // no index needed: direct addressing by mult
- addBinaryField(field, values);
- } else if (numValues < REVERSE_INTERVAL_COUNT) {
- // low cardinality: waste a few KB of ram, but can't really use fancy index etc
- addBinaryField(field, values);
- } else {
- assert numValues > 0; // we don't have to handle the empty case
- // header
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.BINARY);
- meta.writeVInt(BINARY_PREFIX_COMPRESSED);
- meta.writeLong(-1L);
- // now write the bytes: sharing prefixes within a block
- final long startFP = data.getFilePointer();
- // currently, we have to store the delta from expected for every 1/nth term
- // we could avoid this, but it's not much and less overall RAM than the previous approach!
- RAMOutputStream addressBuffer = new RAMOutputStream();
- MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
- // buffers up 16 terms
- RAMOutputStream bytesBuffer = new RAMOutputStream();
- // buffers up block header
- RAMOutputStream headerBuffer = new RAMOutputStream();
- BytesRefBuilder lastTerm = new BytesRefBuilder();
- lastTerm.grow(maxLength);
- long count = 0;
- int suffixDeltas[] = new int[INTERVAL_COUNT];
- for (BytesRef v : values) {
- int termPosition = (int) (count & INTERVAL_MASK);
- if (termPosition == 0) {
- termAddresses.add(data.getFilePointer() - startFP);
- // abs-encode first term
- headerBuffer.writeVInt(v.length);
- headerBuffer.writeBytes(v.bytes, v.offset, v.length);
- lastTerm.copyBytes(v);
- } else {
- // prefix-code: we only share at most 255 characters, to encode the length as a single
- // byte and have random access. Larger terms just get less compression.
- int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
- bytesBuffer.writeByte((byte) sharedPrefix);
- bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
- // we can encode one smaller, because terms are unique.
- suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
- }
-
- count++;
- // flush block
- if ((count & INTERVAL_MASK) == 0) {
- flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
- }
- }
- // flush trailing crap
- int leftover = (int) (count & INTERVAL_MASK);
- if (leftover > 0) {
- Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
- flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
- }
- final long indexStartFP = data.getFilePointer();
- // write addresses of indexed terms
- termAddresses.finish();
- addressBuffer.writeTo(data);
- addressBuffer = null;
- termAddresses = null;
- meta.writeVInt(minLength);
- meta.writeVInt(maxLength);
- meta.writeVLong(count);
- meta.writeLong(startFP);
- meta.writeLong(indexStartFP);
- meta.writeVInt(PackedInts.VERSION_CURRENT);
- meta.writeVInt(MONOTONIC_BLOCK_SIZE);
- addReverseTermIndex(field, values, maxLength);
- }
- }
-
- // writes term dictionary "block"
- // first term is absolute encoded as vint length + bytes.
- // lengths of subsequent N terms are encoded as either N bytes or N shorts.
- // in the double-byte case, the first byte is indicated with -1.
- // subsequent terms are encoded as byte suffixLength + bytes.
- private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
- boolean twoByte = false;
- for (int i = 1; i < suffixDeltas.length; i++) {
- if (suffixDeltas[i] > 254) {
- twoByte = true;
- }
- }
- if (twoByte) {
- headerBuffer.writeByte((byte)255);
- for (int i = 1; i < suffixDeltas.length; i++) {
- headerBuffer.writeShort((short) suffixDeltas[i]);
- }
- } else {
- for (int i = 1; i < suffixDeltas.length; i++) {
- headerBuffer.writeByte((byte) suffixDeltas[i]);
- }
- }
- headerBuffer.writeTo(data);
- headerBuffer.reset();
- bytesBuffer.writeTo(data);
- bytesBuffer.reset();
- }
-
- // writes reverse term index: used for binary searching a term into a range of 64 blocks
- // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
- // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
- private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
- long count = 0;
- BytesRefBuilder priorTerm = new BytesRefBuilder();
- priorTerm.grow(maxLength);
- BytesRef indexTerm = new BytesRef();
- long startFP = data.getFilePointer();
- PagedBytes pagedBytes = new PagedBytes(15);
- MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
-
- for (BytesRef b : values) {
- int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
- if (termPosition == 0) {
- int len = StringHelper.sortKeyLength(priorTerm.get(), b);
- indexTerm.bytes = b.bytes;
- indexTerm.offset = b.offset;
- indexTerm.length = len;
- addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
- } else if (termPosition == REVERSE_INTERVAL_MASK) {
- priorTerm.copyBytes(b);
- }
- count++;
- }
- addresses.finish();
- long numBytes = pagedBytes.getPointer();
- pagedBytes.freeze(true);
- PagedBytesDataInput in = pagedBytes.getDataInput();
- meta.writeLong(startFP);
- data.writeVLong(numBytes);
- data.copyBytes(in, numBytes);
- }
-
- @Override
- public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.SORTED);
- addTermsDict(field, values);
- addNumericField(field, docToOrd, false);
- }
-
- @Override
- public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.SORTED_NUMERIC);
- if (isSingleValued(docToValueCount)) {
- meta.writeVInt(SORTED_SINGLE_VALUED);
- // The field is single-valued, we can encode it as NUMERIC
- addNumericField(field, singletonView(docToValueCount, values, null));
- } else {
- final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
- if (uniqueValueSets != null) {
- meta.writeVInt(SORTED_SET_TABLE);
-
- // write the set_id -> values mapping
- writeDictionary(uniqueValueSets);
-
- // write the doc -> set_id as a numeric field
- addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), false);
- } else {
- meta.writeVInt(SORTED_WITH_ADDRESSES);
- // write the stream of values as a numeric field
- addNumericField(field, values, true);
- // write the doc -> ord count as a absolute index to the stream
- addAddresses(field, docToValueCount);
- }
- }
- }
-
- @Override
- public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.SORTED_SET);
-
- if (isSingleValued(docToOrdCount)) {
- meta.writeVInt(SORTED_SINGLE_VALUED);
- // The field is single-valued, we can encode it as SORTED
- addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
- } else {
- final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
- if (uniqueValueSets != null) {
- meta.writeVInt(SORTED_SET_TABLE);
-
- // write the set_id -> ords mapping
- writeDictionary(uniqueValueSets);
-
- // write the ord -> byte[] as a binary field
- addTermsDict(field, values);
-
- // write the doc -> set_id as a numeric field
- addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), false);
- } else {
- meta.writeVInt(SORTED_WITH_ADDRESSES);
-
- // write the ord -> byte[] as a binary field
- addTermsDict(field, values);
-
- // write the stream of ords as a numeric field
- // NOTE: we could return an iterator that delta-encodes these within a doc
- addNumericField(field, ords, false);
-
- // write the doc -> ord count as a absolute index to the stream
- addAddresses(field, docToOrdCount);
- }
- }
- }
-
- private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
- Set<LongsRef> uniqueValueSet = new HashSet<>();
- LongsRef docValues = new LongsRef(256);
-
- Iterator<Number> valueCountIterator = docToValueCount.iterator();
- Iterator<Number> valueIterator = values.iterator();
- int totalDictSize = 0;
- while (valueCountIterator.hasNext()) {
- docValues.length = valueCountIterator.next().intValue();
- if (docValues.length > 256) {
- return null;
- }
- for (int i = 0; i < docValues.length; ++i) {
- docValues.longs[i] = valueIterator.next().longValue();
- }
- if (uniqueValueSet.contains(docValues)) {
- continue;
- }
- totalDictSize += docValues.length;
- if (totalDictSize > 256) {
- return null;
- }
- uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
- }
- assert valueIterator.hasNext() == false;
- return new TreeSet<>(uniqueValueSet);
- }
-
- private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
- int lengthSum = 0;
- for (LongsRef longs : uniqueValueSets) {
- lengthSum += longs.length;
- }
-
- meta.writeInt(lengthSum);
- for (LongsRef valueSet : uniqueValueSets) {
- for (int i = 0; i < valueSet.length; ++i) {
- meta.writeLong(valueSet.longs[valueSet.offset + i]);
- }
- }
-
- meta.writeInt(uniqueValueSets.size());
- for (LongsRef valueSet : uniqueValueSets) {
- meta.writeInt(valueSet.length);
- }
- }
-
- private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
- final Map<LongsRef, Integer> setIds = new HashMap<>();
- int i = 0;
- for (LongsRef set : uniqueValueSets) {
- setIds.put(set, i++);
- }
- assert i == uniqueValueSets.size();
-
- return new Iterable<Number>() {
-
- @Override
- public Iterator<Number> iterator() {
- final Iterator<Number> valueCountIterator = docToValueCount.iterator();
- final Iterator<Number> valueIterator = values.iterator();
- final LongsRef docValues = new LongsRef(256);
- return new Iterator<Number>() {
-
- @Override
- public boolean hasNext() {
- return valueCountIterator.hasNext();
- }
-
- @Override
- public Number next() {
- docValues.length = valueCountIterator.next().intValue();
- for (int i = 0; i < docValues.length; ++i) {
- docValues.longs[i] = valueIterator.next().longValue();
- }
- final Integer id = setIds.get(docValues);
- assert id != null;
- return id;
- }
-
- };
-
- }
- };
- }
-
- // writes addressing information as MONOTONIC_COMPRESSED integer
- private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
- meta.writeVInt(field.number);
- meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
- meta.writeVInt(MONOTONIC_COMPRESSED);
- meta.writeLong(-1L);
- meta.writeLong(data.getFilePointer());
- meta.writeVLong(maxDoc);
- meta.writeVInt(PackedInts.VERSION_CURRENT);
- meta.writeVInt(MONOTONIC_BLOCK_SIZE);
-
- final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
- long addr = 0;
- writer.add(addr);
- for (Number v : values) {
- addr += v.longValue();
- writer.add(addr);
- }
- writer.finish();
- meta.writeLong(data.getFilePointer());
- }
-
- @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); // write checksum
- }
- success = true;
- } finally {
- if (success) {
- IOUtils.close(data, meta);
- } else {
- IOUtils.closeWhileHandlingException(data, meta);
- }
- meta = data = null;
- }
- }
-}
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (working copy)
@@ -1,270 +0,0 @@
-package org.apache.lucene.codecs.lucene50;
-
-/*
- * 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.
- */
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.packed.DirectWriter;
-import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
-
-/**
- * Lucene 5.0 DocValues format.
- * <p>
- * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
- * <p>
- * {@link DocValuesType#NUMERIC NUMERIC}:
- * <ul>
- * <li>Delta-compressed: per-document integers written as deltas from the minimum value,
- * compressed with bitpacking. For more information, see {@link DirectWriter}.
- * <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
- * when there are unused "gaps" in the range of values used (such as {@link SmallFloat}),
- * a lookup table is written instead. Each per-document entry is instead the ordinal
- * to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}).
- * <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
- * common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
- * <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
- * as blocks of bitpacked integers, encoding the deviation from the expected delta.
- * <li>Const-compressed: when there is only one possible non-missing value, only the missing
- * bitset is encoded.
- * </ul>
- * <p>
- * {@link DocValuesType#BINARY BINARY}:
- * <ul>
- * <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
- * Each document's value can be addressed directly with multiplication ({@code docID * length}).
- * <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
- * for each document. The addresses are written as Monotonic-compressed numerics.
- * <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
- * completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
- * numerics. A reverse lookup index is written from a portion of every 1024th term.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED SORTED}:
- * <ul>
- * <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary,
- * along with the per-document ordinals written using one of the numeric strategies above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_SET SORTED_SET}:
- * <ul>
- * <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
- * <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
- * an id, a lookup table is written and the mapping from document to set id is written using the
- * numeric strategies above.
- * <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary,
- * an ordinal list and per-document index into this list are written using the numeric strategies
- * above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
- * <ul>
- * <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
- * <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
- * an id, a lookup table is written and the mapping from document to set id is written using the
- * numeric strategies above.
- * <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
- * strategies above.
- * </ul>
- * <p>
- * Files:
- * <ol>
- * <li><tt>.dvd</tt>: DocValues data</li>
- * <li><tt>.dvm</tt>: DocValues metadata</li>
- * </ol>
- * <ol>
- * <li><a name="dvm"></a>
- * <p>The DocValues metadata or .dvm file.</p>
- * <p>For DocValues field, this stores metadata, such as the offset into the
- * DocValues data (.dvd)</p>
- * <p>DocValues metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- * <ul>
- * <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
- * <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
- * <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD,BitsPerValue</li>
- * <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
- * <li>DeltaNumericEntry --&gt; NumericHeader,MinValue,BitsPerValue</li>
- * <li>MonotonicNumericEntry --&gt; NumericHeader,PackedVersion,BlockSize</li>
- * <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
- * <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
- * <li>FixedBinaryEntry --&gt; BinaryHeader</li>
- * <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
- * <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
- * <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
- * <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
- * <li>SortedSetEntry --&gt; SingleSortedSetEntry | AddressesSortedSetEntry | TableSortedSetEntry</li>
- * <li>SingleSortedSetEntry --&gt; SetHeader,SortedEntry</li>
- * <li>AddressesSortedSetEntry --&gt; SetHeader,BinaryEntry,NumericEntry,NumericEntry</li>
- * <li>TableSortedSetEntry --&gt; SetHeader,TotalTableLength,{@link DataOutput#writeLong Int64}<sup>TotalTableLength</sup>,TableSize,{@link DataOutput#writeInt Int32}<sup>TableSize</sup>,BinaryEntry,NumericEntry</li>
- * <li>SetHeader --&gt; FieldNumber,EntryType,SetType</li>
- * <li>SortedNumericEntry --&gt; SingleSortedNumericEntry | AddressesSortedNumericEntry | TableSortedNumericEntry</li>
- * <li>SingleNumericEntry --&gt; SetHeader,NumericEntry</li>
- * <li>AddressesSortedNumericEntry --&gt; SetHeader,NumericEntry,NumericEntry</li>
- * <li>TableSortedNumericEntry --&gt; SetHeader,TotalTableLength,{@link DataOutput#writeLong Int64}<sup>TotalTableLength</sup>,TableSize,{@link DataOutput#writeInt Int32}<sup>TableSize</sup>,NumericEntry</li>
- * <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- * <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
- * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- * <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --&gt; {@link DataOutput#writeLong Int64}</li>
- * <li>TableSize,BitsPerValue,TotalTableLength --&gt; {@link DataOutput#writeVInt vInt}</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
- * and an ordinary NumericEntry for the document-to-ord metadata.</p>
- * <p>FieldNumber of -1 indicates the end of metadata.</p>
- * <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
- * <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
- * <p>EndOffset is the pointer to the end of the data in the DocValues data (.dvd)</p>
- * <p>NumericType indicates how Numeric values will be compressed:
- * <ul>
- * <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
- * from the minimum value within the block.
- * <li>1 --&gt; gcd-compressed. When all integers share a common divisor, only quotients are stored
- * using blocks of delta-encoded ints.
- * <li>2 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
- * a lookup table of unique values is written, followed by the ordinal for each document.
- * <li>3 --&gt; monotonic-compressed. Used to implement addressing for BINARY, SORTED_SET, SORTED_NUMERIC.
- * <li>4 --&gt; const-compressed. Used when all non-missing values are the same.
- * </ul>
- * <p>BinaryType indicates how Binary values will be stored:
- * <ul>
- * <li>0 --&gt; fixed-width. All values have the same length, addressing by multiplication.
- * <li>1 --&gt; variable-width. An address for each value is stored.
- * <li>2 --&gt; prefix-compressed. An address to the start of every interval'th value is stored.
- * </ul>
- * <p>SetType indicates how SortedSet and SortedNumeric values will be stored:
- * <ul>
- * <li>0 --&gt; with addresses. There are two numeric entries: a first one from document to start
- * offset, and a second one from offset to ord/value.
- * <li>1 --&gt; single-valued. Used when all documents have at most one value and is encoded like
- * a regular Sorted/Numeric entry.
- * <li>2 --&gt; table-encoded. A lookup table of unique sets of values is written, followed by a
- * numeric entry that maps each document to an ordinal in this table.
- * </ul>
- * <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
- * If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
- * Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
- * is written for the addresses.
- * <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
- * If it's -1, then there are no missing values. If it's -2, all values are missing.
- * <li><a name="dvd"></a>
- * <p>The DocValues data or .dvd file.</p>
- * <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
- * <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
- * <ul>
- * <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
- * <li>BinaryData --&gt; {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
- * <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
- * <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --&gt; {@link DirectWriter PackedInts}</li>
- * <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * </ol>
- * @lucene.experimental
- */
-public final class Lucene50DocValuesFormat extends DocValuesFormat {
-
- /** Sole Constructor */
- public Lucene50DocValuesFormat() {
- super("Lucene50");
- }
-
- @Override
- public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
- }
-
- @Override
- public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
- return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
- }
-
- static final String DATA_CODEC = "Lucene50DocValuesData";
- static final String DATA_EXTENSION = "dvd";
- static final String META_CODEC = "Lucene50DocValuesMetadata";
- static final String META_EXTENSION = "dvm";
- static final int VERSION_START = 0;
- static final int VERSION_SORTEDSET_TABLE = 1;
- static final int VERSION_CURRENT = VERSION_SORTEDSET_TABLE;
-
- // indicates docvalues type
- static final byte NUMERIC = 0;
- static final byte BINARY = 1;
- static final byte SORTED = 2;
- static final byte SORTED_SET = 3;
- static final byte SORTED_NUMERIC = 4;
-
- // address terms in blocks of 16 terms
- static final int INTERVAL_SHIFT = 4;
- static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
- static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
-
- // build reverse index from every 1024th term
- static final int REVERSE_INTERVAL_SHIFT = 10;
- static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
- static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
-
- // for conversion from reverse index to block
- static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
- static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
- static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
-
- /** Compressed using packed blocks of ints. */
- static final int DELTA_COMPRESSED = 0;
- /** Compressed by computing the GCD. */
- static final int GCD_COMPRESSED = 1;
- /** Compressed by giving IDs to unique values. */
- static final int TABLE_COMPRESSED = 2;
- /** Compressed with monotonically increasing values */
- static final int MONOTONIC_COMPRESSED = 3;
- /** Compressed with constant value (uses only missing bitset) */
- static final int CONST_COMPRESSED = 4;
-
- /** Uncompressed binary, written directly (fixed length). */
- static final int BINARY_FIXED_UNCOMPRESSED = 0;
- /** Uncompressed binary, written directly (variable length). */
- static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
- /** Compressed binary with shared prefixes */
- static final int BINARY_PREFIX_COMPRESSED = 2;
-
- /** Standard storage for sorted set values with 1 level of indirection:
- * {@code docId -> address -> ord}. */
- static final int SORTED_WITH_ADDRESSES = 0;
- /** Single-valued sorted set values, encoded as sorted values, so no level
- * of indirection: {@code docId -> ord}. */
- static final int SORTED_SINGLE_VALUED = 1;
- /** Compressed giving IDs to unique sets of values:
- * {@code docId -> setId -> ords} */
- static final int SORTED_SET_TABLE = 2;
-
- /** placeholder for missing offset that means there are no missing values */
- static final int ALL_LIVE = -1;
- /** placeholder for missing offset that means all values are missing */
- static final int ALL_MISSING = -2;
-
- // addressing uses 16k blocks
- static final int MONOTONIC_BLOCK_SIZE = 16384;
-}
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (working copy)
@@ -1,1299 +0,0 @@
-package org.apache.lucene.codecs.lucene50;
-
-/*
- * 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.
- */
-
-import java.io.Closeable;
-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.BinaryDocValues;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomAccessOrds;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-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.IOUtils;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.DirectReader;
-import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
-
-/** reader for {@link Lucene50DocValuesFormat} */
-class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
- private final Map<String,NumericEntry> numerics = new HashMap<>();
- private final Map<String,BinaryEntry> binaries = new HashMap<>();
- private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
- private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
- private final Map<String,NumericEntry> ords = new HashMap<>();
- private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
- private final int numFields;
- private final AtomicLong ramBytesUsed;
- private final IndexInput data;
- private final int maxDoc;
-
- // memory-resident structures
- private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
- private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
- private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
-
- private final boolean merging;
-
- // clone for merge: when merging we don't do any instances.put()s
- Lucene50DocValuesProducer(Lucene50DocValuesProducer original) throws IOException {
- assert Thread.holdsLock(original);
- numerics.putAll(original.numerics);
- binaries.putAll(original.binaries);
- sortedSets.putAll(original.sortedSets);
- sortedNumerics.putAll(original.sortedNumerics);
- ords.putAll(original.ords);
- ordIndexes.putAll(original.ordIndexes);
- numFields = original.numFields;
- ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
- data = original.data.clone();
- maxDoc = original.maxDoc;
-
- addressInstances.putAll(original.addressInstances);
- ordIndexInstances.putAll(original.ordIndexInstances);
- reverseIndexInstances.putAll(original.reverseIndexInstances);
- merging = true;
- }
-
- /** expert: instantiates a new reader */
- Lucene50DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
- String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
- this.maxDoc = state.segmentInfo.maxDoc();
- merging = false;
- ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
-
- int version = -1;
- int numFields = -1;
-
- // read in the entries from the metadata file.
- try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
- Throwable priorE = null;
- try {
- version = CodecUtil.checkIndexHeader(in, metaCodec,
- Lucene50DocValuesFormat.VERSION_START,
- Lucene50DocValuesFormat.VERSION_CURRENT,
- state.segmentInfo.getId(),
- state.segmentSuffix);
- numFields = readFields(in, state.fieldInfos);
- } catch (Throwable exception) {
- priorE = exception;
- } finally {
- CodecUtil.checkFooter(in, priorE);
- }
- }
-
- this.numFields = numFields;
- String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
- this.data = state.directory.openInput(dataName, state.context);
- boolean success = false;
- try {
- final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
- Lucene50DocValuesFormat.VERSION_START,
- Lucene50DocValuesFormat.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 void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
- // sorted = binary + numeric
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
- throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
- }
- BinaryEntry b = readBinaryEntry(meta);
- binaries.put(info.name, b);
-
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry n = readNumericEntry(meta);
- ords.put(info.name, n);
- }
-
- private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
- // sortedset = binary + numeric (addresses) + ordIndex
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- BinaryEntry b = readBinaryEntry(meta);
- binaries.put(info.name, b);
-
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry n1 = readNumericEntry(meta);
- ords.put(info.name, n1);
-
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry n2 = readNumericEntry(meta);
- ordIndexes.put(info.name, n2);
- }
-
- private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
- // sortedset table = binary + ordset table + ordset index
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
-
- BinaryEntry b = readBinaryEntry(meta);
- binaries.put(info.name, b);
-
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry n = readNumericEntry(meta);
- ords.put(info.name, n);
- }
-
- private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
- int numFields = 0;
- int fieldNumber = meta.readVInt();
- while (fieldNumber != -1) {
- numFields++;
- FieldInfo info = infos.fieldInfo(fieldNumber);
- if (info == null) {
- // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
- throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
- }
- byte type = meta.readByte();
- if (type == Lucene50DocValuesFormat.NUMERIC) {
- numerics.put(info.name, readNumericEntry(meta));
- } else if (type == Lucene50DocValuesFormat.BINARY) {
- BinaryEntry b = readBinaryEntry(meta);
- binaries.put(info.name, b);
- } else if (type == Lucene50DocValuesFormat.SORTED) {
- readSortedField(info, meta);
- } else if (type == Lucene50DocValuesFormat.SORTED_SET) {
- SortedSetEntry ss = readSortedSetEntry(meta);
- sortedSets.put(info.name, ss);
- if (ss.format == SORTED_WITH_ADDRESSES) {
- readSortedSetFieldWithAddresses(info, meta);
- } else if (ss.format == SORTED_SET_TABLE) {
- readSortedSetFieldWithTable(info, meta);
- } else if (ss.format == SORTED_SINGLE_VALUED) {
- if (meta.readVInt() != fieldNumber) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.SORTED) {
- throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
- }
- readSortedField(info, meta);
- } else {
- throw new AssertionError();
- }
- } else if (type == Lucene50DocValuesFormat.SORTED_NUMERIC) {
- SortedSetEntry ss = readSortedSetEntry(meta);
- sortedNumerics.put(info.name, ss);
- if (ss.format == SORTED_WITH_ADDRESSES) {
- if (meta.readVInt() != fieldNumber) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- numerics.put(info.name, readNumericEntry(meta));
- if (meta.readVInt() != fieldNumber) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry ordIndex = readNumericEntry(meta);
- ordIndexes.put(info.name, ordIndex);
- } else if (ss.format == SORTED_SET_TABLE) {
- if (meta.readVInt() != info.number) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- NumericEntry n = readNumericEntry(meta);
- ords.put(info.name, n);
- } else if (ss.format == SORTED_SINGLE_VALUED) {
- if (meta.readVInt() != fieldNumber) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
- throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
- }
- numerics.put(info.name, readNumericEntry(meta));
- } else {
- throw new AssertionError();
- }
- } else {
- throw new CorruptIndexException("invalid type: " + type, meta);
- }
- fieldNumber = meta.readVInt();
- }
- return numFields;
- }
-
- private NumericEntry readNumericEntry(IndexInput meta) throws IOException {
- NumericEntry entry = new NumericEntry();
- entry.format = meta.readVInt();
- entry.missingOffset = meta.readLong();
- entry.offset = meta.readLong();
- entry.count = meta.readVLong();
- switch(entry.format) {
- case CONST_COMPRESSED:
- entry.minValue = meta.readLong();
- if (entry.count > Integer.MAX_VALUE) {
- // currently just a limitation e.g. of bits interface and so on.
- throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
- }
- break;
- case GCD_COMPRESSED:
- entry.minValue = meta.readLong();
- entry.gcd = meta.readLong();
- entry.bitsPerValue = meta.readVInt();
- break;
- case TABLE_COMPRESSED:
- final int uniqueValues = meta.readVInt();
- if (uniqueValues > 256) {
- throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
- }
- entry.table = new long[uniqueValues];
- for (int i = 0; i < uniqueValues; ++i) {
- entry.table[i] = meta.readLong();
- }
- ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
- entry.bitsPerValue = meta.readVInt();
- break;
- case DELTA_COMPRESSED:
- entry.minValue = meta.readLong();
- entry.bitsPerValue = meta.readVInt();
- break;
- case MONOTONIC_COMPRESSED:
- entry.packedIntsVersion = meta.readVInt();
- entry.blockSize = meta.readVInt();
- break;
- default:
- throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
- }
- entry.endOffset = meta.readLong();
- return entry;
- }
-
- static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
- BinaryEntry entry = new BinaryEntry();
- entry.format = meta.readVInt();
- entry.missingOffset = meta.readLong();
- entry.minLength = meta.readVInt();
- entry.maxLength = meta.readVInt();
- entry.count = meta.readVLong();
- entry.offset = meta.readLong();
- switch(entry.format) {
- case BINARY_FIXED_UNCOMPRESSED:
- break;
- case BINARY_PREFIX_COMPRESSED:
- entry.addressesOffset = meta.readLong();
- entry.packedIntsVersion = meta.readVInt();
- entry.blockSize = meta.readVInt();
- entry.reverseIndexOffset = meta.readLong();
- break;
- case BINARY_VARIABLE_UNCOMPRESSED:
- entry.addressesOffset = meta.readLong();
- entry.packedIntsVersion = meta.readVInt();
- entry.blockSize = meta.readVInt();
- break;
- default:
- throw new CorruptIndexException("Unknown format: " + entry.format, meta);
- }
- return entry;
- }
-
- SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
- SortedSetEntry entry = new SortedSetEntry();
- entry.format = meta.readVInt();
- if (entry.format == SORTED_SET_TABLE) {
- final int totalTableLength = meta.readInt();
- if (totalTableLength > 256) {
- throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
- }
- entry.table = new long[totalTableLength];
- for (int i = 0; i < totalTableLength; ++i) {
- entry.table[i] = meta.readLong();
- }
- ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
- final int tableSize = meta.readInt();
- if (tableSize > totalTableLength + 1) { // +1 because of the empty set
- throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
- }
- entry.tableOffsets = new int[tableSize + 1];
- for (int i = 1; i < entry.tableOffsets.length; ++i) {
- entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
- }
- ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
- } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
- throw new CorruptIndexException("Unknown format: " + entry.format, meta);
- }
- return entry;
- }
-
- @Override
- public NumericDocValues getNumeric(FieldInfo field) throws IOException {
- NumericEntry entry = numerics.get(field.name);
- return getNumeric(entry);
- }
-
- @Override
- public long ramBytesUsed() {
- return ramBytesUsed.get();
- }
-
- @Override
- public synchronized Collection<Accountable> getChildResources() {
- List<Accountable> resources = new ArrayList<>();
- resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
- resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
- resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
- return Collections.unmodifiableList(resources);
- }
-
- @Override
- public void checkIntegrity() throws IOException {
- CodecUtil.checksumEntireFile(data);
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + "(fields=" + numFields + ")";
- }
-
- LongValues getNumeric(NumericEntry entry) throws IOException {
- switch (entry.format) {
- case CONST_COMPRESSED: {
- final long constant = entry.minValue;
- final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
- return new LongValues() {
- @Override
- public long get(long index) {
- return live.get((int)index) ? constant : 0;
- }
- };
- }
- case DELTA_COMPRESSED: {
- RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
- final long delta = entry.minValue;
- final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
- return new LongValues() {
- @Override
- public long get(long id) {
- return delta + values.get(id);
- }
- };
- }
- case GCD_COMPRESSED: {
- RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
- final long min = entry.minValue;
- final long mult = entry.gcd;
- final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue);
- return new LongValues() {
- @Override
- public long get(long id) {
- return min + mult * quotientReader.get(id);
- }
- };
- }
- case TABLE_COMPRESSED: {
- RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
- final long table[] = entry.table;
- final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
- return new LongValues() {
- @Override
- public long get(long id) {
- return table[(int) ords.get(id)];
- }
- };
- }
- default:
- throw new AssertionError();
- }
- }
-
- @Override
- public BinaryDocValues getBinary(FieldInfo field) throws IOException {
- BinaryEntry bytes = binaries.get(field.name);
- switch(bytes.format) {
- case BINARY_FIXED_UNCOMPRESSED:
- return getFixedBinary(field, bytes);
- case BINARY_VARIABLE_UNCOMPRESSED:
- return getVariableBinary(field, bytes);
- case BINARY_PREFIX_COMPRESSED:
- return getCompressedBinary(field, bytes);
- default:
- throw new AssertionError();
- }
- }
-
- private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
- final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
-
- final BytesRef term = new BytesRef(bytes.maxLength);
- final byte[] buffer = term.bytes;
- final int length = term.length = bytes.maxLength;
-
- return new LongBinaryDocValues() {
- @Override
- public BytesRef get(long id) {
- try {
- data.seek(id * length);
- data.readBytes(buffer, 0, buffer.length);
- return term;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- };
- }
-
- /** returns an address instance for variable-length binary values. */
- private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
- MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
- if (addresses == null) {
- data.seek(bytes.addressesOffset);
- addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
- if (!merging) {
- addressInstances.put(field.name, addresses);
- ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
- }
- }
- return addresses;
- }
-
- private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
- final MonotonicBlockPackedReader addresses = getAddressInstance(field, bytes);
-
- final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
- final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
- final byte buffer[] = term.bytes;
-
- return new LongBinaryDocValues() {
- @Override
- public BytesRef get(long id) {
- long startAddress = addresses.get(id);
- long endAddress = addresses.get(id+1);
- int length = (int) (endAddress - startAddress);
- try {
- data.seek(startAddress);
- data.readBytes(buffer, 0, length);
- term.length = length;
- return term;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- };
- }
-
- /** returns an address instance for prefix-compressed binary values. */
- private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
- MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
- if (addresses == null) {
- data.seek(bytes.addressesOffset);
- final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
- addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
- if (!merging) {
- addressInstances.put(field.name, addresses);
- ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
- }
- }
- return addresses;
- }
-
- /** returns a reverse lookup instance for prefix-compressed binary values. */
- private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
- ReverseTermsIndex index = reverseIndexInstances.get(field.name);
- if (index == null) {
- index = new ReverseTermsIndex();
- data.seek(bytes.reverseIndexOffset);
- long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
- index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
- long dataSize = data.readVLong();
- PagedBytes pagedBytes = new PagedBytes(15);
- pagedBytes.copy(data, dataSize);
- index.terms = pagedBytes.freeze(true);
- if (!merging) {
- reverseIndexInstances.put(field.name, index);
- ramBytesUsed.addAndGet(index.ramBytesUsed());
- }
- }
- return index;
- }
-
- private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
- final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
- final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
- assert addresses.size() > 0; // we don't have to handle empty case
- IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
- return new CompressedBinaryDocValues(bytes, addresses, index, slice);
- }
-
- @Override
- public SortedDocValues getSorted(FieldInfo field) throws IOException {
- final int valueCount = (int) binaries.get(field.name).count;
- final BinaryDocValues binary = getBinary(field);
- NumericEntry entry = ords.get(field.name);
- final LongValues ordinals = getNumeric(entry);
- return new SortedDocValues() {
-
- @Override
- public int getOrd(int docID) {
- return (int) ordinals.get(docID);
- }
-
- @Override
- public BytesRef lookupOrd(int ord) {
- return binary.get(ord);
- }
-
- @Override
- public int getValueCount() {
- return valueCount;
- }
-
- @Override
- public int lookupTerm(BytesRef key) {
- if (binary instanceof CompressedBinaryDocValues) {
- return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
- } else {
- return super.lookupTerm(key);
- }
- }
-
- @Override
- public TermsEnum termsEnum() {
- if (binary instanceof CompressedBinaryDocValues) {
- return ((CompressedBinaryDocValues)binary).getTermsEnum();
- } else {
- return super.termsEnum();
- }
- }
- };
- }
-
- /** returns an address instance for sortedset ordinal lists */
- private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
- MonotonicBlockPackedReader instance = ordIndexInstances.get(field.name);
- if (instance == null) {
- data.seek(entry.offset);
- instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
- if (!merging) {
- ordIndexInstances.put(field.name, instance);
- ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
- }
- }
- return instance;
- }
-
- @Override
- public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
- SortedSetEntry ss = sortedNumerics.get(field.name);
- if (ss.format == SORTED_SINGLE_VALUED) {
- NumericEntry numericEntry = numerics.get(field.name);
- final LongValues values = getNumeric(numericEntry);
- final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
- return DocValues.singleton(values, docsWithField);
- } else if (ss.format == SORTED_WITH_ADDRESSES) {
- NumericEntry numericEntry = numerics.get(field.name);
- final LongValues values = getNumeric(numericEntry);
- final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
- return new SortedNumericDocValues() {
- long startOffset;
- long endOffset;
-
- @Override
- public void setDocument(int doc) {
- startOffset = ordIndex.get(doc);
- endOffset = ordIndex.get(doc+1L);
- }
-
- @Override
- public long valueAt(int index) {
- return values.get(startOffset + index);
- }
-
- @Override
- public int count() {
- return (int) (endOffset - startOffset);
- }
- };
- } else if (ss.format == SORTED_SET_TABLE) {
- NumericEntry entry = ords.get(field.name);
- final LongValues ordinals = getNumeric(entry);
-
- final long[] table = ss.table;
- final int[] offsets = ss.tableOffsets;
- return new SortedNumericDocValues() {
- int startOffset;
- int endOffset;
-
- @Override
- public void setDocument(int doc) {
- final int ord = (int) ordinals.get(doc);
- startOffset = offsets[ord];
- endOffset = offsets[ord + 1];
- }
-
- @Override
- public long valueAt(int index) {
- return table[startOffset + index];
- }
-
- @Override
- public int count() {
- return endOffset - startOffset;
- }
- };
- } else {
- throw new AssertionError();
- }
- }
-
- @Override
- public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
- SortedSetEntry ss = sortedSets.get(field.name);
- switch (ss.format) {
- case SORTED_SINGLE_VALUED:
- final SortedDocValues values = getSorted(field);
- return DocValues.singleton(values);
- case SORTED_WITH_ADDRESSES:
- return getSortedSetWithAddresses(field);
- case SORTED_SET_TABLE:
- return getSortedSetTable(field, ss);
- default:
- throw new AssertionError();
- }
- }
-
- private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
- final long valueCount = binaries.get(field.name).count;
- // we keep the byte[]s and list of ords on disk, these could be large
- final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
- final LongValues ordinals = getNumeric(ords.get(field.name));
- // but the addresses to the ord stream are in RAM
- final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
- return new RandomAccessOrds() {
- long startOffset;
- long offset;
- long endOffset;
-
- @Override
- public long nextOrd() {
- if (offset == endOffset) {
- return NO_MORE_ORDS;
- } else {
- long ord = ordinals.get(offset);
- offset++;
- return ord;
- }
- }
-
- @Override
- public void setDocument(int docID) {
- startOffset = offset = ordIndex.get(docID);
- endOffset = ordIndex.get(docID+1L);
- }
-
- @Override
- public BytesRef lookupOrd(long ord) {
- return binary.get(ord);
- }
-
- @Override
- public long getValueCount() {
- return valueCount;
- }
-
- @Override
- public long lookupTerm(BytesRef key) {
- if (binary instanceof CompressedBinaryDocValues) {
- return ((CompressedBinaryDocValues)binary).lookupTerm(key);
- } else {
- return super.lookupTerm(key);
- }
- }
-
- @Override
- public TermsEnum termsEnum() {
- if (binary instanceof CompressedBinaryDocValues) {
- return ((CompressedBinaryDocValues)binary).getTermsEnum();
- } else {
- return super.termsEnum();
- }
- }
-
- @Override
- public long ordAt(int index) {
- return ordinals.get(startOffset + index);
- }
-
- @Override
- public int cardinality() {
- return (int) (endOffset - startOffset);
- }
- };
- }
-
- private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
- final long valueCount = binaries.get(field.name).count;
- final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
- final LongValues ordinals = getNumeric(ords.get(field.name));
-
- final long[] table = ss.table;
- final int[] offsets = ss.tableOffsets;
-
- return new RandomAccessOrds() {
-
- int offset, startOffset, endOffset;
-
- @Override
- public void setDocument(int docID) {
- final int ord = (int) ordinals.get(docID);
- offset = startOffset = offsets[ord];
- endOffset = offsets[ord + 1];
- }
-
- @Override
- public long ordAt(int index) {
- return table[startOffset + index];
- }
-
- @Override
- public long nextOrd() {
- if (offset == endOffset) {
- return NO_MORE_ORDS;
- } else {
- return table[offset++];
- }
- }
-
- @Override
- public int cardinality() {
- return endOffset - startOffset;
- }
-
- @Override
- public BytesRef lookupOrd(long ord) {
- return binary.get(ord);
- }
-
- @Override
- public long getValueCount() {
- return valueCount;
- }
-
- @Override
- public long lookupTerm(BytesRef key) {
- if (binary instanceof CompressedBinaryDocValues) {
- return ((CompressedBinaryDocValues) binary).lookupTerm(key);
- } else {
- return super.lookupTerm(key);
- }
- }
-
- @Override
- public TermsEnum termsEnum() {
- if (binary instanceof CompressedBinaryDocValues) {
- return ((CompressedBinaryDocValues) binary).getTermsEnum();
- } else {
- return super.termsEnum();
- }
- }
-
- };
- }
-
- private Bits getLiveBits(final long offset, final int count) throws IOException {
- if (offset == ALL_MISSING) {
- return new Bits.MatchNoBits(count);
- } else if (offset == ALL_LIVE) {
- return new Bits.MatchAllBits(count);
- } else {
- int length = (int) ((count + 7L) >>> 3);
- final RandomAccessInput in = data.randomAccessSlice(offset, length);
- return new Bits() {
- @Override
- public boolean get(int index) {
- try {
- return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public int length() {
- return count;
- }
- };
- }
- }
-
- @Override
- public Bits getDocsWithField(FieldInfo field) throws IOException {
- switch(field.getDocValuesType()) {
- case SORTED_SET:
- return DocValues.docsWithValue(getSortedSet(field), maxDoc);
- case SORTED_NUMERIC:
- return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
- case SORTED:
- return DocValues.docsWithValue(getSorted(field), maxDoc);
- case BINARY:
- BinaryEntry be = binaries.get(field.name);
- return getLiveBits(be.missingOffset, maxDoc);
- case NUMERIC:
- NumericEntry ne = numerics.get(field.name);
- return getLiveBits(ne.missingOffset, maxDoc);
- default:
- throw new AssertionError();
- }
- }
-
- @Override
- public synchronized DocValuesProducer getMergeInstance() throws IOException {
- return new Lucene50DocValuesProducer(this);
- }
-
- @Override
- public void close() throws IOException {
- data.close();
- }
-
- /** metadata entry for a numeric docvalues field */
- static class NumericEntry {
- private NumericEntry() {}
- /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
- long missingOffset;
- /** offset to the actual numeric values */
- public long offset;
- /** end offset to the actual numeric values */
- public long endOffset;
- /** bits per value used to pack the numeric values */
- public int bitsPerValue;
-
- int format;
- /** packed ints version used to encode these numerics */
- public int packedIntsVersion;
- /** count of values written */
- public long count;
- /** packed ints blocksize */
- public int blockSize;
-
- long minValue;
- long gcd;
- long table[];
- }
-
- /** metadata entry for a binary docvalues field */
- static class BinaryEntry {
- private BinaryEntry() {}
- /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
- long missingOffset;
- /** offset to the actual binary values */
- long offset;
-
- int format;
- /** count of values written */
- public long count;
- int minLength;
- int maxLength;
- /** offset to the addressing data that maps a value to its slice of the byte[] */
- public long addressesOffset;
- /** offset to the reverse index */
- public long reverseIndexOffset;
- /** packed ints version used to encode addressing information */
- public int packedIntsVersion;
- /** packed ints blocksize */
- public int blockSize;
- }
-
- /** metadata entry for a sorted-set docvalues field */
- static class SortedSetEntry {
- private SortedSetEntry() {}
- int format;
-
- long[] table;
- int[] tableOffsets;
- }
-
- // internally we compose complex dv (sorted/sortedset) from other ones
- static abstract class LongBinaryDocValues extends BinaryDocValues {
- @Override
- public final BytesRef get(int docID) {
- return get((long)docID);
- }
-
- abstract BytesRef get(long id);
- }
-
- // used for reverse lookup to a small range of blocks
- static class ReverseTermsIndex implements Accountable {
- public MonotonicBlockPackedReader termAddresses;
- public PagedBytes.Reader terms;
-
- @Override
- public long ramBytesUsed() {
- return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
- }
-
- @Override
- public Collection<Accountable> getChildResources() {
- List<Accountable> resources = new ArrayList<>();
- resources.add(Accountables.namedAccountable("term bytes", terms));
- resources.add(Accountables.namedAccountable("term addresses", termAddresses));
- return Collections.unmodifiableList(resources);
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
- }
- }
-
- //in the compressed case, we add a few additional operations for
- //more efficient reverse lookup and enumeration
- static final class CompressedBinaryDocValues extends LongBinaryDocValues {
- final long numValues;
- final long numIndexValues;
- final int maxTermLength;
- final MonotonicBlockPackedReader addresses;
- final IndexInput data;
- final CompressedBinaryTermsEnum termsEnum;
- final PagedBytes.Reader reverseTerms;
- final MonotonicBlockPackedReader reverseAddresses;
- final long numReverseIndexValues;
-
- public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
- this.maxTermLength = bytes.maxLength;
- this.numValues = bytes.count;
- this.addresses = addresses;
- this.numIndexValues = addresses.size();
- this.data = data;
- this.reverseTerms = index.terms;
- this.reverseAddresses = index.termAddresses;
- this.numReverseIndexValues = reverseAddresses.size();
- this.termsEnum = getTermsEnum(data);
- }
-
- @Override
- public BytesRef get(long id) {
- try {
- termsEnum.seekExact(id);
- return termsEnum.term();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- long lookupTerm(BytesRef key) {
- try {
- switch (termsEnum.seekCeil(key)) {
- case FOUND: return termsEnum.ord();
- case NOT_FOUND: return -termsEnum.ord()-1;
- default: return -numValues-1;
- }
- } catch (IOException bogus) {
- throw new RuntimeException(bogus);
- }
- }
-
- TermsEnum getTermsEnum() {
- try {
- return getTermsEnum(data.clone());
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
- return new CompressedBinaryTermsEnum(input);
- }
-
- class CompressedBinaryTermsEnum extends TermsEnum {
- private long currentOrd = -1;
- // offset to the start of the current block
- private long currentBlockStart;
- private final IndexInput input;
- // delta from currentBlockStart to start of each term
- private final int offsets[] = new int[INTERVAL_COUNT];
- private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
-
- private final BytesRef term = new BytesRef(maxTermLength);
- private final BytesRef firstTerm = new BytesRef(maxTermLength);
- private final BytesRef scratch = new BytesRef();
-
- CompressedBinaryTermsEnum(IndexInput input) throws IOException {
- this.input = input;
- input.seek(0);
- }
-
- private void readHeader() throws IOException {
- firstTerm.length = input.readVInt();
- input.readBytes(firstTerm.bytes, 0, firstTerm.length);
- input.readBytes(buffer, 0, INTERVAL_COUNT-1);
- if (buffer[0] == -1) {
- readShortAddresses();
- } else {
- readByteAddresses();
- }
- currentBlockStart = input.getFilePointer();
- }
-
- // read single byte addresses: each is delta - 2
- // (shared prefix byte and length > 0 are both implicit)
- private void readByteAddresses() throws IOException {
- int addr = 0;
- for (int i = 1; i < offsets.length; i++) {
- addr += 2 + (buffer[i-1] & 0xFF);
- offsets[i] = addr;
- }
- }
-
- // read double byte addresses: each is delta - 2
- // (shared prefix byte and length > 0 are both implicit)
- private void readShortAddresses() throws IOException {
- input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
- int addr = 0;
- for (int i = 1; i < offsets.length; i++) {
- int x = i<<1;
- addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
- offsets[i] = addr;
- }
- }
-
- // set term to the first term
- private void readFirstTerm() throws IOException {
- term.length = firstTerm.length;
- System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
- }
-
- // read term at offset, delta encoded from first term
- private void readTerm(int offset) throws IOException {
- int start = input.readByte() & 0xFF;
- System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
- int suffix = offsets[offset] - offsets[offset-1] - 1;
- input.readBytes(term.bytes, start, suffix);
- term.length = start + suffix;
- }
-
- @Override
- public BytesRef next() throws IOException {
- currentOrd++;
- if (currentOrd >= numValues) {
- return null;
- } else {
- int offset = (int) (currentOrd & INTERVAL_MASK);
- if (offset == 0) {
- // switch to next block
- readHeader();
- readFirstTerm();
- } else {
- readTerm(offset);
- }
- return term;
- }
- }
-
- // binary search reverse index to find smaller
- // range of blocks to search
- long binarySearchIndex(BytesRef text) throws IOException {
- long low = 0;
- long high = numReverseIndexValues - 1;
- while (low <= high) {
- long mid = (low + high) >>> 1;
- reverseTerms.fill(scratch, reverseAddresses.get(mid));
- int cmp = scratch.compareTo(text);
-
- if (cmp < 0) {
- low = mid + 1;
- } else if (cmp > 0) {
- high = mid - 1;
- } else {
- return mid;
- }
- }
- return high;
- }
-
- // binary search against first term in block range
- // to find term's block
- long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
- while (low <= high) {
- long mid = (low + high) >>> 1;
- input.seek(addresses.get(mid));
- term.length = input.readVInt();
- input.readBytes(term.bytes, 0, term.length);
- int cmp = term.compareTo(text);
-
- if (cmp < 0) {
- low = mid + 1;
- } else if (cmp > 0) {
- high = mid - 1;
- } else {
- return mid;
- }
- }
- return high;
- }
-
- @Override
- public SeekStatus seekCeil(BytesRef text) throws IOException {
- // locate block: narrow to block range with index, then search blocks
- final long block;
- long indexPos = binarySearchIndex(text);
- if (indexPos < 0) {
- block = 0;
- } else {
- long low = indexPos << BLOCK_INTERVAL_SHIFT;
- long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
- block = Math.max(low, binarySearchBlock(text, low, high));
- }
-
- // position before block, then scan to term.
- input.seek(addresses.get(block));
- currentOrd = (block << INTERVAL_SHIFT) - 1;
-
- while (next() != null) {
- int cmp = term.compareTo(text);
- if (cmp == 0) {
- return SeekStatus.FOUND;
- } else if (cmp > 0) {
- return SeekStatus.NOT_FOUND;
- }
- }
- return SeekStatus.END;
- }
-
- @Override
- public void seekExact(long ord) throws IOException {
- long block = ord >>> INTERVAL_SHIFT;
- if (block != currentOrd >>> INTERVAL_SHIFT) {
- // switch to different block
- input.seek(addresses.get(block));
- readHeader();
- }
-
- currentOrd = ord;
-
- int offset = (int) (ord & INTERVAL_MASK);
- if (offset == 0) {
- readFirstTerm();
- } else {
- input.seek(currentBlockStart + offsets[offset-1]);
- readTerm(offset);
- }
- }
-
- @Override
- public BytesRef term() throws IOException {
- return term;
- }
-
- @Override
- public long ord() throws IOException {
- return currentOrd;
- }
-
- @Override
- public int docFreq() throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public long totalTermFreq() throws IOException {
- return -1;
- }
-
- @Override
- public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- }
- }
-}
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java (working copy)
@@ -53,9 +53,9 @@
* These two options can be configured like this:
* <pre class="prettyprint">
* // the default: for high performance
- * indexWriterConfig.setCodec(new Lucene53Codec(Mode.BEST_SPEED));
+ * indexWriterConfig.setCodec(new Lucene54Codec(Mode.BEST_SPEED));
* // instead for higher performance (but slower):
- * // indexWriterConfig.setCodec(new Lucene53Codec(Mode.BEST_COMPRESSION));
+ * // indexWriterConfig.setCodec(new Lucene54Codec(Mode.BEST_COMPRESSION));
* </pre>
* <p><b>File formats</b>
* <p>Stored fields are represented by two files:
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (working copy)
@@ -1,179 +0,0 @@
-package org.apache.lucene.codecs.lucene53;
-
-/*
- * 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.
- */
-
-import java.util.Objects;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.CompoundFormat;
-import org.apache.lucene.codecs.DimensionalFormat;
-import org.apache.lucene.codecs.DimensionalReader;
-import org.apache.lucene.codecs.DimensionalWriter;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FilterCodec;
-import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-
-/**
- * Implements the Lucene 5.3 index format, with configurable per-field postings
- * and docvalues formats.
- * <p>
- * If you want to reuse functionality of this codec in another codec, extend
- * {@link FilterCodec}.
- *
- * @see org.apache.lucene.codecs.lucene53 package documentation for file format details.
- * @lucene.experimental
- */
-public class Lucene53Codec extends Codec {
- private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
- private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
- private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
- private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
- private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
-
- private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- return Lucene53Codec.this.getPostingsFormatForField(field);
- }
- };
-
- private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
- @Override
- public DocValuesFormat getDocValuesFormatForField(String field) {
- return Lucene53Codec.this.getDocValuesFormatForField(field);
- }
- };
-
- private final StoredFieldsFormat storedFieldsFormat;
-
- /**
- * Instantiates a new codec.
- */
- public Lucene53Codec() {
- this(Mode.BEST_SPEED);
- }
-
- /**
- * Instantiates a new codec, specifying the stored fields compression
- * mode to use.
- * @param mode stored fields compression mode to use for newly
- * flushed/merged segments.
- */
- public Lucene53Codec(Mode mode) {
- super("Lucene53");
- this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
- }
-
- @Override
- public final StoredFieldsFormat storedFieldsFormat() {
- return storedFieldsFormat;
- }
-
- @Override
- public final TermVectorsFormat termVectorsFormat() {
- return vectorsFormat;
- }
-
- @Override
- public final PostingsFormat postingsFormat() {
- return postingsFormat;
- }
-
- @Override
- public final FieldInfosFormat fieldInfosFormat() {
- return fieldInfosFormat;
- }
-
- @Override
- public final SegmentInfoFormat segmentInfoFormat() {
- return segmentInfosFormat;
- }
-
- @Override
- public final LiveDocsFormat liveDocsFormat() {
- return liveDocsFormat;
- }
-
- @Override
- public final CompoundFormat compoundFormat() {
- return compoundFormat;
- }
-
- /** Returns the postings format that should be used for writing
- * new segments of <code>field</code>.
- *
- * The default implementation always returns "Lucene50".
- * <p>
- * <b>WARNING:</b> if you subclass, you are responsible for index
- * backwards compatibility: future version of Lucene are only
- * guaranteed to be able to read the default implementation.
- */
- public PostingsFormat getPostingsFormatForField(String field) {
- return defaultFormat;
- }
-
- /** Returns the docvalues format that should be used for writing
- * new segments of <code>field</code>.
- *
- * The default implementation always returns "Lucene50".
- * <p>
- * <b>WARNING:</b> if you subclass, you are responsible for index
- * backwards compatibility: future version of Lucene are only
- * guaranteed to be able to read the default implementation.
- */
- public DocValuesFormat getDocValuesFormatForField(String field) {
- return defaultDVFormat;
- }
-
- @Override
- public final DocValuesFormat docValuesFormat() {
- return docValuesFormat;
- }
-
- @Override
- public final DimensionalFormat dimensionalFormat() {
- return DimensionalFormat.EMPTY;
- }
-
- private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
- private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
-
- private final NormsFormat normsFormat = new Lucene53NormsFormat();
-
- @Override
- public final NormsFormat normsFormat() {
- return normsFormat;
- }
-}
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java (working copy)
@@ -16,386 +16,8 @@
*/
/**
- * Lucene 5.3 file format.
- *
- * <h1>Apache Lucene - Index File Formats</h1>
- * <div>
- * <ul>
- * <li><a href="#Introduction">Introduction</a></li>
- * <li><a href="#Definitions">Definitions</a>
- * <ul>
- * <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
- * <li><a href="#Types_of_Fields">Types of Fields</a></li>
- * <li><a href="#Segments">Segments</a></li>
- * <li><a href="#Document_Numbers">Document Numbers</a></li>
- * </ul>
- * </li>
- * <li><a href="#Overview">Index Structure Overview</a></li>
- * <li><a href="#File_Naming">File Naming</a></li>
- * <li><a href="#file-names">Summary of File Extensions</a>
- * <ul>
- * <li><a href="#Lock_File">Lock File</a></li>
- * <li><a href="#History">History</a></li>
- * <li><a href="#Limitations">Limitations</a></li>
- * </ul>
- * </li>
- * </ul>
- * </div>
- * <a name="Introduction"></a>
- * <h2>Introduction</h2>
- * <div>
- * <p>This document defines the index file formats used in this version of Lucene.
- * If you are using a different version of Lucene, please consult the copy of
- * <code>docs/</code> that was distributed with
- * the version you are using.</p>
- * <p>Apache Lucene is written in Java, but several efforts are underway to write
- * <a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
- * Lucene in other programming languages</a>. If these versions are to remain
- * compatible with Apache Lucene, then a language-independent definition of the
- * Lucene index format is required. This document thus attempts to provide a
- * complete and independent definition of the Apache Lucene file formats.</p>
- * <p>As Lucene evolves, this document should evolve. Versions of Lucene in
- * different programming languages should endeavor to agree on file formats, and
- * generate new versions of this document.</p>
- * </div>
- * <a name="Definitions"></a>
- * <h2>Definitions</h2>
- * <div>
- * <p>The fundamental concepts in Lucene are index, document, field and term.</p>
- * <p>An index contains a sequence of documents.</p>
- * <ul>
- * <li>A document is a sequence of fields.</li>
- * <li>A field is a named sequence of terms.</li>
- * <li>A term is a sequence of bytes.</li>
- * </ul>
- * <p>The same sequence of bytes in two different fields is considered a different
- * term. Thus terms are represented as a pair: the string naming the field, and the
- * bytes within the field.</p>
- * <a name="Inverted_Indexing"></a>
- * <h3>Inverted Indexing</h3>
- * <p>The index stores statistics about terms in order to make term-based search
- * more efficient. Lucene's index falls into the family of indexes known as an
- * <i>inverted index.</i> This is because it can list, for a term, the documents
- * that contain it. This is the inverse of the natural relationship, in which
- * documents list terms.</p>
- * <a name="Types_of_Fields"></a>
- * <h3>Types of Fields</h3>
- * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
- * in the index literally, in a non-inverted manner. Fields that are inverted are
- * called <i>indexed</i>. A field may be both stored and indexed.</p>
- * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
- * text of a field may be used literally as a term to be indexed. Most fields are
- * tokenized, but sometimes it is useful for certain identifier fields to be
- * indexed literally.</p>
- * <p>See the {@link org.apache.lucene.document.Field Field}
- * java docs for more information on Fields.</p>
- * <a name="Segments"></a>
- * <h3>Segments</h3>
- * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
- * Each segment is a fully independent index, which could be searched separately.
- * Indexes evolve by:</p>
- * <ol>
- * <li>Creating new segments for newly added documents.</li>
- * <li>Merging existing segments.</li>
- * </ol>
- * <p>Searches may involve multiple segments and/or multiple indexes, each index
- * potentially composed of a set of segments.</p>
- * <a name="Document_Numbers"></a>
- * <h3>Document Numbers</h3>
- * <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
- * The first document added to an index is numbered zero, and each subsequent
- * document added gets a number one greater than the previous.</p>
- * <p>Note that a document's number may change, so caution should be taken when
- * storing these numbers outside of Lucene. In particular, numbers may change in
- * the following situations:</p>
- * <ul>
- * <li>
- * <p>The numbers stored in each segment are unique only within the segment, and
- * must be converted before they can be used in a larger context. The standard
- * technique is to allocate each segment a range of values, based on the range of
- * numbers used in that segment. To convert a document number from a segment to an
- * external value, the segment's <i>base</i> document number is added. To convert
- * an external value back to a segment-specific value, the segment is identified
- * by the range that the external value is in, and the segment's base value is
- * subtracted. For example two five document segments might be combined, so that
- * the first segment has a base value of zero, and the second of five. Document
- * three from the second segment would have an external value of eight.</p>
- * </li>
- * <li>
- * <p>When documents are deleted, gaps are created in the numbering. These are
- * eventually removed as the index evolves through merging. Deleted documents are
- * dropped when segments are merged. A freshly-merged segment thus has no gaps in
- * its numbering.</p>
- * </li>
- * </ul>
- * </div>
- * <a name="Overview"></a>
- * <h2>Index Structure Overview</h2>
- * <div>
- * <p>Each segment index maintains the following:</p>
- * <ul>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
- * This contains metadata about a segment, such as the number of documents,
- * what files it uses,
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}.
- * This contains the set of field names used in the index.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
- * This contains, for each document, a list of attribute-value pairs, where the attributes
- * are field names. These are used to store auxiliary information about the document, such as
- * its title, url, or an identifier to access a database. The set of stored fields are what is
- * returned for each hit when searching. This is keyed by document number.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
- * A dictionary containing all of the terms used in all of the
- * indexed fields of all of the documents. The dictionary also contains the number
- * of documents which contain the term, and pointers to the term's frequency and
- * proximity data.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
- * For each term in the dictionary, the numbers of all the
- * documents that contain that term, and the frequency of the term in that
- * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
- * For each term in the dictionary, the positions that the
- * term occurs in each document. Note that this will not exist if all fields in
- * all documents omit position data.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
- * For each field in each document, a value is stored
- * that is multiplied into the score for hits on that field.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
- * For each field in each document, the term vector (sometimes
- * called document vector) may be stored. A term vector consists of term text and
- * term frequency. To add Term Vectors to your index see the
- * {@link org.apache.lucene.document.Field Field} constructors
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-document values}.
- * Like stored values, these are also keyed by document
- * number, but are generally intended to be loaded into main memory for fast
- * access. Whereas stored values are generally intended for summary results from
- * searches, per-document values are useful for things like scoring factors.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
- * An optional file indicating which documents are live.
- * </li>
- * </ul>
- * <p>Details on each of these are provided in their linked pages.</p>
- * </div>
- * <a name="File_Naming"></a>
- * <h2>File Naming</h2>
- * <div>
- * <p>All files belonging to a segment have the same name with varying extensions.
- * The extensions correspond to the different file formats described below. When
- * using the Compound File format (default in 1.4 and greater) these files (except
- * for the Segment info file, the Lock file, and Deleted documents file) are collapsed
- * into a single .cfs file (see below for details)</p>
- * <p>Typically, all segments in an index are stored in a single directory,
- * although this is not required.</p>
- * <p>As of version 2.1 (lock-less commits), file names are never re-used.
- * That is, when any file is saved
- * to the Directory it is given a never before used filename. This is achieved
- * using a simple generations approach. For example, the first segments file is
- * segments_1, then segments_2, etc. The generation is a sequential long integer
- * represented in alpha-numeric (base 36) form.</p>
- * </div>
- * <a name="file-names"></a>
- * <h2>Summary of File Extensions</h2>
- * <div>
- * <p>The following table summarizes the names and extensions of the files in
- * Lucene:</p>
- * <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
- * <tr>
- * <th>Name</th>
- * <th>Extension</th>
- * <th>Brief Description</th>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
- * <td>segments_N</td>
- * <td>Stores information about a commit point</td>
- * </tr>
- * <tr>
- * <td><a href="#Lock_File">Lock File</a></td>
- * <td>write.lock</td>
- * <td>The Write lock prevents multiple IndexWriters from writing to the same
- * file.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
- * <td>.si</td>
- * <td>Stores metadata about a segment</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
- * <td>.cfs, .cfe</td>
- * <td>An optional "virtual" file consisting of all the other index files for
- * systems that frequently run out of file handles.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
- * <td>.fnm</td>
- * <td>Stores information about the fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
- * <td>.fdx</td>
- * <td>Contains pointers to field data</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
- * <td>.fdt</td>
- * <td>The stored fields for documents</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
- * <td>.tim</td>
- * <td>The term dictionary, stores term info</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
- * <td>.tip</td>
- * <td>The index into the Term Dictionary</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
- * <td>.doc</td>
- * <td>Contains the list of docs which contain each term along with frequency</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
- * <td>.pos</td>
- * <td>Stores position information about where a term occurs in the index</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
- * <td>.pay</td>
- * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
- * <td>.nvd, .nvm</td>
- * <td>Encodes length and boost factors for docs and fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-Document Values}</td>
- * <td>.dvd, .dvm</td>
- * <td>Encodes additional scoring factors or other per-document information.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
- * <td>.tvx</td>
- * <td>Stores offset into the document data file</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
- * <td>.tvd</td>
- * <td>Contains information about each document that has term vectors</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
- * <td>.tvf</td>
- * <td>The field level info about term vectors</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
- * <td>.liv</td>
- * <td>Info about what files are live</td>
- * </tr>
- * </table>
- * </div>
- * <a name="Lock_File"></a>
- * <h2>Lock File</h2>
- * The write lock, which is stored in the index directory by default, is named
- * "write.lock". If the lock directory is different from the index directory then
- * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
- * derived from the full path to the index directory. When this file is present, a
- * writer is currently modifying the index (adding or removing documents). This
- * lock file ensures that only one writer is modifying the index at a time.
- * <a name="History"></a>
- * <h2>History</h2>
- * <p>Compatibility notes are provided in this document, describing how file
- * formats have changed from prior versions:</p>
- * <ul>
- * <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
- * no more commit lock). The change is fully backwards compatible: you can open a
- * pre-2.1 index for searching or adding/deleting of docs. When the new segments
- * file is saved (committed), it will be written in the new file format (meaning
- * no specific "upgrade" process is needed). But note that once a commit has
- * occurred, pre-2.1 Lucene will not be able to read the index.</li>
- * <li>In version 2.3, the file format was changed to allow segments to share a
- * single set of doc store (vectors &amp; stored fields) files. This allows for
- * faster indexing in certain cases. The change is fully backwards compatible (in
- * the same way as the lock-less commits change in 2.1).</li>
- * <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
- * Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
- * LUCENE-510</a> for details.</li>
- * <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
- * may be passed to IndexWriter's commit methods (and later retrieved), which is
- * recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
- * LUCENE-1382</a> for details. Also,
- * diagnostics were added to each segment written recording details about why it
- * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
- * <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
- * <li>In version 3.0, compressed fields are no longer written to the index (they
- * can still be read, but on merge the new segment will write them, uncompressed).
- * See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
- * for details.</li>
- * <li>In version 3.1, segments records the code version that created them. See
- * <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
- * Additionally segments track explicitly whether or not they have term vectors.
- * See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
- * for details.</li>
- * <li>In version 3.2, numeric fields are written as natively to stored fields
- * file, previously they were stored in text format only.</li>
- * <li>In version 3.4, fields can omit position data while still indexing term
- * frequencies.</li>
- * <li>In version 4.0, the format of the inverted index became extensible via
- * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
- * ({@code DocValues}) was introduced. Normalization factors need no longer be a
- * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
- * Terms need not be unicode strings, they can be any byte sequence. Term offsets
- * can optionally be indexed into the postings lists. Payloads can be stored in the
- * term vectors.</li>
- * <li>In version 4.1, the format of the postings list changed to use either
- * of FOR compression or variable-byte encoding, depending upon the frequency
- * of the term. Terms appearing only once were changed to inline directly into
- * the term dictionary. Stored fields are compressed by default. </li>
- * <li>In version 4.2, term vectors are compressed by default. DocValues has
- * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
- * on multi-valued fields.</li>
- * <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
- * <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
- * allow updating NumericDocValues fields.</li>
- * <li>In version 4.8, checksum footers were added to the end of each index file
- * for improved data integrity. Specifically, the last 8 bytes of every index file
- * contain the zlib-crc32 checksum of the file.</li>
- * <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
- * that is suitable for faceting/sorting/analytics.
- * </li>
- * </ul>
- * <a name="Limitations"></a>
- * <h2>Limitations</h2>
- * <div>
- * <p>Lucene uses a Java <code>int</code> to refer to
- * document numbers, and the index file format uses an <code>Int32</code>
- * on-disk to store document numbers. This is a limitation
- * of both the index file format and the current implementation. Eventually these
- * should be replaced with either <code>UInt64</code> values, or
- * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
- * </div>
+ * Components from the Lucene 5.3 index format
+ * See {@link org.apache.lucene.codecs.lucene54} for an overview
+ * of the index format.
*/
package org.apache.lucene.codecs.lucene53;
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java (working copy)
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.
+ */
+
+import java.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 5.4 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene54 package documentation for file format details.
+ * @lucene.experimental
+ */
+public class Lucene54Codec extends Codec {
+ private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+ private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
+ private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
+ private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+ private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+
+ private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return Lucene54Codec.this.getPostingsFormatForField(field);
+ }
+ };
+
+ private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return Lucene54Codec.this.getDocValuesFormatForField(field);
+ }
+ };
+
+ private final StoredFieldsFormat storedFieldsFormat;
+
+ /**
+ * Instantiates a new codec.
+ */
+ public Lucene54Codec() {
+ this(Mode.BEST_SPEED);
+ }
+
+ /**
+ * Instantiates a new codec, specifying the stored fields compression
+ * mode to use.
+ * @param mode stored fields compression mode to use for newly
+ * flushed/merged segments.
+ */
+ public Lucene54Codec(Mode mode) {
+ super("Lucene54");
+ this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
+ }
+
+ @Override
+ public final StoredFieldsFormat storedFieldsFormat() {
+ return storedFieldsFormat;
+ }
+
+ @Override
+ public final TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public final PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ @Override
+ public final FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
+ public final SegmentInfoFormat segmentInfoFormat() {
+ return segmentInfosFormat;
+ }
+
+ @Override
+ public final LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
+
+ @Override
+ public final CompoundFormat compoundFormat() {
+ return compoundFormat;
+ }
+
+ /** Returns the postings format that should be used for writing
+ * new segments of <code>field</code>.
+ *
+ * The default implementation always returns "Lucene50".
+ * <p>
+ * <b>WARNING:</b> if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return defaultFormat;
+ }
+
+ /** Returns the docvalues format that should be used for writing
+ * new segments of <code>field</code>.
+ *
+ * The default implementation always returns "Lucene54".
+ * <p>
+ * <b>WARNING:</b> if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return defaultDVFormat;
+ }
+
+ @Override
+ public final DocValuesFormat docValuesFormat() {
+ return docValuesFormat;
+ }
+
+ @Override
+ public final DimensionalFormat dimensionalFormat() {
+ return DimensionalFormat.EMPTY;
+ }
+
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+ private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
+
+ private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+ @Override
+ public final NormsFormat normsFormat() {
+ return normsFormat;
+ }
+}
Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java (working copy)
@@ -0,0 +1,658 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
+
+/** writer for {@link Lucene54DocValuesFormat} */
+final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+ IndexOutput data, meta;
+ final int maxDoc;
+
+ /** expert: Creates a new writer */
+ public Lucene54DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ 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, Lucene54DocValuesFormat.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, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+ maxDoc = state.segmentInfo.maxDoc();
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(this);
+ }
+ }
+ }
+
+ @Override
+ public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+ addNumericField(field, values, true);
+ }
+
+ void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
+ long count = 0;
+ long minValue = Long.MAX_VALUE;
+ long maxValue = Long.MIN_VALUE;
+ long gcd = 0;
+ long missingCount = 0;
+ long zeroCount = 0;
+ // TODO: more efficient?
+ HashSet<Long> uniqueValues = null;
+ if (optimizeStorage) {
+ uniqueValues = new HashSet<>();
+
+ for (Number nv : values) {
+ final long v;
+ if (nv == null) {
+ v = 0;
+ missingCount++;
+ zeroCount++;
+ } else {
+ v = nv.longValue();
+ if (v == 0) {
+ zeroCount++;
+ }
+ }
+
+ 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);
+ }
+ }
+
+ minValue = Math.min(minValue, v);
+ maxValue = Math.max(maxValue, v);
+
+ if (uniqueValues != null) {
+ if (uniqueValues.add(v)) {
+ if (uniqueValues.size() > 256) {
+ uniqueValues = null;
+ }
+ }
+ }
+
+ ++count;
+ }
+ } else {
+ for (Number nv : values) {
+ long v = nv.longValue();
+ minValue = Math.min(minValue, v);
+ maxValue = Math.max(maxValue, v);
+ ++count;
+ }
+ }
+
+ final long delta = maxValue - minValue;
+ final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+ final int tableBitsRequired = uniqueValues == null
+ ? Integer.MAX_VALUE
+ : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+ final int format;
+ if (uniqueValues != null
+ && count <= Integer.MAX_VALUE
+ && (uniqueValues.size() == 1
+ || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
+ // either one unique value C or two unique values: "missing" and C
+ format = CONST_COMPRESSED;
+ } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
+ format = TABLE_COMPRESSED;
+ } else if (gcd != 0 && gcd != 1) {
+ final long gcdDelta = (maxValue - minValue) / gcd;
+ final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
+ format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
+ } else {
+ format = DELTA_COMPRESSED;
+ }
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+ meta.writeVInt(format);
+ if (missingCount == 0) {
+ meta.writeLong(ALL_LIVE);
+ } else if (missingCount == count) {
+ meta.writeLong(ALL_MISSING);
+ } else {
+ meta.writeLong(data.getFilePointer());
+ writeMissingBitset(values);
+ }
+ meta.writeLong(data.getFilePointer());
+ meta.writeVLong(count);
+
+ switch (format) {
+ case CONST_COMPRESSED:
+ // write the constant (nonzero value in the n=2 case, singleton value otherwise)
+ meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
+ break;
+ case GCD_COMPRESSED:
+ meta.writeLong(minValue);
+ meta.writeLong(gcd);
+ final long maxDelta = (maxValue - minValue) / gcd;
+ final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
+ meta.writeVInt(bits);
+ final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
+ for (Number nv : values) {
+ long value = nv == null ? 0 : nv.longValue();
+ quotientWriter.add((value - minValue) / gcd);
+ }
+ quotientWriter.finish();
+ break;
+ case DELTA_COMPRESSED:
+ final long minDelta = delta < 0 ? 0 : minValue;
+ meta.writeLong(minDelta);
+ meta.writeVInt(deltaBitsRequired);
+ final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
+ for (Number nv : values) {
+ long v = nv == null ? 0 : nv.longValue();
+ writer.add(v - minDelta);
+ }
+ writer.finish();
+ break;
+ case TABLE_COMPRESSED:
+ final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+ Arrays.sort(decode);
+ final HashMap<Long,Integer> encode = new HashMap<>();
+ meta.writeVInt(decode.length);
+ for (int i = 0; i < decode.length; i++) {
+ meta.writeLong(decode[i]);
+ encode.put(decode[i], i);
+ }
+ meta.writeVInt(tableBitsRequired);
+ final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
+ for (Number nv : values) {
+ ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
+ }
+ ordsWriter.finish();
+ break;
+ default:
+ throw new AssertionError();
+ }
+ meta.writeLong(data.getFilePointer());
+ }
+
+ // 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 {
+ byte bits = 0;
+ int count = 0;
+ for (Object v : values) {
+ if (count == 8) {
+ data.writeByte(bits);
+ count = 0;
+ bits = 0;
+ }
+ if (v != null) {
+ bits |= 1 << (count & 7);
+ }
+ count++;
+ }
+ if (count > 0) {
+ data.writeByte(bits);
+ }
+ }
+
+ @Override
+ public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+ // write the byte[] data
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.BINARY);
+ int minLength = Integer.MAX_VALUE;
+ int maxLength = Integer.MIN_VALUE;
+ final long startFP = data.getFilePointer();
+ long count = 0;
+ long missingCount = 0;
+ for(BytesRef v : values) {
+ final int length;
+ if (v == null) {
+ length = 0;
+ missingCount++;
+ } else {
+ length = v.length;
+ }
+ minLength = Math.min(minLength, length);
+ maxLength = Math.max(maxLength, length);
+ if (v != null) {
+ data.writeBytes(v.bytes, v.offset, v.length);
+ }
+ count++;
+ }
+ meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
+ if (missingCount == 0) {
+ meta.writeLong(ALL_LIVE);
+ } else if (missingCount == count) {
+ meta.writeLong(ALL_MISSING);
+ } else {
+ meta.writeLong(data.getFilePointer());
+ writeMissingBitset(values);
+ }
+ meta.writeVInt(minLength);
+ meta.writeVInt(maxLength);
+ meta.writeVLong(count);
+ meta.writeLong(startFP);
+
+ // 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.writeLong(data.getFilePointer());
+ meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+ final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+ long addr = 0;
+ writer.add(addr);
+ for (BytesRef v : values) {
+ if (v != null) {
+ addr += v.length;
+ }
+ writer.add(addr);
+ }
+ writer.finish();
+ meta.writeLong(data.getFilePointer());
+ }
+ }
+
+ /** expert: writes a value dictionary for a sorted/sortedset field */
+ private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+ // first check if it's a "fixed-length" terms dict
+ int minLength = Integer.MAX_VALUE;
+ int maxLength = Integer.MIN_VALUE;
+ long numValues = 0;
+ for (BytesRef v : values) {
+ minLength = Math.min(minLength, v.length);
+ maxLength = Math.max(maxLength, v.length);
+ numValues++;
+ }
+ if (minLength == maxLength) {
+ // no index needed: direct addressing by mult
+ addBinaryField(field, values);
+ } else if (numValues < REVERSE_INTERVAL_COUNT) {
+ // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+ addBinaryField(field, values);
+ } else {
+ assert numValues > 0; // we don't have to handle the empty case
+ // header
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.BINARY);
+ meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+ meta.writeLong(-1L);
+ // now write the bytes: sharing prefixes within a block
+ final long startFP = data.getFilePointer();
+ // currently, we have to store the delta from expected for every 1/nth term
+ // we could avoid this, but it's not much and less overall RAM than the previous approach!
+ RAMOutputStream addressBuffer = new RAMOutputStream();
+ MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
+ // buffers up 16 terms
+ RAMOutputStream bytesBuffer = new RAMOutputStream();
+ // buffers up block header
+ RAMOutputStream headerBuffer = new RAMOutputStream();
+ BytesRefBuilder lastTerm = new BytesRefBuilder();
+ lastTerm.grow(maxLength);
+ long count = 0;
+ int suffixDeltas[] = new int[INTERVAL_COUNT];
+ for (BytesRef v : values) {
+ int termPosition = (int) (count & INTERVAL_MASK);
+ if (termPosition == 0) {
+ termAddresses.add(data.getFilePointer() - startFP);
+ // abs-encode first term
+ headerBuffer.writeVInt(v.length);
+ headerBuffer.writeBytes(v.bytes, v.offset, v.length);
+ lastTerm.copyBytes(v);
+ } else {
+ // prefix-code: we only share at most 255 characters, to encode the length as a single
+ // byte and have random access. Larger terms just get less compression.
+ int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
+ bytesBuffer.writeByte((byte) sharedPrefix);
+ bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+ // we can encode one smaller, because terms are unique.
+ suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
+ }
+
+ count++;
+ // flush block
+ if ((count & INTERVAL_MASK) == 0) {
+ flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+ }
+ }
+ // flush trailing crap
+ int leftover = (int) (count & INTERVAL_MASK);
+ if (leftover > 0) {
+ Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
+ flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+ }
+ final long indexStartFP = data.getFilePointer();
+ // write addresses of indexed terms
+ termAddresses.finish();
+ addressBuffer.writeTo(data);
+ addressBuffer = null;
+ termAddresses = null;
+ meta.writeVInt(minLength);
+ meta.writeVInt(maxLength);
+ meta.writeVLong(count);
+ meta.writeLong(startFP);
+ meta.writeLong(indexStartFP);
+ meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+ addReverseTermIndex(field, values, maxLength);
+ }
+ }
+
+ // writes term dictionary "block"
+ // first term is absolute encoded as vint length + bytes.
+ // lengths of subsequent N terms are encoded as either N bytes or N shorts.
+ // in the double-byte case, the first byte is indicated with -1.
+ // subsequent terms are encoded as byte suffixLength + bytes.
+ private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
+ boolean twoByte = false;
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ if (suffixDeltas[i] > 254) {
+ twoByte = true;
+ }
+ }
+ if (twoByte) {
+ headerBuffer.writeByte((byte)255);
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ headerBuffer.writeShort((short) suffixDeltas[i]);
+ }
+ } else {
+ for (int i = 1; i < suffixDeltas.length; i++) {
+ headerBuffer.writeByte((byte) suffixDeltas[i]);
+ }
+ }
+ headerBuffer.writeTo(data);
+ headerBuffer.reset();
+ bytesBuffer.writeTo(data);
+ bytesBuffer.reset();
+ }
+
+ // writes reverse term index: used for binary searching a term into a range of 64 blocks
+ // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
+ // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
+ private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
+ long count = 0;
+ BytesRefBuilder priorTerm = new BytesRefBuilder();
+ priorTerm.grow(maxLength);
+ BytesRef indexTerm = new BytesRef();
+ long startFP = data.getFilePointer();
+ PagedBytes pagedBytes = new PagedBytes(15);
+ MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+
+ for (BytesRef b : values) {
+ int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
+ if (termPosition == 0) {
+ int len = StringHelper.sortKeyLength(priorTerm.get(), b);
+ indexTerm.bytes = b.bytes;
+ indexTerm.offset = b.offset;
+ indexTerm.length = len;
+ addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
+ } else if (termPosition == REVERSE_INTERVAL_MASK) {
+ priorTerm.copyBytes(b);
+ }
+ count++;
+ }
+ addresses.finish();
+ long numBytes = pagedBytes.getPointer();
+ pagedBytes.freeze(true);
+ PagedBytesDataInput in = pagedBytes.getDataInput();
+ meta.writeLong(startFP);
+ data.writeVLong(numBytes);
+ data.copyBytes(in, numBytes);
+ }
+
+ @Override
+ public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.SORTED);
+ addTermsDict(field, values);
+ addNumericField(field, docToOrd, false);
+ }
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC);
+ if (isSingleValued(docToValueCount)) {
+ meta.writeVInt(SORTED_SINGLE_VALUED);
+ // The field is single-valued, we can encode it as NUMERIC
+ addNumericField(field, singletonView(docToValueCount, values, null));
+ } else {
+ final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
+ if (uniqueValueSets != null) {
+ meta.writeVInt(SORTED_SET_TABLE);
+
+ // write the set_id -> values mapping
+ writeDictionary(uniqueValueSets);
+
+ // write the doc -> set_id as a numeric field
+ addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), false);
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
+ // write the stream of values as a numeric field
+ addNumericField(field, values, true);
+ // write the doc -> ord count as a absolute index to the stream
+ addOrdIndex(field, docToValueCount);
+ }
+ }
+ }
+
+ @Override
+ public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.SORTED_SET);
+
+ if (isSingleValued(docToOrdCount)) {
+ meta.writeVInt(SORTED_SINGLE_VALUED);
+ // The field is single-valued, we can encode it as SORTED
+ addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+ } else {
+ final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
+ if (uniqueValueSets != null) {
+ meta.writeVInt(SORTED_SET_TABLE);
+
+ // write the set_id -> ords mapping
+ writeDictionary(uniqueValueSets);
+
+ // write the ord -> byte[] as a binary field
+ addTermsDict(field, values);
+
+ // write the doc -> set_id as a numeric field
+ addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), false);
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+ // write the ord -> byte[] as a binary field
+ addTermsDict(field, values);
+
+ // write the stream of ords as a numeric field
+ // NOTE: we could return an iterator that delta-encodes these within a doc
+ addNumericField(field, ords, false);
+
+ // write the doc -> ord count as a absolute index to the stream
+ addOrdIndex(field, docToOrdCount);
+ }
+ }
+ }
+
+ private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
+ Set<LongsRef> uniqueValueSet = new HashSet<>();
+ LongsRef docValues = new LongsRef(256);
+
+ Iterator<Number> valueCountIterator = docToValueCount.iterator();
+ Iterator<Number> valueIterator = values.iterator();
+ int totalDictSize = 0;
+ while (valueCountIterator.hasNext()) {
+ docValues.length = valueCountIterator.next().intValue();
+ if (docValues.length > 256) {
+ return null;
+ }
+ for (int i = 0; i < docValues.length; ++i) {
+ docValues.longs[i] = valueIterator.next().longValue();
+ }
+ if (uniqueValueSet.contains(docValues)) {
+ continue;
+ }
+ totalDictSize += docValues.length;
+ if (totalDictSize > 256) {
+ return null;
+ }
+ uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
+ }
+ assert valueIterator.hasNext() == false;
+ return new TreeSet<>(uniqueValueSet);
+ }
+
+ private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
+ int lengthSum = 0;
+ for (LongsRef longs : uniqueValueSets) {
+ lengthSum += longs.length;
+ }
+
+ meta.writeInt(lengthSum);
+ for (LongsRef valueSet : uniqueValueSets) {
+ for (int i = 0; i < valueSet.length; ++i) {
+ meta.writeLong(valueSet.longs[valueSet.offset + i]);
+ }
+ }
+
+ meta.writeInt(uniqueValueSets.size());
+ for (LongsRef valueSet : uniqueValueSets) {
+ meta.writeInt(valueSet.length);
+ }
+ }
+
+ private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
+ final Map<LongsRef, Integer> setIds = new HashMap<>();
+ int i = 0;
+ for (LongsRef set : uniqueValueSets) {
+ setIds.put(set, i++);
+ }
+ assert i == uniqueValueSets.size();
+
+ return new Iterable<Number>() {
+
+ @Override
+ public Iterator<Number> iterator() {
+ final Iterator<Number> valueCountIterator = docToValueCount.iterator();
+ final Iterator<Number> valueIterator = values.iterator();
+ final LongsRef docValues = new LongsRef(256);
+ return new Iterator<Number>() {
+
+ @Override
+ public boolean hasNext() {
+ return valueCountIterator.hasNext();
+ }
+
+ @Override
+ public Number next() {
+ docValues.length = valueCountIterator.next().intValue();
+ for (int i = 0; i < docValues.length; ++i) {
+ docValues.longs[i] = valueIterator.next().longValue();
+ }
+ final Integer id = setIds.get(docValues);
+ assert id != null;
+ return id;
+ }
+
+ };
+
+ }
+ };
+ }
+
+ // writes addressing information as MONOTONIC_COMPRESSED integer
+ private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+ meta.writeVInt(MONOTONIC_COMPRESSED);
+ meta.writeLong(-1L);
+ meta.writeLong(data.getFilePointer());
+ meta.writeVLong(maxDoc);
+ meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+ final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+ long addr = 0;
+ writer.add(addr);
+ for (Number v : values) {
+ addr += v.longValue();
+ writer.add(addr);
+ }
+ writer.finish();
+ meta.writeLong(data.getFilePointer());
+ }
+
+ @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); // write checksum
+ }
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(data, meta);
+ } else {
+ IOUtils.closeWhileHandlingException(data, meta);
+ }
+ meta = data = null;
+ }
+ }
+}
Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java (working copy)
@@ -0,0 +1,271 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+
+/**
+ * Lucene 5.4 DocValues format.
+ * <p>
+ * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
+ * <p>
+ * {@link DocValuesType#NUMERIC NUMERIC}:
+ * <ul>
+ * <li>Delta-compressed: per-document integers written as deltas from the minimum value,
+ * compressed with bitpacking. For more information, see {@link DirectWriter}.
+ * <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
+ * when there are unused "gaps" in the range of values used (such as {@link SmallFloat}),
+ * a lookup table is written instead. Each per-document entry is instead the ordinal
+ * to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}).
+ * <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
+ * common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
+ * <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
+ * as blocks of bitpacked integers, encoding the deviation from the expected delta.
+ * <li>Const-compressed: when there is only one possible non-missing value, only the missing
+ * bitset is encoded.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#BINARY BINARY}:
+ * <ul>
+ * <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
+ * Each document's value can be addressed directly with multiplication ({@code docID * length}).
+ * <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
+ * for each document. The addresses are written as Monotonic-compressed numerics.
+ * <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
+ * completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
+ * numerics. A reverse lookup index is written from a portion of every 1024th term.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED SORTED}:
+ * <ul>
+ * <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary,
+ * along with the per-document ordinals written using one of the numeric strategies above.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED_SET SORTED_SET}:
+ * <ul>
+ * <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
+ * <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ * an id, a lookup table is written and the mapping from document to set id is written using the
+ * numeric strategies above.
+ * <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary,
+ * an ordinal list and per-document index into this list are written using the numeric strategies
+ * above.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
+ * <ul>
+ * <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
+ * <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ * an id, a lookup table is written and the mapping from document to set id is written using the
+ * numeric strategies above.
+ * <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
+ * strategies above.
+ * </ul>
+ * <p>
+ * Files:
+ * <ol>
+ * <li><tt>.dvd</tt>: DocValues data</li>
+ * <li><tt>.dvm</tt>: DocValues metadata</li>
+ * </ol>
+ * <ol>
+ * <li><a name="dvm"></a>
+ * <p>The DocValues metadata or .dvm file.</p>
+ * <p>For DocValues field, this stores metadata, such as the offset into the
+ * DocValues data (.dvd)</p>
+ * <p>DocValues metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
+ * <ul>
+ * <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
+ * <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
+ * <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD,BitsPerValue</li>
+ * <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
+ * <li>DeltaNumericEntry --&gt; NumericHeader,MinValue,BitsPerValue</li>
+ * <li>MonotonicNumericEntry --&gt; NumericHeader,PackedVersion,BlockSize</li>
+ * <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
+ * <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
+ * <li>FixedBinaryEntry --&gt; BinaryHeader</li>
+ * <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
+ * <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
+ * <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
+ * <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
+ * <li>SortedSetEntry --&gt; SingleSortedSetEntry | AddressesSortedSetEntry | TableSortedSetEntry</li>
+ * <li>SingleSortedSetEntry --&gt; SetHeader,SortedEntry</li>
+ * <li>AddressesSortedSetEntry --&gt; SetHeader,BinaryEntry,NumericEntry,NumericEntry</li>
+ * <li>TableSortedSetEntry --&gt; SetHeader,TotalTableLength,{@link DataOutput#writeLong Int64}<sup>TotalTableLength</sup>,TableSize,{@link DataOutput#writeInt Int32}<sup>TableSize</sup>,BinaryEntry,NumericEntry</li>
+ * <li>SetHeader --&gt; FieldNumber,EntryType,SetType</li>
+ * <li>SortedNumericEntry --&gt; SingleSortedNumericEntry | AddressesSortedNumericEntry | TableSortedNumericEntry</li>
+ * <li>SingleNumericEntry --&gt; SetHeader,NumericEntry</li>
+ * <li>AddressesSortedNumericEntry --&gt; SetHeader,NumericEntry,NumericEntry</li>
+ * <li>TableSortedNumericEntry --&gt; SetHeader,TotalTableLength,{@link DataOutput#writeLong Int64}<sup>TotalTableLength</sup>,TableSize,{@link DataOutput#writeInt Int32}<sup>TableSize</sup>,NumericEntry</li>
+ * <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ * <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ * <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --&gt; {@link DataOutput#writeLong Int64}</li>
+ * <li>TableSize,BitsPerValue,TotalTableLength --&gt; {@link DataOutput#writeVInt vInt}</li>
+ * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
+ * and an ordinary NumericEntry for the document-to-ord metadata.</p>
+ * <p>FieldNumber of -1 indicates the end of metadata.</p>
+ * <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
+ * <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
+ * <p>EndOffset is the pointer to the end of the data in the DocValues data (.dvd)</p>
+ * <p>NumericType indicates how Numeric values will be compressed:
+ * <ul>
+ * <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
+ * from the minimum value within the block.
+ * <li>1 --&gt; gcd-compressed. When all integers share a common divisor, only quotients are stored
+ * using blocks of delta-encoded ints.
+ * <li>2 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
+ * a lookup table of unique values is written, followed by the ordinal for each document.
+ * <li>3 --&gt; monotonic-compressed. Used to implement addressing for BINARY, SORTED_SET, SORTED_NUMERIC.
+ * <li>4 --&gt; const-compressed. Used when all non-missing values are the same.
+ * </ul>
+ * <p>BinaryType indicates how Binary values will be stored:
+ * <ul>
+ * <li>0 --&gt; fixed-width. All values have the same length, addressing by multiplication.
+ * <li>1 --&gt; variable-width. An address for each value is stored.
+ * <li>2 --&gt; prefix-compressed. An address to the start of every interval'th value is stored.
+ * </ul>
+ * <p>SetType indicates how SortedSet and SortedNumeric values will be stored:
+ * <ul>
+ * <li>0 --&gt; with addresses. There are two numeric entries: a first one from document to start
+ * offset, and a second one from offset to ord/value.
+ * <li>1 --&gt; single-valued. Used when all documents have at most one value and is encoded like
+ * a regular Sorted/Numeric entry.
+ * <li>2 --&gt; table-encoded. A lookup table of unique sets of values is written, followed by a
+ * numeric entry that maps each document to an ordinal in this table.
+ * </ul>
+ * <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
+ * If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
+ * Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
+ * is written for the addresses.
+ * <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
+ * If it's -1, then there are no missing values. If it's -2, all values are missing.
+ * <li><a name="dvd"></a>
+ * <p>The DocValues data or .dvd file.</p>
+ * <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
+ * <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
+ * <ul>
+ * <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
+ * <li>BinaryData --&gt; {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
+ * <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
+ * <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --&gt; {@link DirectWriter PackedInts}</li>
+ * <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
+ * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * </ol>
+ * @lucene.experimental
+ */
+public final class Lucene54DocValuesFormat extends DocValuesFormat {
+
+ /** Sole Constructor */
+ public Lucene54DocValuesFormat() {
+ super("Lucene54");
+ }
+
+ @Override
+ public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ return new Lucene54DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ @Override
+ public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+ return new Lucene54DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ static final String DATA_CODEC = "Lucene54DocValuesData";
+ static final String DATA_EXTENSION = "dvd";
+ static final String META_CODEC = "Lucene54DocValuesMetadata";
+ static final String META_EXTENSION = "dvm";
+ static final int VERSION_START = 0;
+ static final int VERSION_SORTEDSET_TABLE = 1;
+ static final int VERSION_CURRENT = VERSION_SORTEDSET_TABLE;
+
+ // indicates docvalues type
+ static final byte NUMERIC = 0;
+ static final byte BINARY = 1;
+ static final byte SORTED = 2;
+ static final byte SORTED_SET = 3;
+ static final byte SORTED_NUMERIC = 4;
+
+ // address terms in blocks of 16 terms
+ static final int INTERVAL_SHIFT = 4;
+ static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
+ static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
+
+ // build reverse index from every 1024th term
+ static final int REVERSE_INTERVAL_SHIFT = 10;
+ static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
+ static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
+
+ // for conversion from reverse index to block
+ static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
+ static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
+ static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
+
+ /** Compressed using packed blocks of ints. */
+ static final int DELTA_COMPRESSED = 0;
+ /** Compressed by computing the GCD. */
+ static final int GCD_COMPRESSED = 1;
+ /** Compressed by giving IDs to unique values. */
+ static final int TABLE_COMPRESSED = 2;
+ /** Compressed with monotonically increasing values */
+ static final int MONOTONIC_COMPRESSED = 3;
+ /** Compressed with constant value (uses only missing bitset) */
+ static final int CONST_COMPRESSED = 4;
+
+ /** Uncompressed binary, written directly (fixed length). */
+ static final int BINARY_FIXED_UNCOMPRESSED = 0;
+ /** Uncompressed binary, written directly (variable length). */
+ static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+ /** Compressed binary with shared prefixes */
+ static final int BINARY_PREFIX_COMPRESSED = 2;
+
+ /** Standard storage for sorted set values with 1 level of indirection:
+ * {@code docId -> address -> ord}. */
+ static final int SORTED_WITH_ADDRESSES = 0;
+ /** Single-valued sorted set values, encoded as sorted values, so no level
+ * of indirection: {@code docId -> ord}. */
+ static final int SORTED_SINGLE_VALUED = 1;
+ /** Compressed giving IDs to unique sets of values:
+ * {@code docId -> setId -> ords} */
+ static final int SORTED_SET_TABLE = 2;
+
+ /** placeholder for missing offset that means there are no missing values */
+ static final int ALL_LIVE = -1;
+ /** placeholder for missing offset that means all values are missing */
+ static final int ALL_MISSING = -2;
+
+ // addressing uses 16k blocks
+ static final int MONOTONIC_BLOCK_SIZE = 16384;
+ static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
+}
Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java (working copy)
@@ -0,0 +1,1284 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable;
+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.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomAccessOrds;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+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.IOUtils;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+
+import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
+
+/** reader for {@link Lucene54DocValuesFormat} */
+final class Lucene54DocValuesProducer extends DocValuesProducer implements Closeable {
+ private final Map<String,NumericEntry> numerics = new HashMap<>();
+ private final Map<String,BinaryEntry> binaries = new HashMap<>();
+ private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+ private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+ private final Map<String,NumericEntry> ords = new HashMap<>();
+ private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
+ private final int numFields;
+ private final AtomicLong ramBytesUsed;
+ private final IndexInput data;
+ private final int maxDoc;
+
+ // memory-resident structures
+ private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+ private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
+ private final Map<String,DirectMonotonicReader.Meta> directAddressesMeta = new HashMap<>();
+
+ private final boolean merging;
+
+ // clone for merge: when merging we don't do any instances.put()s
+ Lucene54DocValuesProducer(Lucene54DocValuesProducer original) throws IOException {
+ assert Thread.holdsLock(original);
+ numerics.putAll(original.numerics);
+ binaries.putAll(original.binaries);
+ sortedSets.putAll(original.sortedSets);
+ sortedNumerics.putAll(original.sortedNumerics);
+ ords.putAll(original.ords);
+ ordIndexes.putAll(original.ordIndexes);
+ numFields = original.numFields;
+ ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+ data = original.data.clone();
+ maxDoc = original.maxDoc;
+
+ addressInstances.putAll(original.addressInstances);
+ reverseIndexInstances.putAll(original.reverseIndexInstances);
+ merging = true;
+ }
+
+ /** expert: instantiates a new reader */
+ Lucene54DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+ this.maxDoc = state.segmentInfo.maxDoc();
+ merging = false;
+ ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+
+ int version = -1;
+ int numFields = -1;
+
+ // read in the entries from the metadata file.
+ try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+ Throwable priorE = null;
+ try {
+ version = CodecUtil.checkIndexHeader(in, metaCodec,
+ Lucene54DocValuesFormat.VERSION_START,
+ Lucene54DocValuesFormat.VERSION_CURRENT,
+ state.segmentInfo.getId(),
+ state.segmentSuffix);
+ numFields = readFields(in, state.fieldInfos);
+ } catch (Throwable exception) {
+ priorE = exception;
+ } finally {
+ CodecUtil.checkFooter(in, priorE);
+ }
+ }
+
+ this.numFields = numFields;
+ String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+ this.data = state.directory.openInput(dataName, state.context);
+ boolean success = false;
+ try {
+ final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
+ Lucene54DocValuesFormat.VERSION_START,
+ Lucene54DocValuesFormat.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 void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
+ // sorted = binary + numeric
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ BinaryEntry b = readBinaryEntry(info, meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(info, meta);
+ ords.put(info.name, n);
+ }
+
+ private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
+ // sortedset = binary + numeric (addresses) + ordIndex
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ BinaryEntry b = readBinaryEntry(info, meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n1 = readNumericEntry(info, meta);
+ ords.put(info.name, n1);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n2 = readNumericEntry(info, meta);
+ ordIndexes.put(info.name, n2);
+ }
+
+ private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
+ // sortedset table = binary + ordset table + ordset index
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+
+ BinaryEntry b = readBinaryEntry(info, meta);
+ binaries.put(info.name, b);
+
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(info, meta);
+ ords.put(info.name, n);
+ }
+
+ private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+ int numFields = 0;
+ int fieldNumber = meta.readVInt();
+ while (fieldNumber != -1) {
+ numFields++;
+ FieldInfo info = infos.fieldInfo(fieldNumber);
+ if (info == null) {
+ // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
+ throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+ }
+ byte type = meta.readByte();
+ if (type == Lucene54DocValuesFormat.NUMERIC) {
+ numerics.put(info.name, readNumericEntry(info, meta));
+ } else if (type == Lucene54DocValuesFormat.BINARY) {
+ BinaryEntry b = readBinaryEntry(info, meta);
+ binaries.put(info.name, b);
+ } else if (type == Lucene54DocValuesFormat.SORTED) {
+ readSortedField(info, meta);
+ } else if (type == Lucene54DocValuesFormat.SORTED_SET) {
+ SortedSetEntry ss = readSortedSetEntry(meta);
+ sortedSets.put(info.name, ss);
+ if (ss.format == SORTED_WITH_ADDRESSES) {
+ readSortedSetFieldWithAddresses(info, meta);
+ } else if (ss.format == SORTED_SET_TABLE) {
+ readSortedSetFieldWithTable(info, meta);
+ } else if (ss.format == SORTED_SINGLE_VALUED) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.SORTED) {
+ throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+ }
+ readSortedField(info, meta);
+ } else {
+ throw new AssertionError();
+ }
+ } else if (type == Lucene54DocValuesFormat.SORTED_NUMERIC) {
+ SortedSetEntry ss = readSortedSetEntry(meta);
+ sortedNumerics.put(info.name, ss);
+ if (ss.format == SORTED_WITH_ADDRESSES) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ numerics.put(info.name, readNumericEntry(info, meta));
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry ordIndex = readNumericEntry(info, meta);
+ ordIndexes.put(info.name, ordIndex);
+ } else if (ss.format == SORTED_SET_TABLE) {
+ if (meta.readVInt() != info.number) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ NumericEntry n = readNumericEntry(info, meta);
+ ords.put(info.name, n);
+ } else if (ss.format == SORTED_SINGLE_VALUED) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+ }
+ numerics.put(info.name, readNumericEntry(info, meta));
+ } else {
+ throw new AssertionError();
+ }
+ } else {
+ throw new CorruptIndexException("invalid type: " + type, meta);
+ }
+ fieldNumber = meta.readVInt();
+ }
+ return numFields;
+ }
+
+ private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException {
+ NumericEntry entry = new NumericEntry();
+ entry.format = meta.readVInt();
+ entry.missingOffset = meta.readLong();
+ entry.offset = meta.readLong();
+ entry.count = meta.readVLong();
+ switch(entry.format) {
+ case CONST_COMPRESSED:
+ entry.minValue = meta.readLong();
+ if (entry.count > Integer.MAX_VALUE) {
+ // currently just a limitation e.g. of bits interface and so on.
+ throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
+ }
+ break;
+ case GCD_COMPRESSED:
+ entry.minValue = meta.readLong();
+ entry.gcd = meta.readLong();
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case TABLE_COMPRESSED:
+ final int uniqueValues = meta.readVInt();
+ if (uniqueValues > 256) {
+ throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
+ }
+ entry.table = new long[uniqueValues];
+ for (int i = 0; i < uniqueValues; ++i) {
+ entry.table[i] = meta.readLong();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case DELTA_COMPRESSED:
+ entry.minValue = meta.readLong();
+ entry.bitsPerValue = meta.readVInt();
+ break;
+ case MONOTONIC_COMPRESSED:
+ final int blockShift = meta.readVInt();
+ entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc, blockShift);
+ ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
+ directAddressesMeta.put(info.name, entry.monotonicMeta);
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
+ }
+ entry.endOffset = meta.readLong();
+ return entry;
+ }
+
+ private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException {
+ BinaryEntry entry = new BinaryEntry();
+ entry.format = meta.readVInt();
+ entry.missingOffset = meta.readLong();
+ entry.minLength = meta.readVInt();
+ entry.maxLength = meta.readVInt();
+ entry.count = meta.readVLong();
+ entry.offset = meta.readLong();
+ switch(entry.format) {
+ case BINARY_FIXED_UNCOMPRESSED:
+ break;
+ case BINARY_PREFIX_COMPRESSED:
+ entry.addressesOffset = meta.readLong();
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ entry.reverseIndexOffset = meta.readLong();
+ break;
+ case BINARY_VARIABLE_UNCOMPRESSED:
+ entry.addressesOffset = meta.readLong();
+ final int blockShift = meta.readVInt();
+ entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift);
+ ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed());
+ directAddressesMeta.put(info.name, entry.addressesMeta);
+ entry.addressesEndOffset = meta.readLong();
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+ }
+ return entry;
+ }
+
+ SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+ SortedSetEntry entry = new SortedSetEntry();
+ entry.format = meta.readVInt();
+ if (entry.format == SORTED_SET_TABLE) {
+ final int totalTableLength = meta.readInt();
+ if (totalTableLength > 256) {
+ throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
+ }
+ entry.table = new long[totalTableLength];
+ for (int i = 0; i < totalTableLength; ++i) {
+ entry.table[i] = meta.readLong();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+ final int tableSize = meta.readInt();
+ if (tableSize > totalTableLength + 1) { // +1 because of the empty set
+ throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
+ }
+ entry.tableOffsets = new int[tableSize + 1];
+ for (int i = 1; i < entry.tableOffsets.length; ++i) {
+ entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
+ }
+ ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
+ } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
+ throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+ }
+ return entry;
+ }
+
+ @Override
+ public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+ NumericEntry entry = numerics.get(field.name);
+ return getNumeric(entry);
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return ramBytesUsed.get();
+ }
+
+ @Override
+ public synchronized Collection<Accountable> getChildResources() {
+ List<Accountable> resources = new ArrayList<>();
+ resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
+ resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
+ resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta));
+ return Collections.unmodifiableList(resources);
+ }
+
+ @Override
+ public void checkIntegrity() throws IOException {
+ CodecUtil.checksumEntireFile(data);
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(fields=" + numFields + ")";
+ }
+
+ LongValues getNumeric(NumericEntry entry) throws IOException {
+ switch (entry.format) {
+ case CONST_COMPRESSED: {
+ final long constant = entry.minValue;
+ final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
+ return new LongValues() {
+ @Override
+ public long get(long index) {
+ return live.get((int)index) ? constant : 0;
+ }
+ };
+ }
+ case DELTA_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long delta = entry.minValue;
+ final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return delta + values.get(id);
+ }
+ };
+ }
+ case GCD_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long min = entry.minValue;
+ final long mult = entry.gcd;
+ final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return min + mult * quotientReader.get(id);
+ }
+ };
+ }
+ case TABLE_COMPRESSED: {
+ RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ final long table[] = entry.table;
+ final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+ return new LongValues() {
+ @Override
+ public long get(long id) {
+ return table[(int) ords.get(id)];
+ }
+ };
+ }
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+ BinaryEntry bytes = binaries.get(field.name);
+ switch(bytes.format) {
+ case BINARY_FIXED_UNCOMPRESSED:
+ return getFixedBinary(field, bytes);
+ case BINARY_VARIABLE_UNCOMPRESSED:
+ return getVariableBinary(field, bytes);
+ case BINARY_PREFIX_COMPRESSED:
+ return getCompressedBinary(field, bytes);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
+
+ final BytesRef term = new BytesRef(bytes.maxLength);
+ final byte[] buffer = term.bytes;
+ final int length = term.length = bytes.maxLength;
+
+ return new LongBinaryDocValues() {
+ @Override
+ public BytesRef get(long id) {
+ try {
+ data.seek(id * length);
+ data.readBytes(buffer, 0, buffer.length);
+ return term;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ }
+
+ private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
+ final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
+
+ final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
+ final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
+ final byte buffer[] = term.bytes;
+
+ return new LongBinaryDocValues() {
+ @Override
+ public BytesRef get(long id) {
+ long startAddress = addresses.get(id);
+ long endAddress = addresses.get(id+1);
+ int length = (int) (endAddress - startAddress);
+ try {
+ data.seek(startAddress);
+ data.readBytes(buffer, 0, length);
+ term.length = length;
+ return term;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+ }
+
+ /** returns an address instance for prefix-compressed binary values. */
+ private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+ MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+ if (addresses == null) {
+ data.seek(bytes.addressesOffset);
+ final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
+ addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+ if (!merging) {
+ addressInstances.put(field.name, addresses);
+ ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+ }
+ }
+ return addresses;
+ }
+
+ /** returns a reverse lookup instance for prefix-compressed binary values. */
+ private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+ ReverseTermsIndex index = reverseIndexInstances.get(field.name);
+ if (index == null) {
+ index = new ReverseTermsIndex();
+ data.seek(bytes.reverseIndexOffset);
+ long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
+ index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+ long dataSize = data.readVLong();
+ PagedBytes pagedBytes = new PagedBytes(15);
+ pagedBytes.copy(data, dataSize);
+ index.terms = pagedBytes.freeze(true);
+ if (!merging) {
+ reverseIndexInstances.put(field.name, index);
+ ramBytesUsed.addAndGet(index.ramBytesUsed());
+ }
+ }
+ return index;
+ }
+
+ private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+ final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
+ final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
+ assert addresses.size() > 0; // we don't have to handle empty case
+ IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
+ return new CompressedBinaryDocValues(bytes, addresses, index, slice);
+ }
+
+ @Override
+ public SortedDocValues getSorted(FieldInfo field) throws IOException {
+ final int valueCount = (int) binaries.get(field.name).count;
+ final BinaryDocValues binary = getBinary(field);
+ NumericEntry entry = ords.get(field.name);
+ final LongValues ordinals = getNumeric(entry);
+ return new SortedDocValues() {
+
+ @Override
+ public int getOrd(int docID) {
+ return (int) ordinals.get(docID);
+ }
+
+ @Override
+ public BytesRef lookupOrd(int ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public int getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public int lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+ };
+ }
+
+ /** returns an address instance for sortedset ordinal lists */
+ private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
+ RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+ return DirectMonotonicReader.getInstance(entry.monotonicMeta, data);
+ }
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedNumerics.get(field.name);
+ if (ss.format == SORTED_SINGLE_VALUED) {
+ NumericEntry numericEntry = numerics.get(field.name);
+ final LongValues values = getNumeric(numericEntry);
+ final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+ return DocValues.singleton(values, docsWithField);
+ } else if (ss.format == SORTED_WITH_ADDRESSES) {
+ NumericEntry numericEntry = numerics.get(field.name);
+ final LongValues values = getNumeric(numericEntry);
+ final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+ return new SortedNumericDocValues() {
+ long startOffset;
+ long endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ startOffset = ordIndex.get(doc);
+ endOffset = ordIndex.get(doc+1L);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values.get(startOffset + index);
+ }
+
+ @Override
+ public int count() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ } else if (ss.format == SORTED_SET_TABLE) {
+ NumericEntry entry = ords.get(field.name);
+ final LongValues ordinals = getNumeric(entry);
+
+ final long[] table = ss.table;
+ final int[] offsets = ss.tableOffsets;
+ return new SortedNumericDocValues() {
+ int startOffset;
+ int endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ final int ord = (int) ordinals.get(doc);
+ startOffset = offsets[ord];
+ endOffset = offsets[ord + 1];
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return table[startOffset + index];
+ }
+
+ @Override
+ public int count() {
+ return endOffset - startOffset;
+ }
+ };
+ } else {
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedSets.get(field.name);
+ switch (ss.format) {
+ case SORTED_SINGLE_VALUED:
+ final SortedDocValues values = getSorted(field);
+ return DocValues.singleton(values);
+ case SORTED_WITH_ADDRESSES:
+ return getSortedSetWithAddresses(field);
+ case SORTED_SET_TABLE:
+ return getSortedSetTable(field, ss);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
+ final long valueCount = binaries.get(field.name).count;
+ // we keep the byte[]s and list of ords on disk, these could be large
+ final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+ final LongValues ordinals = getNumeric(ords.get(field.name));
+ // but the addresses to the ord stream are in RAM
+ final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+ return new RandomAccessOrds() {
+ long startOffset;
+ long offset;
+ long endOffset;
+
+ @Override
+ public long nextOrd() {
+ if (offset == endOffset) {
+ return NO_MORE_ORDS;
+ } else {
+ long ord = ordinals.get(offset);
+ offset++;
+ return ord;
+ }
+ }
+
+ @Override
+ public void setDocument(int docID) {
+ startOffset = offset = ordIndex.get(docID);
+ endOffset = ordIndex.get(docID+1L);
+ }
+
+ @Override
+ public BytesRef lookupOrd(long ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public long getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public long lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues)binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+
+ @Override
+ public long ordAt(int index) {
+ return ordinals.get(startOffset + index);
+ }
+
+ @Override
+ public int cardinality() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ }
+
+ private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
+ final long valueCount = binaries.get(field.name).count;
+ final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+ final LongValues ordinals = getNumeric(ords.get(field.name));
+
+ final long[] table = ss.table;
+ final int[] offsets = ss.tableOffsets;
+
+ return new RandomAccessOrds() {
+
+ int offset, startOffset, endOffset;
+
+ @Override
+ public void setDocument(int docID) {
+ final int ord = (int) ordinals.get(docID);
+ offset = startOffset = offsets[ord];
+ endOffset = offsets[ord + 1];
+ }
+
+ @Override
+ public long ordAt(int index) {
+ return table[startOffset + index];
+ }
+
+ @Override
+ public long nextOrd() {
+ if (offset == endOffset) {
+ return NO_MORE_ORDS;
+ } else {
+ return table[offset++];
+ }
+ }
+
+ @Override
+ public int cardinality() {
+ return endOffset - startOffset;
+ }
+
+ @Override
+ public BytesRef lookupOrd(long ord) {
+ return binary.get(ord);
+ }
+
+ @Override
+ public long getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public long lookupTerm(BytesRef key) {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues) binary).lookupTerm(key);
+ } else {
+ return super.lookupTerm(key);
+ }
+ }
+
+ @Override
+ public TermsEnum termsEnum() {
+ if (binary instanceof CompressedBinaryDocValues) {
+ return ((CompressedBinaryDocValues) binary).getTermsEnum();
+ } else {
+ return super.termsEnum();
+ }
+ }
+
+ };
+ }
+
+ private Bits getLiveBits(final long offset, final int count) throws IOException {
+ if (offset == ALL_MISSING) {
+ return new Bits.MatchNoBits(count);
+ } else if (offset == ALL_LIVE) {
+ return new Bits.MatchAllBits(count);
+ } else {
+ int length = (int) ((count + 7L) >>> 3);
+ final RandomAccessInput in = data.randomAccessSlice(offset, length);
+ return new Bits() {
+ @Override
+ public boolean get(int index) {
+ try {
+ return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public int length() {
+ return count;
+ }
+ };
+ }
+ }
+
+ @Override
+ public Bits getDocsWithField(FieldInfo field) throws IOException {
+ switch(field.getDocValuesType()) {
+ case SORTED_SET:
+ return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+ case SORTED_NUMERIC:
+ return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
+ case SORTED:
+ return DocValues.docsWithValue(getSorted(field), maxDoc);
+ case BINARY:
+ BinaryEntry be = binaries.get(field.name);
+ return getLiveBits(be.missingOffset, maxDoc);
+ case NUMERIC:
+ NumericEntry ne = numerics.get(field.name);
+ return getLiveBits(ne.missingOffset, maxDoc);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public synchronized DocValuesProducer getMergeInstance() throws IOException {
+ return new Lucene54DocValuesProducer(this);
+ }
+
+ @Override
+ public void close() throws IOException {
+ data.close();
+ }
+
+ /** metadata entry for a numeric docvalues field */
+ static class NumericEntry {
+ private NumericEntry() {}
+ /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+ long missingOffset;
+ /** offset to the actual numeric values */
+ public long offset;
+ /** end offset to the actual numeric values */
+ public long endOffset;
+ /** bits per value used to pack the numeric values */
+ public int bitsPerValue;
+
+ int format;
+ /** count of values written */
+ public long count;
+
+ /** monotonic meta */
+ public DirectMonotonicReader.Meta monotonicMeta;
+
+ long minValue;
+ long gcd;
+ long table[];
+ }
+
+ /** metadata entry for a binary docvalues field */
+ static class BinaryEntry {
+ private BinaryEntry() {}
+ /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+ long missingOffset;
+ /** offset to the actual binary values */
+ long offset;
+
+ int format;
+ /** count of values written */
+ public long count;
+ int minLength;
+ int maxLength;
+ /** offset to the addressing data that maps a value to its slice of the byte[] */
+ public long addressesOffset, addressesEndOffset;
+ /** meta data for addresses */
+ public DirectMonotonicReader.Meta addressesMeta;
+ /** offset to the reverse index */
+ public long reverseIndexOffset;
+ /** packed ints version used to encode addressing information */
+ public int packedIntsVersion;
+ /** packed ints blocksize */
+ public int blockSize;
+ }
+
+ /** metadata entry for a sorted-set docvalues field */
+ static class SortedSetEntry {
+ private SortedSetEntry() {}
+ int format;
+
+ long[] table;
+ int[] tableOffsets;
+ }
+
+ // internally we compose complex dv (sorted/sortedset) from other ones
+ static abstract class LongBinaryDocValues extends BinaryDocValues {
+ @Override
+ public final BytesRef get(int docID) {
+ return get((long)docID);
+ }
+
+ abstract BytesRef get(long id);
+ }
+
+ // used for reverse lookup to a small range of blocks
+ static class ReverseTermsIndex implements Accountable {
+ public MonotonicBlockPackedReader termAddresses;
+ public PagedBytes.Reader terms;
+
+ @Override
+ public long ramBytesUsed() {
+ return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+ }
+
+ @Override
+ public Collection<Accountable> getChildResources() {
+ List<Accountable> resources = new ArrayList<>();
+ resources.add(Accountables.namedAccountable("term bytes", terms));
+ resources.add(Accountables.namedAccountable("term addresses", termAddresses));
+ return Collections.unmodifiableList(resources);
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+ }
+ }
+
+ //in the compressed case, we add a few additional operations for
+ //more efficient reverse lookup and enumeration
+ static final class CompressedBinaryDocValues extends LongBinaryDocValues {
+ final long numValues;
+ final long numIndexValues;
+ final int maxTermLength;
+ final MonotonicBlockPackedReader addresses;
+ final IndexInput data;
+ final CompressedBinaryTermsEnum termsEnum;
+ final PagedBytes.Reader reverseTerms;
+ final MonotonicBlockPackedReader reverseAddresses;
+ final long numReverseIndexValues;
+
+ public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
+ this.maxTermLength = bytes.maxLength;
+ this.numValues = bytes.count;
+ this.addresses = addresses;
+ this.numIndexValues = addresses.size();
+ this.data = data;
+ this.reverseTerms = index.terms;
+ this.reverseAddresses = index.termAddresses;
+ this.numReverseIndexValues = reverseAddresses.size();
+ this.termsEnum = getTermsEnum(data);
+ }
+
+ @Override
+ public BytesRef get(long id) {
+ try {
+ termsEnum.seekExact(id);
+ return termsEnum.term();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ long lookupTerm(BytesRef key) {
+ try {
+ switch (termsEnum.seekCeil(key)) {
+ case FOUND: return termsEnum.ord();
+ case NOT_FOUND: return -termsEnum.ord()-1;
+ default: return -numValues-1;
+ }
+ } catch (IOException bogus) {
+ throw new RuntimeException(bogus);
+ }
+ }
+
+ TermsEnum getTermsEnum() {
+ try {
+ return getTermsEnum(data.clone());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
+ return new CompressedBinaryTermsEnum(input);
+ }
+
+ class CompressedBinaryTermsEnum extends TermsEnum {
+ private long currentOrd = -1;
+ // offset to the start of the current block
+ private long currentBlockStart;
+ private final IndexInput input;
+ // delta from currentBlockStart to start of each term
+ private final int offsets[] = new int[INTERVAL_COUNT];
+ private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
+
+ private final BytesRef term = new BytesRef(maxTermLength);
+ private final BytesRef firstTerm = new BytesRef(maxTermLength);
+ private final BytesRef scratch = new BytesRef();
+
+ CompressedBinaryTermsEnum(IndexInput input) throws IOException {
+ this.input = input;
+ input.seek(0);
+ }
+
+ private void readHeader() throws IOException {
+ firstTerm.length = input.readVInt();
+ input.readBytes(firstTerm.bytes, 0, firstTerm.length);
+ input.readBytes(buffer, 0, INTERVAL_COUNT-1);
+ if (buffer[0] == -1) {
+ readShortAddresses();
+ } else {
+ readByteAddresses();
+ }
+ currentBlockStart = input.getFilePointer();
+ }
+
+ // read single byte addresses: each is delta - 2
+ // (shared prefix byte and length > 0 are both implicit)
+ private void readByteAddresses() throws IOException {
+ int addr = 0;
+ for (int i = 1; i < offsets.length; i++) {
+ addr += 2 + (buffer[i-1] & 0xFF);
+ offsets[i] = addr;
+ }
+ }
+
+ // read double byte addresses: each is delta - 2
+ // (shared prefix byte and length > 0 are both implicit)
+ private void readShortAddresses() throws IOException {
+ input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
+ int addr = 0;
+ for (int i = 1; i < offsets.length; i++) {
+ int x = i<<1;
+ addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
+ offsets[i] = addr;
+ }
+ }
+
+ // set term to the first term
+ private void readFirstTerm() throws IOException {
+ term.length = firstTerm.length;
+ System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
+ }
+
+ // read term at offset, delta encoded from first term
+ private void readTerm(int offset) throws IOException {
+ int start = input.readByte() & 0xFF;
+ System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
+ int suffix = offsets[offset] - offsets[offset-1] - 1;
+ input.readBytes(term.bytes, start, suffix);
+ term.length = start + suffix;
+ }
+
+ @Override
+ public BytesRef next() throws IOException {
+ currentOrd++;
+ if (currentOrd >= numValues) {
+ return null;
+ } else {
+ int offset = (int) (currentOrd & INTERVAL_MASK);
+ if (offset == 0) {
+ // switch to next block
+ readHeader();
+ readFirstTerm();
+ } else {
+ readTerm(offset);
+ }
+ return term;
+ }
+ }
+
+ // binary search reverse index to find smaller
+ // range of blocks to search
+ long binarySearchIndex(BytesRef text) throws IOException {
+ long low = 0;
+ long high = numReverseIndexValues - 1;
+ while (low <= high) {
+ long mid = (low + high) >>> 1;
+ reverseTerms.fill(scratch, reverseAddresses.get(mid));
+ int cmp = scratch.compareTo(text);
+
+ if (cmp < 0) {
+ low = mid + 1;
+ } else if (cmp > 0) {
+ high = mid - 1;
+ } else {
+ return mid;
+ }
+ }
+ return high;
+ }
+
+ // binary search against first term in block range
+ // to find term's block
+ long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
+ while (low <= high) {
+ long mid = (low + high) >>> 1;
+ input.seek(addresses.get(mid));
+ term.length = input.readVInt();
+ input.readBytes(term.bytes, 0, term.length);
+ int cmp = term.compareTo(text);
+
+ if (cmp < 0) {
+ low = mid + 1;
+ } else if (cmp > 0) {
+ high = mid - 1;
+ } else {
+ return mid;
+ }
+ }
+ return high;
+ }
+
+ @Override
+ public SeekStatus seekCeil(BytesRef text) throws IOException {
+ // locate block: narrow to block range with index, then search blocks
+ final long block;
+ long indexPos = binarySearchIndex(text);
+ if (indexPos < 0) {
+ block = 0;
+ } else {
+ long low = indexPos << BLOCK_INTERVAL_SHIFT;
+ long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
+ block = Math.max(low, binarySearchBlock(text, low, high));
+ }
+
+ // position before block, then scan to term.
+ input.seek(addresses.get(block));
+ currentOrd = (block << INTERVAL_SHIFT) - 1;
+
+ while (next() != null) {
+ int cmp = term.compareTo(text);
+ if (cmp == 0) {
+ return SeekStatus.FOUND;
+ } else if (cmp > 0) {
+ return SeekStatus.NOT_FOUND;
+ }
+ }
+ return SeekStatus.END;
+ }
+
+ @Override
+ public void seekExact(long ord) throws IOException {
+ long block = ord >>> INTERVAL_SHIFT;
+ if (block != currentOrd >>> INTERVAL_SHIFT) {
+ // switch to different block
+ input.seek(addresses.get(block));
+ readHeader();
+ }
+
+ currentOrd = ord;
+
+ int offset = (int) (ord & INTERVAL_MASK);
+ if (offset == 0) {
+ readFirstTerm();
+ } else {
+ input.seek(currentBlockStart + offsets[offset-1]);
+ readTerm(offset);
+ }
+ }
+
+ @Override
+ public BytesRef term() throws IOException {
+ return term;
+ }
+
+ @Override
+ public long ord() throws IOException {
+ return currentOrd;
+ }
+
+ @Override
+ public int docFreq() throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long totalTermFreq() throws IOException {
+ return -1;
+ }
+
+ @Override
+ public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ }
+ }
+}
Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java (working copy)
@@ -0,0 +1,403 @@
+/*
+ * 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.
+ */
+
+/**
+ * Lucene 5.4 file format.
+ *
+ * <h1>Apache Lucene - Index File Formats</h1>
+ * <div>
+ * <ul>
+ * <li><a href="#Introduction">Introduction</a></li>
+ * <li><a href="#Definitions">Definitions</a>
+ * <ul>
+ * <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
+ * <li><a href="#Types_of_Fields">Types of Fields</a></li>
+ * <li><a href="#Segments">Segments</a></li>
+ * <li><a href="#Document_Numbers">Document Numbers</a></li>
+ * </ul>
+ * </li>
+ * <li><a href="#Overview">Index Structure Overview</a></li>
+ * <li><a href="#File_Naming">File Naming</a></li>
+ * <li><a href="#file-names">Summary of File Extensions</a>
+ * <ul>
+ * <li><a href="#Lock_File">Lock File</a></li>
+ * <li><a href="#History">History</a></li>
+ * <li><a href="#Limitations">Limitations</a></li>
+ * </ul>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Introduction"></a>
+ * <h2>Introduction</h2>
+ * <div>
+ * <p>This document defines the index file formats used in this version of Lucene.
+ * If you are using a different version of Lucene, please consult the copy of
+ * <code>docs/</code> that was distributed with
+ * the version you are using.</p>
+ * <p>Apache Lucene is written in Java, but several efforts are underway to write
+ * <a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
+ * Lucene in other programming languages</a>. If these versions are to remain
+ * compatible with Apache Lucene, then a language-independent definition of the
+ * Lucene index format is required. This document thus attempts to provide a
+ * complete and independent definition of the Apache Lucene file formats.</p>
+ * <p>As Lucene evolves, this document should evolve. Versions of Lucene in
+ * different programming languages should endeavor to agree on file formats, and
+ * generate new versions of this document.</p>
+ * </div>
+ * <a name="Definitions"></a>
+ * <h2>Definitions</h2>
+ * <div>
+ * <p>The fundamental concepts in Lucene are index, document, field and term.</p>
+ * <p>An index contains a sequence of documents.</p>
+ * <ul>
+ * <li>A document is a sequence of fields.</li>
+ * <li>A field is a named sequence of terms.</li>
+ * <li>A term is a sequence of bytes.</li>
+ * </ul>
+ * <p>The same sequence of bytes in two different fields is considered a different
+ * term. Thus terms are represented as a pair: the string naming the field, and the
+ * bytes within the field.</p>
+ * <a name="Inverted_Indexing"></a>
+ * <h3>Inverted Indexing</h3>
+ * <p>The index stores statistics about terms in order to make term-based search
+ * more efficient. Lucene's index falls into the family of indexes known as an
+ * <i>inverted index.</i> This is because it can list, for a term, the documents
+ * that contain it. This is the inverse of the natural relationship, in which
+ * documents list terms.</p>
+ * <a name="Types_of_Fields"></a>
+ * <h3>Types of Fields</h3>
+ * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
+ * in the index literally, in a non-inverted manner. Fields that are inverted are
+ * called <i>indexed</i>. A field may be both stored and indexed.</p>
+ * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
+ * text of a field may be used literally as a term to be indexed. Most fields are
+ * tokenized, but sometimes it is useful for certain identifier fields to be
+ * indexed literally.</p>
+ * <p>See the {@link org.apache.lucene.document.Field Field}
+ * java docs for more information on Fields.</p>
+ * <a name="Segments"></a>
+ * <h3>Segments</h3>
+ * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
+ * Each segment is a fully independent index, which could be searched separately.
+ * Indexes evolve by:</p>
+ * <ol>
+ * <li>Creating new segments for newly added documents.</li>
+ * <li>Merging existing segments.</li>
+ * </ol>
+ * <p>Searches may involve multiple segments and/or multiple indexes, each index
+ * potentially composed of a set of segments.</p>
+ * <a name="Document_Numbers"></a>
+ * <h3>Document Numbers</h3>
+ * <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
+ * The first document added to an index is numbered zero, and each subsequent
+ * document added gets a number one greater than the previous.</p>
+ * <p>Note that a document's number may change, so caution should be taken when
+ * storing these numbers outside of Lucene. In particular, numbers may change in
+ * the following situations:</p>
+ * <ul>
+ * <li>
+ * <p>The numbers stored in each segment are unique only within the segment, and
+ * must be converted before they can be used in a larger context. The standard
+ * technique is to allocate each segment a range of values, based on the range of
+ * numbers used in that segment. To convert a document number from a segment to an
+ * external value, the segment's <i>base</i> document number is added. To convert
+ * an external value back to a segment-specific value, the segment is identified
+ * by the range that the external value is in, and the segment's base value is
+ * subtracted. For example two five document segments might be combined, so that
+ * the first segment has a base value of zero, and the second of five. Document
+ * three from the second segment would have an external value of eight.</p>
+ * </li>
+ * <li>
+ * <p>When documents are deleted, gaps are created in the numbering. These are
+ * eventually removed as the index evolves through merging. Deleted documents are
+ * dropped when segments are merged. A freshly-merged segment thus has no gaps in
+ * its numbering.</p>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Overview"></a>
+ * <h2>Index Structure Overview</h2>
+ * <div>
+ * <p>Each segment index maintains the following:</p>
+ * <ul>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
+ * This contains metadata about a segment, such as the number of documents,
+ * what files it uses,
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}.
+ * This contains the set of field names used in the index.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
+ * This contains, for each document, a list of attribute-value pairs, where the attributes
+ * are field names. These are used to store auxiliary information about the document, such as
+ * its title, url, or an identifier to access a database. The set of stored fields are what is
+ * returned for each hit when searching. This is keyed by document number.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
+ * A dictionary containing all of the terms used in all of the
+ * indexed fields of all of the documents. The dictionary also contains the number
+ * of documents which contain the term, and pointers to the term's frequency and
+ * proximity data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
+ * For each term in the dictionary, the numbers of all the
+ * documents that contain that term, and the frequency of the term in that
+ * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
+ * For each term in the dictionary, the positions that the
+ * term occurs in each document. Note that this will not exist if all fields in
+ * all documents omit position data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
+ * For each field in each document, a value is stored
+ * that is multiplied into the score for hits on that field.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
+ * For each field in each document, the term vector (sometimes
+ * called document vector) may be stored. A term vector consists of term text and
+ * term frequency. To add Term Vectors to your index see the
+ * {@link org.apache.lucene.document.Field Field} constructors
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-document values}.
+ * Like stored values, these are also keyed by document
+ * number, but are generally intended to be loaded into main memory for fast
+ * access. Whereas stored values are generally intended for summary results from
+ * searches, per-document values are useful for things like scoring factors.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
+ * An optional file indicating which documents are live.
+ * </li>
+ * </ul>
+ * <p>Details on each of these are provided in their linked pages.</p>
+ * </div>
+ * <a name="File_Naming"></a>
+ * <h2>File Naming</h2>
+ * <div>
+ * <p>All files belonging to a segment have the same name with varying extensions.
+ * The extensions correspond to the different file formats described below. When
+ * using the Compound File format (default in 1.4 and greater) these files (except
+ * for the Segment info file, the Lock file, and Deleted documents file) are collapsed
+ * into a single .cfs file (see below for details)</p>
+ * <p>Typically, all segments in an index are stored in a single directory,
+ * although this is not required.</p>
+ * <p>As of version 2.1 (lock-less commits), file names are never re-used.
+ * That is, when any file is saved
+ * to the Directory it is given a never before used filename. This is achieved
+ * using a simple generations approach. For example, the first segments file is
+ * segments_1, then segments_2, etc. The generation is a sequential long integer
+ * represented in alpha-numeric (base 36) form.</p>
+ * </div>
+ * <a name="file-names"></a>
+ * <h2>Summary of File Extensions</h2>
+ * <div>
+ * <p>The following table summarizes the names and extensions of the files in
+ * Lucene:</p>
+ * <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
+ * <tr>
+ * <th>Name</th>
+ * <th>Extension</th>
+ * <th>Brief Description</th>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
+ * <td>segments_N</td>
+ * <td>Stores information about a commit point</td>
+ * </tr>
+ * <tr>
+ * <td><a href="#Lock_File">Lock File</a></td>
+ * <td>write.lock</td>
+ * <td>The Write lock prevents multiple IndexWriters from writing to the same
+ * file.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
+ * <td>.si</td>
+ * <td>Stores metadata about a segment</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
+ * <td>.cfs, .cfe</td>
+ * <td>An optional "virtual" file consisting of all the other index files for
+ * systems that frequently run out of file handles.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
+ * <td>.fnm</td>
+ * <td>Stores information about the fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
+ * <td>.fdx</td>
+ * <td>Contains pointers to field data</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
+ * <td>.fdt</td>
+ * <td>The stored fields for documents</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
+ * <td>.tim</td>
+ * <td>The term dictionary, stores term info</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
+ * <td>.tip</td>
+ * <td>The index into the Term Dictionary</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
+ * <td>.doc</td>
+ * <td>Contains the list of docs which contain each term along with frequency</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
+ * <td>.pos</td>
+ * <td>Stores position information about where a term occurs in the index</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
+ * <td>.pay</td>
+ * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
+ * <td>.nvd, .nvm</td>
+ * <td>Encodes length and boost factors for docs and fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
+ * <td>.dvd, .dvm</td>
+ * <td>Encodes additional scoring factors or other per-document information.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
+ * <td>.tvx</td>
+ * <td>Stores offset into the document data file</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
+ * <td>.tvd</td>
+ * <td>Contains information about each document that has term vectors</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
+ * <td>.tvf</td>
+ * <td>The field level info about term vectors</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
+ * <td>.liv</td>
+ * <td>Info about what files are live</td>
+ * </tr>
+ * </table>
+ * </div>
+ * <a name="Lock_File"></a>
+ * <h2>Lock File</h2>
+ * The write lock, which is stored in the index directory by default, is named
+ * "write.lock". If the lock directory is different from the index directory then
+ * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
+ * derived from the full path to the index directory. When this file is present, a
+ * writer is currently modifying the index (adding or removing documents). This
+ * lock file ensures that only one writer is modifying the index at a time.
+ * <a name="History"></a>
+ * <h2>History</h2>
+ * <p>Compatibility notes are provided in this document, describing how file
+ * formats have changed from prior versions:</p>
+ * <ul>
+ * <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
+ * no more commit lock). The change is fully backwards compatible: you can open a
+ * pre-2.1 index for searching or adding/deleting of docs. When the new segments
+ * file is saved (committed), it will be written in the new file format (meaning
+ * no specific "upgrade" process is needed). But note that once a commit has
+ * occurred, pre-2.1 Lucene will not be able to read the index.</li>
+ * <li>In version 2.3, the file format was changed to allow segments to share a
+ * single set of doc store (vectors &amp; stored fields) files. This allows for
+ * faster indexing in certain cases. The change is fully backwards compatible (in
+ * the same way as the lock-less commits change in 2.1).</li>
+ * <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+ * Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
+ * LUCENE-510</a> for details.</li>
+ * <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
+ * may be passed to IndexWriter's commit methods (and later retrieved), which is
+ * recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
+ * LUCENE-1382</a> for details. Also,
+ * diagnostics were added to each segment written recording details about why it
+ * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
+ * <li>In version 3.0, compressed fields are no longer written to the index (they
+ * can still be read, but on merge the new segment will write them, uncompressed).
+ * See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
+ * for details.</li>
+ * <li>In version 3.1, segments records the code version that created them. See
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
+ * Additionally segments track explicitly whether or not they have term vectors.
+ * See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
+ * for details.</li>
+ * <li>In version 3.2, numeric fields are written as natively to stored fields
+ * file, previously they were stored in text format only.</li>
+ * <li>In version 3.4, fields can omit position data while still indexing term
+ * frequencies.</li>
+ * <li>In version 4.0, the format of the inverted index became extensible via
+ * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
+ * ({@code DocValues}) was introduced. Normalization factors need no longer be a
+ * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
+ * Terms need not be unicode strings, they can be any byte sequence. Term offsets
+ * can optionally be indexed into the postings lists. Payloads can be stored in the
+ * term vectors.</li>
+ * <li>In version 4.1, the format of the postings list changed to use either
+ * of FOR compression or variable-byte encoding, depending upon the frequency
+ * of the term. Terms appearing only once were changed to inline directly into
+ * the term dictionary. Stored fields are compressed by default. </li>
+ * <li>In version 4.2, term vectors are compressed by default. DocValues has
+ * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
+ * on multi-valued fields.</li>
+ * <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
+ * <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
+ * allow updating NumericDocValues fields.</li>
+ * <li>In version 4.8, checksum footers were added to the end of each index file
+ * for improved data integrity. Specifically, the last 8 bytes of every index file
+ * contain the zlib-crc32 checksum of the file.</li>
+ * <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
+ * that is suitable for faceting/sorting/analytics.
+ * <li>In version 5.4, DocValues have been improved to store more information on disk:
+ * addresses for binary fields and ord indexes for multi-valued fields.
+ * </li>
+ * </ul>
+ * <a name="Limitations"></a>
+ * <h2>Limitations</h2>
+ * <div>
+ * <p>Lucene uses a Java <code>int</code> to refer to
+ * document numbers, and the index file format uses an <code>Int32</code>
+ * on-disk to store document numbers. This is a limitation
+ * of both the index file format and the current implementation. Eventually these
+ * should be replaced with either <code>UInt64</code> values, or
+ * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
+ * </div>
+ */
+package org.apache.lucene.codecs.lucene54;
Property changes on: lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java (working copy)
@@ -0,0 +1,122 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Retrieves an instance previously written by {@link DirectMonotonicWriter}.
+ * @see DirectMonotonicWriter
+ */
+public final class DirectMonotonicReader {
+
+ /** An instance that always returns {@code 0}. */
+ private static final LongValues EMPTY = new LongValues() {
+
+ @Override
+ public long get(long index) {
+ return 0;
+ }
+
+ };
+
+ /** In-memory metadata that needs to be kept around for
+ * {@link DirectMonotonicReader} to read data from disk. */
+ public static class Meta implements Accountable {
+ private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Meta.class);
+
+ final long numValues;
+ final int blockShift;
+ final int numBlocks;
+ final long[] mins;
+ final float[] avgs;
+ final byte[] bpvs;
+ final long[] offsets;
+
+ Meta(long numValues, int blockShift) {
+ this.numValues = numValues;
+ this.blockShift = blockShift;
+ long numBlocks = numValues >>> blockShift;
+ if ((numBlocks << blockShift) < numValues) {
+ numBlocks += 1;
+ }
+ this.numBlocks = (int) numBlocks;
+ this.mins = new long[this.numBlocks];
+ this.avgs = new float[this.numBlocks];
+ this.bpvs = new byte[this.numBlocks];
+ this.offsets = new long[this.numBlocks];
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return BASE_RAM_BYTES_USED
+ + RamUsageEstimator.sizeOf(mins)
+ + RamUsageEstimator.sizeOf(avgs)
+ + RamUsageEstimator.sizeOf(bpvs)
+ + RamUsageEstimator.sizeOf(offsets);
+ }
+ }
+
+ /** Load metadata from the given {@link IndexInput}.
+ * @see DirectMonotonicReader#getInstance(Meta, RandomAccessInput) */
+ public static Meta loadMeta(IndexInput metaIn, long numValues, int blockShift) throws IOException {
+ Meta meta = new Meta(numValues, blockShift);
+ for (int i = 0; i < meta.numBlocks; ++i) {
+ meta.mins[i] = metaIn.readLong();
+ meta.avgs[i] = Float.intBitsToFloat(metaIn.readInt());
+ meta.offsets[i] = metaIn.readLong();
+ meta.bpvs[i] = metaIn.readByte();
+ }
+ return meta;
+ }
+
+ /**
+ * Retrieves an instance from the specified slice.
+ */
+ public static LongValues getInstance(Meta meta, RandomAccessInput data) throws IOException {
+ final LongValues[] readers = new LongValues[meta.numBlocks];
+ for (int i = 0; i < meta.mins.length; ++i) {
+ if (meta.bpvs[i] == 0) {
+ readers[i] = EMPTY;
+ } else {
+ readers[i] = DirectReader.getInstance(data, meta.bpvs[i], meta.offsets[i]);
+ }
+ }
+ final int blockShift = meta.blockShift;
+
+ final long[] mins = meta.mins;
+ final float[] avgs = meta.avgs;
+ return new LongValues() {
+
+ @Override
+ public long get(long index) {
+ final int block = (int) (index >>> blockShift);
+ final long blockIndex = index & ((1 << blockShift) - 1);
+ final long delta = readers[block].get(blockIndex);
+ return mins[block] + (long) (avgs[block] * blockIndex) + delta;
+ }
+
+ };
+ }
+}
Property changes on: lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java (working copy)
@@ -0,0 +1,137 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.
+ */
+
+
+import java.io.IOException;
+
+import org.apache.lucene.store.IndexOutput;
+
+/**
+ * Write monotonically-increasing sequences of integers. This writer splits
+ * data into blocks and then for each block, computes the average slope, the
+ * minimum value and only encode the delta from the expected value using a
+ * {@link DirectWriter}.
+ *
+ * @see DirectMonotonicReader
+ * @lucene.internal
+ */
+public final class DirectMonotonicWriter {
+
+ public static final int MIN_BLOCK_SHIFT = 2;
+ public static final int MAX_BLOCK_SHIFT = 22;
+
+ final IndexOutput meta;
+ final IndexOutput data;
+ final long numValues;
+ final long baseDataPointer;
+ final long[] buffer;
+ int bufferSize;
+ long count;
+ boolean finished;
+
+ DirectMonotonicWriter(IndexOutput metaOut, IndexOutput dataOut, long numValues, int blockShift) {
+ this.meta = metaOut;
+ this.data = dataOut;
+ this.numValues = numValues;
+ if (blockShift < 2 || blockShift > 30) {
+ throw new IllegalArgumentException("blockShift must be in [3-30], got " + blockShift);
+ }
+ final int blockSize = 1 << blockShift;
+ this.buffer = new long[blockSize];
+ this.bufferSize = 0;
+ this.baseDataPointer = dataOut.getFilePointer();
+ }
+
+ private void flush() throws IOException {
+ assert bufferSize != 0;
+
+ final float avgInc = (float) ((double) (buffer[bufferSize-1] - buffer[0]) / Math.max(1, bufferSize - 1));
+ for (int i = 0; i < bufferSize; ++i) {
+ final long expected = (long) (avgInc * (long) i);
+ buffer[i] -= expected;
+ }
+
+ long min = buffer[0];
+ for (int i = 1; i < bufferSize; ++i) {
+ min = Math.min(buffer[i], min);
+ }
+
+ long maxDelta = 0;
+ for (int i = 0; i < bufferSize; ++i) {
+ buffer[i] -= min;
+ // use | will change nothing when it comes to computing required bits
+ // but has the benefit of working fine with negative values too
+ // (in case of overflow)
+ maxDelta |= buffer[i];
+ }
+
+ meta.writeLong(min);
+ meta.writeInt(Float.floatToIntBits(avgInc));
+ meta.writeLong(data.getFilePointer() - baseDataPointer);
+ if (maxDelta == 0) {
+ meta.writeByte((byte) 0);
+ } else {
+ final int bitsRequired = DirectWriter.unsignedBitsRequired(maxDelta);
+ DirectWriter writer = DirectWriter.getInstance(data, bufferSize, bitsRequired);
+ for (int i = 0; i < bufferSize; ++i) {
+ writer.add(buffer[i]);
+ }
+ writer.finish();
+ meta.writeByte((byte) bitsRequired);
+ }
+ bufferSize = 0;
+ }
+
+ long previous = Long.MIN_VALUE;
+
+ /** Write a new value. Note that data might not make it to storage until
+ * {@link #finish()} is called.
+ * @throws IllegalArgumentException if values don't come in order */
+ public void add(long v) throws IOException {
+ if (v < previous) {
+ throw new IllegalArgumentException("Values do not come in order: " + previous + ", " + v);
+ }
+ if (bufferSize == buffer.length) {
+ flush();
+ }
+ buffer[bufferSize++] = v;
+ previous = v;
+ count++;
+ }
+
+ /** This must be called exactly once after all values have been {@link #add(long) added}. */
+ public void finish() throws IOException {
+ if (count != numValues) {
+ throw new IllegalStateException("Wrong number of values added, expected: " + numValues + ", got: " + count);
+ }
+ if (finished) {
+ throw new IllegalStateException("#finish has been called already");
+ }
+ flush();
+ finished = true;
+ }
+
+ /** Returns an instance suitable for encoding {@code numValues} into monotonic
+ * blocks of 2<sup>{@code blockShift}</sup> values. Metadata will be written
+ * to {@code metaOut} and actual data to {@code dataOut}. */
+ public static DirectMonotonicWriter getInstance(IndexOutput metaOut, IndexOutput dataOut, long numValues, int blockShift) {
+ return new DirectMonotonicWriter(metaOut, dataOut, numValues, blockShift);
+ }
+
+}
Property changes on: lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java (revision 1710831)
+++ lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java (working copy)
@@ -43,21 +43,29 @@
* {@code bitsPerValue} for each value
*/
public static LongValues getInstance(RandomAccessInput slice, int bitsPerValue) {
+ return getInstance(slice, bitsPerValue, 0);
+ }
+
+ /**
+ * Retrieves an instance from the specified {@code offset} of the given slice
+ * decoding {@code bitsPerValue} for each value
+ */
+ public static LongValues getInstance(RandomAccessInput slice, int bitsPerValue, long offset) {
switch (bitsPerValue) {
- case 1: return new DirectPackedReader1(slice);
- case 2: return new DirectPackedReader2(slice);
- case 4: return new DirectPackedReader4(slice);
- case 8: return new DirectPackedReader8(slice);
- case 12: return new DirectPackedReader12(slice);
- case 16: return new DirectPackedReader16(slice);
- case 20: return new DirectPackedReader20(slice);
- case 24: return new DirectPackedReader24(slice);
- case 28: return new DirectPackedReader28(slice);
- case 32: return new DirectPackedReader32(slice);
- case 40: return new DirectPackedReader40(slice);
- case 48: return new DirectPackedReader48(slice);
- case 56: return new DirectPackedReader56(slice);
- case 64: return new DirectPackedReader64(slice);
+ case 1: return new DirectPackedReader1(slice, offset);
+ case 2: return new DirectPackedReader2(slice, offset);
+ case 4: return new DirectPackedReader4(slice, offset);
+ case 8: return new DirectPackedReader8(slice, offset);
+ case 12: return new DirectPackedReader12(slice, offset);
+ case 16: return new DirectPackedReader16(slice, offset);
+ case 20: return new DirectPackedReader20(slice, offset);
+ case 24: return new DirectPackedReader24(slice, offset);
+ case 28: return new DirectPackedReader28(slice, offset);
+ case 32: return new DirectPackedReader32(slice, offset);
+ case 40: return new DirectPackedReader40(slice, offset);
+ case 48: return new DirectPackedReader48(slice, offset);
+ case 56: return new DirectPackedReader56(slice, offset);
+ case 64: return new DirectPackedReader64(slice, offset);
default: throw new IllegalArgumentException("unsupported bitsPerValue: " + bitsPerValue);
}
}
@@ -64,9 +72,11 @@
static final class DirectPackedReader1 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader1(RandomAccessInput in) {
+ DirectPackedReader1(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -73,7 +83,7 @@
public long get(long index) {
try {
int shift = 7 - (int) (index & 7);
- return (in.readByte(index >>> 3) >>> shift) & 0x1;
+ return (in.readByte(offset + (index >>> 3)) >>> shift) & 0x1;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -82,9 +92,11 @@
static final class DirectPackedReader2 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader2(RandomAccessInput in) {
+ DirectPackedReader2(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -91,7 +103,7 @@
public long get(long index) {
try {
int shift = (3 - (int)(index & 3)) << 1;
- return (in.readByte(index >>> 2) >>> shift) & 0x3;
+ return (in.readByte(offset + (index >>> 2)) >>> shift) & 0x3;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -100,9 +112,11 @@
static final class DirectPackedReader4 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader4(RandomAccessInput in) {
+ DirectPackedReader4(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -109,7 +123,7 @@
public long get(long index) {
try {
int shift = (int) ((index + 1) & 1) << 2;
- return (in.readByte(index >>> 1) >>> shift) & 0xF;
+ return (in.readByte(offset + (index >>> 1)) >>> shift) & 0xF;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -118,15 +132,17 @@
static final class DirectPackedReader8 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader8(RandomAccessInput in) {
+ DirectPackedReader8(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readByte(index) & 0xFF;
+ return in.readByte(offset + index) & 0xFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -135,9 +151,11 @@
static final class DirectPackedReader12 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader12(RandomAccessInput in) {
+ DirectPackedReader12(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -145,7 +163,7 @@
try {
long offset = (index * 12) >>> 3;
int shift = (int) ((index + 1) & 1) << 2;
- return (in.readShort(offset) >>> shift) & 0xFFF;
+ return (in.readShort(this.offset + offset) >>> shift) & 0xFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -154,15 +172,17 @@
static final class DirectPackedReader16 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader16(RandomAccessInput in) {
+ DirectPackedReader16(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readShort(index << 1) & 0xFFFF;
+ return in.readShort(offset + (index << 1)) & 0xFFFF;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -171,9 +191,11 @@
static final class DirectPackedReader20 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader20(RandomAccessInput in) {
+ DirectPackedReader20(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -181,7 +203,7 @@
try {
long offset = (index * 20) >>> 3;
// TODO: clean this up...
- int v = in.readInt(offset) >>> 8;
+ int v = in.readInt(this.offset + offset) >>> 8;
int shift = (int) ((index + 1) & 1) << 2;
return (v >>> shift) & 0xFFFFF;
} catch (IOException e) {
@@ -192,15 +214,17 @@
static final class DirectPackedReader24 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader24(RandomAccessInput in) {
+ DirectPackedReader24(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readInt(index * 3) >>> 8;
+ return in.readInt(offset + index * 3) >>> 8;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -209,9 +233,11 @@
static final class DirectPackedReader28 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader28(RandomAccessInput in) {
+ DirectPackedReader28(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
@@ -219,7 +245,7 @@
try {
long offset = (index * 28) >>> 3;
int shift = (int) ((index + 1) & 1) << 2;
- return (in.readInt(offset) >>> shift) & 0xFFFFFFFL;
+ return (in.readInt(this.offset + offset) >>> shift) & 0xFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -228,15 +254,17 @@
static final class DirectPackedReader32 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader32(RandomAccessInput in) {
+ DirectPackedReader32(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readInt(index << 2) & 0xFFFFFFFFL;
+ return in.readInt(this.offset + (index << 2)) & 0xFFFFFFFFL;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -245,15 +273,17 @@
static final class DirectPackedReader40 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader40(RandomAccessInput in) {
+ DirectPackedReader40(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readLong(index * 5) >>> 24;
+ return in.readLong(this.offset + index * 5) >>> 24;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -262,15 +292,17 @@
static final class DirectPackedReader48 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader48(RandomAccessInput in) {
+ DirectPackedReader48(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readLong(index * 6) >>> 16;
+ return in.readLong(this.offset + index * 6) >>> 16;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -279,15 +311,17 @@
static final class DirectPackedReader56 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader56(RandomAccessInput in) {
+ DirectPackedReader56(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readLong(index * 7) >>> 8;
+ return in.readLong(this.offset + index * 7) >>> 8;
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -296,15 +330,17 @@
static final class DirectPackedReader64 extends LongValues {
final RandomAccessInput in;
+ final long offset;
- DirectPackedReader64(RandomAccessInput in) {
+ DirectPackedReader64(RandomAccessInput in, long offset) {
this.in = in;
+ this.offset = offset;
}
@Override
public long get(long index) {
try {
- return in.readLong(index << 3);
+ return in.readLong(offset + (index << 3));
} catch (IOException e) {
throw new RuntimeException(e);
}
Index: lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
===================================================================
--- lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (revision 1710831)
+++ lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (working copy)
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.lucene53.Lucene53Codec
+org.apache.lucene.codecs.lucene54.Lucene54Codec
Index: lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
===================================================================
--- lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (revision 1710831)
+++ lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (working copy)
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat
+org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
Index: lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (revision 1710831)
+++ lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (working copy)
@@ -1,281 +0,0 @@
-package org.apache.lucene.codecs.lucene50;
-
-/*
- * 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.
- */
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.asserting.AssertingCodec;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedSetDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.TestUtil;
-
-/**
- * Tests Lucene50DocValuesFormat
- */
-public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
- private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
-
- @Override
- protected Codec getCodec() {
- return codec;
- }
-
- // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
- // for now just keep them here, as we want to test this for this format.
-
- @Slow
- public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
- }
- }
-
- @Nightly
- public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
- }
- }
-
- @Slow
- public void testSortedVariableLengthBigVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedVsStoredFields(atLeast(300), 1, 32766);
- }
- }
-
- @Nightly
- public void testSortedVariableLengthManyVsStoredFields() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
- }
- }
-
- @Slow
- public void testTermsEnumFixedWidth() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
- }
- }
-
- @Slow
- public void testTermsEnumVariableWidth() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
- }
- }
-
- @Nightly
- public void testTermsEnumRandomMany() throws Exception {
- int numIterations = atLeast(1);
- for (int i = 0; i < numIterations; i++) {
- doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
- }
- }
-
- // TODO: try to refactor this and some termsenum tests into the base class.
- // to do this we need to fix the test class to get a DVF not a Codec so we can setup
- // the postings format correctly.
- private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
- Directory dir = newFSDirectory(createTempDir());
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setMergeScheduler(new SerialMergeScheduler());
- // set to duel against a codec which has ordinals:
- final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
- final DocValuesFormat dv = new Lucene50DocValuesFormat();
- conf.setCodec(new AssertingCodec() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- return pf;
- }
-
- @Override
- public DocValuesFormat getDocValuesFormatForField(String field) {
- return dv;
- }
- });
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
-
- // index some docs
- for (int i = 0; i < numDocs; i++) {
- Document doc = new Document();
- Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
- doc.add(idField);
- final int length = TestUtil.nextInt(random(), minLength, maxLength);
- int numValues = random().nextInt(17);
- // create a random list of strings
- List<String> values = new ArrayList<>();
- for (int v = 0; v < numValues; v++) {
- values.add(TestUtil.randomSimpleString(random(), minLength, length));
- }
-
- // add in any order to the indexed field
- ArrayList<String> unordered = new ArrayList<>(values);
- Collections.shuffle(unordered, random());
- for (String v : values) {
- doc.add(newStringField("indexed", v, Field.Store.NO));
- }
-
- // add in any order to the dv field
- ArrayList<String> unordered2 = new ArrayList<>(values);
- Collections.shuffle(unordered2, random());
- for (String v : unordered2) {
- doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
- }
-
- writer.addDocument(doc);
- if (random().nextInt(31) == 0) {
- writer.commit();
- }
- }
-
- // delete some docs
- int numDeletions = random().nextInt(numDocs/10);
- for (int i = 0; i < numDeletions; i++) {
- int id = random().nextInt(numDocs);
- writer.deleteDocuments(new Term("id", Integer.toString(id)));
- }
-
- // compare per-segment
- DirectoryReader ir = writer.getReader();
- for (LeafReaderContext context : ir.leaves()) {
- LeafReader r = context.reader();
- Terms terms = r.terms("indexed");
- if (terms != null) {
- SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
- assertEquals(terms.size(), ssdv.getValueCount());
- TermsEnum expected = terms.iterator();
- TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
- assertEquals(terms.size(), expected, actual);
-
- doTestSortedSetEnumAdvanceIndependently(ssdv);
- }
- }
- ir.close();
-
- writer.forceMerge(1);
-
- // now compare again after the merge
- ir = writer.getReader();
- LeafReader ar = getOnlySegmentReader(ir);
- Terms terms = ar.terms("indexed");
- if (terms != null) {
- assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
- TermsEnum expected = terms.iterator();
- TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
- assertEquals(terms.size(), expected, actual);
- }
- ir.close();
-
- writer.close();
- dir.close();
- }
-
- private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
- BytesRef ref;
-
- // sequential next() through all terms
- while ((ref = expected.next()) != null) {
- assertEquals(ref, actual.next());
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
- assertNull(actual.next());
-
- // sequential seekExact(ord) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- actual.seekExact(i);
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // sequential seekExact(BytesRef) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- assertTrue(actual.seekExact(expected.term()));
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // sequential seekCeil(BytesRef) through all terms
- for (long i = 0; i < numOrds; i++) {
- expected.seekExact(i);
- assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekExact(ord)
- for (long i = 0; i < numOrds; i++) {
- long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
- expected.seekExact(randomOrd);
- actual.seekExact(randomOrd);
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekExact(BytesRef)
- for (long i = 0; i < numOrds; i++) {
- long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
- expected.seekExact(randomOrd);
- actual.seekExact(expected.term());
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
-
- // random seekCeil(BytesRef)
- for (long i = 0; i < numOrds; i++) {
- BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
- SeekStatus expectedStatus = expected.seekCeil(target);
- assertEquals(expectedStatus, actual.seekCeil(target));
- if (expectedStatus != SeekStatus.END) {
- assertEquals(expected.ord(), actual.ord());
- assertEquals(expected.term(), actual.term());
- }
- }
- }
-}
Index: lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (revision 1710831)
+++ lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (working copy)
@@ -19,7 +19,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@@ -34,7 +34,7 @@
public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
- return new Lucene53Codec(Mode.BEST_COMPRESSION);
+ return new Lucene54Codec(Mode.BEST_COMPRESSION);
}
/**
@@ -45,7 +45,7 @@
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
- iwc.setCodec(new Lucene53Codec(RandomPicks.randomFrom(random(), Mode.values())));
+ iwc.setCodec(new Lucene54Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
@@ -72,7 +72,7 @@
public void testInvalidOptions() throws Exception {
try {
- new Lucene53Codec(null);
+ new Lucene54Codec(null);
fail("didn't hit exception");
} catch (NullPointerException expected) {
// expected
Index: lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java (revision 1710831)
+++ lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java (working copy)
@@ -18,6 +18,7 @@
*/
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/**
@@ -24,7 +25,7 @@
* Tests Lucene53NormsFormat
*/
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
- private final Codec codec = new Lucene53Codec();
+ private final Codec codec = new Lucene54Codec();
@Override
protected Codec getCodec() {
Index: lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java (revision 0)
+++ lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java (working copy)
@@ -0,0 +1,281 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene54DocValuesFormat
+ */
+public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+ private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat());
+
+ @Override
+ protected Codec getCodec() {
+ return codec;
+ }
+
+ // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+ // for now just keep them here, as we want to test this for this format.
+
+ @Slow
+ public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+ }
+ }
+
+ @Nightly
+ public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
+ }
+ }
+
+ @Slow
+ public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedVsStoredFields(atLeast(300), 1, 32766);
+ }
+ }
+
+ @Nightly
+ public void testSortedVariableLengthManyVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
+ }
+ }
+
+ @Slow
+ public void testTermsEnumFixedWidth() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+ }
+ }
+
+ @Slow
+ public void testTermsEnumVariableWidth() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
+ }
+ }
+
+ @Nightly
+ public void testTermsEnumRandomMany() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
+ }
+ }
+
+ // TODO: try to refactor this and some termsenum tests into the base class.
+ // to do this we need to fix the test class to get a DVF not a Codec so we can setup
+ // the postings format correctly.
+ private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
+ Directory dir = newFSDirectory(createTempDir());
+ IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ conf.setMergeScheduler(new SerialMergeScheduler());
+ // set to duel against a codec which has ordinals:
+ final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
+ final DocValuesFormat dv = new Lucene54DocValuesFormat();
+ conf.setCodec(new AssertingCodec() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return pf;
+ }
+
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return dv;
+ }
+ });
+ RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+
+ // index some docs
+ for (int i = 0; i < numDocs; i++) {
+ Document doc = new Document();
+ Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+ doc.add(idField);
+ final int length = TestUtil.nextInt(random(), minLength, maxLength);
+ int numValues = random().nextInt(17);
+ // create a random list of strings
+ List<String> values = new ArrayList<>();
+ for (int v = 0; v < numValues; v++) {
+ values.add(TestUtil.randomSimpleString(random(), minLength, length));
+ }
+
+ // add in any order to the indexed field
+ ArrayList<String> unordered = new ArrayList<>(values);
+ Collections.shuffle(unordered, random());
+ for (String v : values) {
+ doc.add(newStringField("indexed", v, Field.Store.NO));
+ }
+
+ // add in any order to the dv field
+ ArrayList<String> unordered2 = new ArrayList<>(values);
+ Collections.shuffle(unordered2, random());
+ for (String v : unordered2) {
+ doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
+ }
+
+ writer.addDocument(doc);
+ if (random().nextInt(31) == 0) {
+ writer.commit();
+ }
+ }
+
+ // delete some docs
+ int numDeletions = random().nextInt(numDocs/10);
+ for (int i = 0; i < numDeletions; i++) {
+ int id = random().nextInt(numDocs);
+ writer.deleteDocuments(new Term("id", Integer.toString(id)));
+ }
+
+ // compare per-segment
+ DirectoryReader ir = writer.getReader();
+ for (LeafReaderContext context : ir.leaves()) {
+ LeafReader r = context.reader();
+ Terms terms = r.terms("indexed");
+ if (terms != null) {
+ SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
+ assertEquals(terms.size(), ssdv.getValueCount());
+ TermsEnum expected = terms.iterator();
+ TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+ assertEquals(terms.size(), expected, actual);
+
+ doTestSortedSetEnumAdvanceIndependently(ssdv);
+ }
+ }
+ ir.close();
+
+ writer.forceMerge(1);
+
+ // now compare again after the merge
+ ir = writer.getReader();
+ LeafReader ar = getOnlySegmentReader(ir);
+ Terms terms = ar.terms("indexed");
+ if (terms != null) {
+ assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
+ TermsEnum expected = terms.iterator();
+ TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
+ assertEquals(terms.size(), expected, actual);
+ }
+ ir.close();
+
+ writer.close();
+ dir.close();
+ }
+
+ private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
+ BytesRef ref;
+
+ // sequential next() through all terms
+ while ((ref = expected.next()) != null) {
+ assertEquals(ref, actual.next());
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+ assertNull(actual.next());
+
+ // sequential seekExact(ord) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ actual.seekExact(i);
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // sequential seekExact(BytesRef) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ assertTrue(actual.seekExact(expected.term()));
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // sequential seekCeil(BytesRef) through all terms
+ for (long i = 0; i < numOrds; i++) {
+ expected.seekExact(i);
+ assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekExact(ord)
+ for (long i = 0; i < numOrds; i++) {
+ long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+ expected.seekExact(randomOrd);
+ actual.seekExact(randomOrd);
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekExact(BytesRef)
+ for (long i = 0; i < numOrds; i++) {
+ long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+ expected.seekExact(randomOrd);
+ actual.seekExact(expected.term());
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+
+ // random seekCeil(BytesRef)
+ for (long i = 0; i < numOrds; i++) {
+ BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
+ SeekStatus expectedStatus = expected.seekCeil(target);
+ assertEquals(expectedStatus, actual.seekCeil(target));
+ if (expectedStatus != SeekStatus.END) {
+ assertEquals(expected.ord(), actual.ord());
+ assertEquals(expected.term(), actual.term());
+ }
+ }
+ }
+}
Property changes on: lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java (revision 0)
+++ lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java (working copy)
@@ -0,0 +1,137 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDirectMonotonic extends LuceneTestCase {
+
+ public void testSimple() throws IOException {
+ Directory dir = newDirectory();
+ final int blockShift = 2;
+
+ List<Long> actualValues = Arrays.asList(1L, 2L, 5L, 7L, 8L, 100L);
+ final int numValues = actualValues.size();
+
+ final long dataLength;
+ try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+ IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+ DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+ for (long v : actualValues) {
+ w.add(v);
+ }
+ w.finish();
+ dataLength = dataOut.getFilePointer();
+ }
+
+ try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+ IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+ DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+ LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+ for (int i = 0; i < numValues; ++i) {
+ final long v = values.get(i);
+ assertEquals(actualValues.get(i).longValue(), v);
+ }
+ }
+
+ dir.close();
+ }
+
+ public void testConstantSlope() throws IOException {
+ Directory dir = newDirectory();
+ final int blockShift = TestUtil.nextInt(random(), DirectMonotonicWriter.MIN_BLOCK_SHIFT, DirectMonotonicWriter.MAX_BLOCK_SHIFT);
+ final int numValues = TestUtil.nextInt(random(), 1, 1 << 20);
+ final long min = random().nextLong();
+ final long inc = random().nextInt(1 << random().nextInt(20));
+
+ List<Long> actualValues = new ArrayList<>();
+ for (int i = 0; i < numValues; ++i) {
+ actualValues.add(min + inc * i);
+ }
+
+ final long dataLength;
+ try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+ IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+ DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+ for (long v : actualValues) {
+ w.add(v);
+ }
+ w.finish();
+ dataLength = dataOut.getFilePointer();
+ }
+
+ try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+ IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+ DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+ LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+ for (int i = 0; i < numValues; ++i) {
+ assertEquals(actualValues.get(i).longValue(), values.get(i));
+ }
+ assertEquals(0, dataIn.getFilePointer());
+ }
+
+ dir.close();
+ }
+
+ public void testRandom() throws IOException {
+ Directory dir = newDirectory();
+ final int blockShift = TestUtil.nextInt(random(), DirectMonotonicWriter.MIN_BLOCK_SHIFT, DirectMonotonicWriter.MAX_BLOCK_SHIFT);
+ final int numValues = TestUtil.nextInt(random(), 1, 1 << 20);
+ List<Long> actualValues = new ArrayList<>();
+ long previous = random().nextLong();
+ actualValues.add(previous);
+ for (int i = 1; i < numValues; ++i) {
+ previous += random().nextInt(1 << random().nextInt(20));
+ actualValues.add(previous);
+ }
+
+ final long dataLength;
+ try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+ IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+ DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+ for (long v : actualValues) {
+ w.add(v);
+ }
+ w.finish();
+ dataLength = dataOut.getFilePointer();
+ }
+
+ try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+ IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+ DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+ LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+ for (int i = 0; i < numValues; ++i) {
+ assertEquals(actualValues.get(i).longValue(), values.get(i));
+ }
+ }
+
+ dir.close();
+ }
+
+}
Property changes on: lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java (revision 1710831)
+++ lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java (working copy)
@@ -26,6 +26,7 @@
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.DirectReader;
import org.apache.lucene.util.packed.DirectWriter;
@@ -45,7 +46,7 @@
writer.finish();
output.close();
IndexInput input = dir.openInput("foo", IOContext.DEFAULT);
- NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsPerValue);
+ NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsPerValue, 0);
assertEquals(1, reader.get(0));
assertEquals(0, reader.get(1));
assertEquals(2, reader.get(2));
@@ -78,12 +79,21 @@
public void testRandom() throws Exception {
Directory dir = newDirectory();
for (int bpv = 1; bpv <= 64; bpv++) {
- doTestBpv(dir, bpv);
+ doTestBpv(dir, bpv, 0);
}
dir.close();
}
-
- private void doTestBpv(Directory directory, int bpv) throws Exception {
+
+ public void testRandomWithOffset() throws Exception {
+ Directory dir = newDirectory();
+ final int offset = TestUtil.nextInt(random(), 1, 100);
+ for (int bpv = 1; bpv <= 64; bpv++) {
+ doTestBpv(dir, bpv, offset);
+ }
+ dir.close();
+ }
+
+ private void doTestBpv(Directory directory, int bpv, long offset) throws Exception {
MyRandom random = new MyRandom(random().nextLong());
int numIters = TEST_NIGHTLY ? 100 : 10;
for (int i = 0; i < numIters; i++) {
@@ -91,6 +101,9 @@
int bitsRequired = bpv == 64 ? 64 : DirectWriter.bitsRequired(1L<<(bpv-1));
String name = "bpv" + bpv + "_" + i;
IndexOutput output = directory.createOutput(name, IOContext.DEFAULT);
+ for (long j = 0; j < offset; ++j) {
+ output.writeByte((byte) random().nextInt());
+ }
DirectWriter writer = DirectWriter.getInstance(output, original.length, bitsRequired);
for (int j = 0; j < original.length; j++) {
writer.add(original[j]);
@@ -98,7 +111,7 @@
writer.finish();
output.close();
IndexInput input = directory.openInput(name, IOContext.DEFAULT);
- NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsRequired);
+ NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsRequired, offset);
for (int j = 0; j < original.length; j++) {
assertEquals("bpv=" + bpv, original[j], reader.get(j));
}
Index: lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java
===================================================================
--- lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java (revision 1710831)
+++ lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java (working copy)
@@ -22,7 +22,7 @@
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@@ -31,7 +31,7 @@
* from {@link BKDPointField} for fast bounding-box ({@link BKDPointInBBoxQuery})
* and polygon ({@link BKDPointInPolygonQuery}) queries.
*
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own BKD tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own BKD tree
* structures to disk for fast query-time intersection. See <a
* href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
* for details.
@@ -50,7 +50,7 @@
* <p>The index is also quite compact, because docs only appear once in
* the tree (no "prefix terms").
*
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
* <ol>
* <li><tt>.kdd</tt>: BKD leaf data and index</li>
* <li><tt>.kdm</tt>: BKD metadata</li>
@@ -75,7 +75,7 @@
private final int maxPointsInLeafNode;
private final int maxPointsSortInHeap;
- private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+ private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
/** Default constructor */
public BKDTreeDocValuesFormat() {
Index: lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java
===================================================================
--- lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java (revision 1710831)
+++ lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java (working copy)
@@ -20,7 +20,7 @@
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField; // javadocs
import org.apache.lucene.index.SegmentReadState;
@@ -34,7 +34,7 @@
* for numeric range queries using ({@link NumericRangeTreeQuery}) and arbitrary binary
* range queries using {@link SortedSetRangeTreeQuery}.
*
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own numeric tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own numeric tree
* structures to disk for fast query-time intersection. See <a
* href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
* for details.
@@ -53,7 +53,7 @@
* <p>The index is also quite compact, because docs only appear once in
* the tree (no "prefix terms").
*
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
* <ol>
* <li><tt>.ndd</tt>: numeric tree leaf data and index</li>
* <li><tt>.ndm</tt>: numeric tree metadata</li>
@@ -78,7 +78,7 @@
private final int maxPointsInLeafNode;
private final int maxPointsSortInHeap;
- private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+ private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
/** Default constructor */
public RangeTreeDocValuesFormat() {
Index: lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
===================================================================
--- lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (revision 1710831)
+++ lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (working copy)
@@ -17,18 +17,9 @@
* limitations under the License.
*/
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
@@ -37,7 +28,6 @@
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BaseGeoPointTestCase;
@@ -79,7 +69,7 @@
@Override
protected void initIndexWriterConfig(final String fieldName, IndexWriterConfig iwc) {
final DocValuesFormat dvFormat = getDocValuesFormat();
- Codec codec = new Lucene53Codec() {
+ Codec codec = new Lucene54Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
if (field.equals(fieldName)) {
Index: lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
===================================================================
--- lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java (revision 1710831)
+++ lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java (working copy)
@@ -27,7 +27,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
@@ -356,7 +356,7 @@
iwc.setMaxBufferedDocs(values.length/100);
}
final DocValuesFormat dvFormat = getDocValuesFormat();
- Codec codec = new Lucene53Codec() {
+ Codec codec = new Lucene54Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
if (field.equals("sn_value") || field.equals("ss_value")) {
Index: lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
===================================================================
--- lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (revision 1710831)
+++ lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (working copy)
@@ -20,7 +20,7 @@
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
import org.apache.lucene.geo3d.PlanetModel;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@@ -32,7 +32,7 @@
* from {@link Geo3DPointField} for fast shape intersection queries using
* ({@link PointInGeo3DShapeQuery})
*
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own BKD tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own BKD tree
* structures to disk for fast query-time intersection. See <a
* href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
* for details.
@@ -51,7 +51,7 @@
* <p>The index is also quite compact, because docs only appear once in
* the tree (no "prefix terms").
*
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
* <ol>
* <li><tt>.kd3d</tt>: BKD leaf data and index</li>
* <li><tt>.kd3m</tt>: BKD metadata</li>
@@ -77,7 +77,7 @@
private final int maxPointsInLeafNode;
private final int maxPointsSortInHeap;
- private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+ private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
private final PlanetModel planetModel;
Index: lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
===================================================================
--- lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (revision 1710831)
+++ lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (working copy)
@@ -29,7 +29,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
@@ -910,7 +910,7 @@
iwc.setMaxBufferedDocs(lats.length/100);
}
final DocValuesFormat dvFormat = new Geo3DDocValuesFormat(planetModel, maxPointsInLeaf, maxPointsSortInHeap);
- Codec codec = new Lucene53Codec() {
+ Codec codec = new Lucene54Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
if (field.equals("point")) {
Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
===================================================================
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java (revision 1710831)
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java (working copy)
@@ -31,7 +31,7 @@
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.IntField;
@@ -631,7 +631,7 @@
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
iwc.setMergePolicy(newLogMergePolicy());
- Codec filterCodec = new Lucene53Codec() {
+ Codec filterCodec = new Lucene54Codec() {
PostingsFormat postingsFormat = new Completion50PostingsFormat();
@Override
Index: lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (revision 1710831)
+++ lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (working copy)
@@ -33,7 +33,7 @@
import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@@ -182,8 +182,8 @@
codec = new AssertingCodec();
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
codec = CompressingCodec.randomInstance(random);
- } else if ("Lucene53".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene53"))) {
- codec = new Lucene53Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
+ } else if ("Lucene54".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene54"))) {
+ codec = new Lucene54Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
} else if (!"random".equals(TEST_CODEC)) {
codec = Codec.forName(TEST_CODEC);
} else if ("random".equals(TEST_POSTINGSFORMAT)) {
Index: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (revision 1710831)
+++ lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (working copy)
@@ -54,9 +54,9 @@
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@@ -879,7 +879,7 @@
* This may be different than {@link Codec#getDefault()} because that is randomized.
*/
public static Codec getDefaultCodec() {
- return new Lucene53Codec();
+ return new Lucene54Codec();
}
/**
@@ -912,7 +912,7 @@
* Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
*/
public static DocValuesFormat getDefaultDocValuesFormat() {
- return new Lucene50DocValuesFormat();
+ return new Lucene54DocValuesFormat();
}
// TODO: generalize all 'test-checks-for-crazy-codecs' to
Index: solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
===================================================================
--- solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (revision 1710831)
+++ solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (working copy)
@@ -3,7 +3,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.util.plugin.SolrCoreAware;
@@ -51,7 +51,7 @@
@Override
public void init(NamedList args) {
super.init(args);
- codec = new Lucene53Codec() {
+ codec = new Lucene54Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);
Index: solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
===================================================================
--- solr/core/src/test-files/solr/collection1/conf/schema_codec.xml (revision 1710831)
+++ solr/core/src/test-files/solr/collection1/conf/schema_codec.xml (working copy)
@@ -21,7 +21,7 @@
<fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
- <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene50" />
+ <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene54" />
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory" />
<fieldType name="string" class="solr.StrField" />