From 373d0074741dd25f4329036896678f6bded6b81d Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Thu, 16 Oct 2014 01:20:58 +0000
Subject: [PATCH 01/19] LUCENE-5969: add 5.10 dv with segment header, CONST
optimization, and missingBits ghostbuster
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632200 13f79535-47bb-0310-9956-ffa450edef68
---
.../codecs/lucene410/Lucene410Codec.java | 10 +-
.../lucene410/Lucene410DocValuesConsumer.java | 22 +-
.../lucene410/Lucene410DocValuesFormat.java | 61 +
.../lucene410/Lucene410DocValuesProducer.java | 6 +-
.../org.apache.lucene.codecs.DocValuesFormat | 2 +
.../codecs/lucene410/Lucene410RWCodec.java | 71 +
.../lucene410/Lucene410RWDocValuesFormat.java | 42 +
.../TestLucene410DocValuesFormat.java | 4 +-
.../codecs/lucene45/Lucene45RWCodec.java | 3 +-
.../TestDocValuesUpdatesOnOldSegments.java | 95 +-
.../lucene/codecs/lucene410/package.html | 25 -
.../lucene/codecs/lucene50/Lucene50Codec.java | 4 +-
.../lucene50/Lucene50DocValuesConsumer.java | 586 +++++++++
.../Lucene50DocValuesFormat.java} | 22 +-
.../lucene50/Lucene50DocValuesProducer.java | 1153 +++++++++++++++++
.../lucene/codecs/lucene50/package.html | 4 +-
.../org.apache.lucene.codecs.DocValuesFormat | 2 +-
.../lucene50/TestLucene50DocValuesFormat.java | 285 ++++
.../index/BaseDocValuesFormatTestCase.java | 2 +
.../java/org/apache/lucene/util/TestUtil.java | 4 +-
20 files changed, 2308 insertions(+), 95 deletions(-)
rename lucene/{core => backward-codecs}/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java (97%)
create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
rename lucene/{core => backward-codecs}/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java (99%)
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWDocValuesFormat.java
rename lucene/{core => backward-codecs}/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java (98%)
delete mode 100755 lucene/core/src/java/org/apache/lucene/codecs/lucene410/package.html
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
rename lucene/core/src/java/org/apache/lucene/codecs/{lucene410/Lucene410DocValuesFormat.java => lucene50/Lucene50DocValuesFormat.java} (93%)
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java
index 8455cd1f765..6c40db914b9 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java
@@ -74,12 +74,12 @@ public class Lucene410Codec extends Codec {
}
@Override
- public final StoredFieldsFormat storedFieldsFormat() {
+ public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
- public final TermVectorsFormat termVectorsFormat() {
+ public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@@ -94,7 +94,7 @@ public class Lucene410Codec extends Codec {
}
@Override
- public final SegmentInfoFormat segmentInfoFormat() {
+ public SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@@ -127,7 +127,7 @@ public class Lucene410Codec extends Codec {
}
@Override
- public final DocValuesFormat docValuesFormat() {
+ public DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@@ -142,7 +142,7 @@ public class Lucene410Codec extends Codec {
};
@Override
- public final NormsFormat normsFormat() {
+ public NormsFormat normsFormat() {
return normsFormat;
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
similarity index 97%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
index 301ee04448a..c230d33c538 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
@@ -28,6 +28,7 @@ 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.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
@@ -41,7 +42,11 @@ import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
-/** writer for {@link Lucene410DocValuesFormat} */
+/**
+ * writer for 4.10 docvalues format
+ * @deprecated only for old 4.x segments
+ */
+@Deprecated
class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable {
static final int BLOCK_SIZE = 16384;
@@ -108,6 +113,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addNumericField(FieldInfo field, Iterable values) throws IOException {
+ checkCanWrite(field);
addNumericField(field, values, true);
}
@@ -262,6 +268,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addBinaryField(FieldInfo field, Iterable values) throws IOException {
+ checkCanWrite(field);
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.BINARY);
@@ -466,6 +473,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedField(FieldInfo field, Iterable values, Iterable docToOrd) throws IOException {
+ checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED);
addTermsDict(field, values);
@@ -474,6 +482,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedNumericField(FieldInfo field, final Iterable docToValueCount, final Iterable values) throws IOException {
+ checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED_NUMERIC);
if (isSingleValued(docToValueCount)) {
@@ -491,6 +500,7 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
@Override
public void addSortedSetField(FieldInfo field, Iterable values, final Iterable docToOrdCount, final Iterable ords) throws IOException {
+ checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene410DocValuesFormat.SORTED_SET);
@@ -556,4 +566,14 @@ class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable
meta = data = null;
}
}
+
+ void checkCanWrite(FieldInfo field) {
+ if ((field.getDocValuesType() == DocValuesType.NUMERIC ||
+ field.getDocValuesType() == DocValuesType.BINARY) &&
+ field.getDocValuesGen() != -1) {
+ // ok
+ } else {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+ }
}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
new file mode 100644
index 00000000000..472ff9ab085
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
@@ -0,0 +1,61 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * 4.10 docvalues format
+ * @deprecated only for old 4.x segments
+ */
+@Deprecated
+public class Lucene410DocValuesFormat extends DocValuesFormat {
+
+ /** Sole Constructor */
+ public Lucene410DocValuesFormat() {
+ super("Lucene410");
+ }
+
+ @Override
+ public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ @Override
+ public final DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+ return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ }
+
+ static final String DATA_CODEC = "Lucene410DocValuesData";
+ static final String DATA_EXTENSION = "dvd";
+ static final String META_CODEC = "Lucene410ValuesMetadata";
+ static final String META_EXTENSION = "dvm";
+ static final int VERSION_START = 0;
+ static final int VERSION_CURRENT = VERSION_START;
+ 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;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
similarity index 99%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
index d68f4b0b71c..fafea3a8443 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java
@@ -74,7 +74,11 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DirectReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-/** reader for {@link Lucene410DocValuesFormat} */
+/**
+ * reader for 4.10 docvalues format
+ * @deprecated only for old 4.x segments
+ */
+@Deprecated
class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map numerics = new HashMap<>();
private final Map binaries = new HashMap<>();
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 01ce305b441..574c9c564a2 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -16,3 +16,5 @@
org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat
+org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat
+
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
new file mode 100644
index 00000000000..22f891d510e
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
@@ -0,0 +1,71 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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 org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
+import org.apache.lucene.codecs.lucene49.Lucene49RWNormsFormat;
+
+/**
+ * Read-Write version of 4.10 codec for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public final class Lucene410RWCodec extends Lucene410Codec {
+
+ private static final DocValuesFormat docValues = new Lucene410RWDocValuesFormat();
+
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return docValues;
+ }
+
+ private static final NormsFormat norms = new Lucene49RWNormsFormat();
+
+ @Override
+ public NormsFormat normsFormat() {
+ return norms;
+ }
+
+ private static final SegmentInfoFormat segmentInfos = new Lucene46RWSegmentInfoFormat();
+
+ @Override
+ public SegmentInfoFormat segmentInfoFormat() {
+ return segmentInfos;
+ }
+
+ private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
+
+ @Override
+ public StoredFieldsFormat storedFieldsFormat() {
+ return storedFields;
+ }
+
+ private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWDocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWDocValuesFormat.java
new file mode 100644
index 00000000000..02d7b1e16c1
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWDocValuesFormat.java
@@ -0,0 +1,42 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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.index.FieldInfo;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * Read-Write version of 4.10 docvalues format for testing
+ * @deprecated for test purposes only
+ */
+class Lucene410RWDocValuesFormat extends Lucene410DocValuesFormat {
+
+ @Override
+ public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
+ @Override
+ void checkCanWrite(FieldInfo field) {
+ // allow writing all fields
+ }
+ };
+ }
+
+}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
similarity index 98%
rename from lucene/core/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
index ca17aa89a1d..f612f4a975f 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
@@ -51,7 +51,7 @@ import org.apache.lucene.util.TestUtil;
* Tests Lucene410DocValuesFormat
*/
public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
- private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene410DocValuesFormat());
+ private final Codec codec = new Lucene410RWCodec();
@Override
protected Codec getCodec() {
@@ -132,7 +132,7 @@ public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormat
// break;
default: throw new AssertionError();
}
- final DocValuesFormat dv = new Lucene410DocValuesFormat();
+ final DocValuesFormat dv = new Lucene410RWDocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
index a2b2ef45964..6752eb1c7b4 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
@@ -31,8 +31,9 @@ import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
/**
* Read-write version of {@link Lucene45Codec} for testing.
+ * @deprecated for test purposes only
*/
-@SuppressWarnings("deprecation")
+@Deprecated
public final class Lucene45RWCodec extends Lucene45Codec {
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestDocValuesUpdatesOnOldSegments.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestDocValuesUpdatesOnOldSegments.java
index 46f983b51ae..188bc0d4ba2 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestDocValuesUpdatesOnOldSegments.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestDocValuesUpdatesOnOldSegments.java
@@ -32,7 +32,10 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
-
+/**
+ * Tests performing docvalues updates against versions of lucene
+ * that did not support it.
+ */
public class TestDocValuesUpdatesOnOldSegments extends LuceneTestCase {
static long getValue(BinaryDocValues bdv, int idx) {
@@ -60,56 +63,62 @@ public class TestDocValuesUpdatesOnOldSegments extends LuceneTestCase {
public void testBinaryUpdates() throws Exception {
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec(), new Lucene45RWCodec() };
- Directory dir = newDirectory();
-
- // create a segment with an old Codec
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setCodec(oldCodecs[random().nextInt(oldCodecs.length)]);
- IndexWriter writer = new IndexWriter(dir, conf);
- Document doc = new Document();
- doc.add(new StringField("id", "doc", Store.NO));
- doc.add(new BinaryDocValuesField("f", toBytes(5L)));
- writer.addDocument(doc);
- writer.close();
-
- conf = newIndexWriterConfig(new MockAnalyzer(random()));
- writer = new IndexWriter(dir, conf);
- writer.updateBinaryDocValue(new Term("id", "doc"), "f", toBytes(4L));
- try {
+
+ for (Codec codec : oldCodecs) {
+ Directory dir = newDirectory();
+
+ // create a segment with an old Codec
+ IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ conf.setCodec(codec);
+ IndexWriter writer = new IndexWriter(dir, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "doc", Store.NO));
+ doc.add(new BinaryDocValuesField("f", toBytes(5L)));
+ writer.addDocument(doc);
writer.close();
- fail("should not have succeeded to update a segment written with an old Codec");
- } catch (UnsupportedOperationException e) {
- writer.rollback();
+
+ conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ writer = new IndexWriter(dir, conf);
+ writer.updateBinaryDocValue(new Term("id", "doc"), "f", toBytes(4L));
+ try {
+ writer.close();
+ fail("should not have succeeded to update a segment written with an old Codec");
+ } catch (UnsupportedOperationException e) {
+ writer.rollback();
+ }
+
+ dir.close();
}
-
- dir.close();
}
public void testNumericUpdates() throws Exception {
Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec(), new Lucene45RWCodec() };
- Directory dir = newDirectory();
-
- // create a segment with an old Codec
- IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
- conf.setCodec(oldCodecs[random().nextInt(oldCodecs.length)]);
- IndexWriter writer = new IndexWriter(dir, conf);
- Document doc = new Document();
- doc.add(new StringField("id", "doc", Store.NO));
- doc.add(new NumericDocValuesField("f", 5));
- writer.addDocument(doc);
- writer.close();
-
- conf = newIndexWriterConfig(new MockAnalyzer(random()));
- writer = new IndexWriter(dir, conf);
- writer.updateNumericDocValue(new Term("id", "doc"), "f", 4L);
- try {
+
+ for (Codec codec : oldCodecs) {
+ Directory dir = newDirectory();
+
+ // create a segment with an old Codec
+ IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ conf.setCodec(codec);
+ IndexWriter writer = new IndexWriter(dir, conf);
+ Document doc = new Document();
+ doc.add(new StringField("id", "doc", Store.NO));
+ doc.add(new NumericDocValuesField("f", 5));
+ writer.addDocument(doc);
writer.close();
- fail("should not have succeeded to update a segment written with an old Codec");
- } catch (UnsupportedOperationException e) {
- writer.rollback();
+
+ conf = newIndexWriterConfig(new MockAnalyzer(random()));
+ writer = new IndexWriter(dir, conf);
+ writer.updateNumericDocValue(new Term("id", "doc"), "f", 4L);
+ try {
+ writer.close();
+ fail("should not have succeeded to update a segment written with an old Codec");
+ } catch (UnsupportedOperationException e) {
+ writer.rollback();
+ }
+
+ dir.close();
}
-
- dir.close();
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene410/package.html
deleted file mode 100755
index cecf6b8316e..00000000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-
-
-
-
-
-
-
-Lucene 4.10 file format.
-
-
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
index e290a4a77de..b74c4bebd2d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
@@ -115,7 +115,7 @@ public class Lucene50Codec extends Codec {
/** Returns the docvalues format that should be used for writing
* new segments of field
.
*
- * The default implementation always returns "Lucene410"
+ * The default implementation always returns "Lucene50"
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
@@ -127,7 +127,7 @@ public class Lucene50Codec extends Codec {
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
- private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
+ private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
private final NormsFormat normsFormat = new Lucene50NormsFormat();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
new file mode 100644
index 00000000000..874efa7b6f1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
@@ -0,0 +1,586 @@
+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 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.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;
+
+/** writer for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+ static final int BLOCK_SIZE = 16384;
+
+ // 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. */
+ public static final int DELTA_COMPRESSED = 0;
+ /** Compressed by computing the GCD. */
+ public static final int GCD_COMPRESSED = 1;
+ /** Compressed by giving IDs to unique values. */
+ public static final int TABLE_COMPRESSED = 2;
+ /** Compressed with monotonically increasing values */
+ public static final int MONOTONIC_COMPRESSED = 3;
+ /** Compressed with constant value (uses only missing bitset) */
+ public static final int CONST_COMPRESSED = 4;
+
+ /** Uncompressed binary, written directly (fixed length). */
+ public static final int BINARY_FIXED_UNCOMPRESSED = 0;
+ /** Uncompressed binary, written directly (variable length). */
+ public static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+ /** Compressed binary with shared prefixes */
+ public static final int BINARY_PREFIX_COMPRESSED = 2;
+
+ /** Standard storage for sorted set values with 1 level of indirection:
+ * docId -> address -> ord. */
+ public static final int SORTED_WITH_ADDRESSES = 0;
+ /** Single-valued sorted set values, encoded as sorted values, so no level
+ * of indirection: docId -> ord. */
+ public static final int SORTED_SINGLE_VALUED = 1;
+
+ /** placeholder for missing offset that means there are no missing values */
+ public static final int ALL_LIVE = -1;
+ /** placeholder for missing offset that means all values are missing */
+ public static final int ALL_MISSING = -2;
+
+ 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.writeSegmentHeader(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.writeSegmentHeader(meta, metaCodec, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+ maxDoc = state.segmentInfo.getDocCount();
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(this);
+ }
+ }
+ }
+
+ @Override
+ public void addNumericField(FieldInfo field, Iterable values) throws IOException {
+ addNumericField(field, values, true);
+ }
+
+ void addNumericField(FieldInfo field, Iterable 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 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 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 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, its 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(BLOCK_SIZE);
+
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, 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 values) throws IOException {
+ // first check if its 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 its not much and less overall RAM than the previous approach!
+ RAMOutputStream addressBuffer = new RAMOutputStream();
+ MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, 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(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 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, 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 values, Iterable 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 docToValueCount, final Iterable 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 {
+ 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 values, final Iterable docToOrdCount, final Iterable 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 {
+ 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);
+ }
+ }
+
+ // writes addressing information as MONOTONIC_COMPRESSED integer
+ private void addAddresses(FieldInfo field, Iterable 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(BLOCK_SIZE);
+
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, 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;
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
similarity index 93%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
index d2014e497c7..6a167b72d04 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene410;
+package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -33,7 +33,7 @@ import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
/**
- * Lucene 4.10 DocValues format.
+ * Lucene 5.0 DocValues format.
*
* Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
*
@@ -49,6 +49,8 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
* common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
*
Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
* as blocks of bitpacked integers, encoding the deviation from the expected delta.
+ * Const-compressed: when there is only one possible non-missing value, only the missing
+ * bitset is encoded.
*
*
* {@link DocValuesType#BINARY BINARY}:
@@ -145,7 +147,7 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
* Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
* is written for the addresses.
*
MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
- * If its -1, then there are no missing values.
+ * If its -1, then there are no missing values. If its -2, all values are missing.
*
Checksum contains the CRC32 checksum of all bytes in the .dvm file up
* until the checksum. This is used to verify integrity of the file on opening the
* index.
@@ -164,26 +166,26 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
*
* @lucene.experimental
*/
-public final class Lucene410DocValuesFormat extends DocValuesFormat {
+public final class Lucene50DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
- public Lucene410DocValuesFormat() {
- super("Lucene410");
+ public Lucene50DocValuesFormat() {
+ super("Lucene50");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
- return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
- static final String DATA_CODEC = "Lucene410DocValuesData";
+ static final String DATA_CODEC = "Lucene50DocValuesData";
static final String DATA_EXTENSION = "dvd";
- static final String META_CODEC = "Lucene410ValuesMetadata";
+ static final String META_CODEC = "Lucene50ValuesMetadata";
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
new file mode 100644
index 00000000000..9e5d9b260dc
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
@@ -0,0 +1,1153 @@
+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 static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.CONST_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.MONOTONIC_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_SINGLE_VALUED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_WITH_ADDRESSES;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.ArrayList;
+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.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+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;
+
+/** reader for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
+ private final Map numerics = new HashMap<>();
+ private final Map binaries = new HashMap<>();
+ private final Map sortedSets = new HashMap<>();
+ private final Map sortedNumerics = new HashMap<>();
+ private final Map ords = new HashMap<>();
+ private final Map 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 addressInstances = new HashMap<>();
+ private final Map ordIndexInstances = new HashMap<>();
+ private final Map 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.getDocCount();
+ merging = false;
+
+ 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.checkSegmentHeader(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.checkSegmentHeader(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);
+ }
+ }
+
+ ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+ }
+
+ 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 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_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 (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 (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);
+ }
+ NumericEntry ordIndex = readNumericEntry(meta);
+ ordIndexes.put(info.name, ordIndex);
+ } else if (ss.format != SORTED_SINGLE_VALUED) {
+ 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();
+ }
+ 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_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 Iterable extends Accountable> getChildResources() {
+ List 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);
+ NumericEntry numericEntry = numerics.get(field.name);
+ final LongValues values = getNumeric(numericEntry);
+ if (ss.format == SORTED_SINGLE_VALUED) {
+ final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+ return DocValues.singleton(values, docsWithField);
+ } else if (ss.format == SORTED_WITH_ADDRESSES) {
+ 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 {
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedSets.get(field.name);
+ if (ss.format == SORTED_SINGLE_VALUED) {
+ final SortedDocValues values = getSorted(field);
+ return DocValues.singleton(values);
+ } else if (ss.format != SORTED_WITH_ADDRESSES) {
+ throw new AssertionError();
+ }
+
+ 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 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;
+ }
+
+ // 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 Iterable extends Accountable> getChildResources() {
+ List 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 DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
index 9ce0a294e2f..cd8ac80e94f 100755
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
@@ -185,7 +185,7 @@ term frequency. To add Term Vectors to your index see the
{@link org.apache.lucene.document.Field Field} constructors
-{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-document values}.
+{@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
@@ -294,7 +294,7 @@ systems that frequently run out of file handles.
Encodes length and boost factors for docs and fields |
-{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-Document Values} |
+{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-Document Values} |
.dvd, .dvm |
Encodes additional scoring factors or other per-document information. |
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 8cc6f70d0f4..c5d32072081 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat
+org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
new file mode 100644
index 00000000000..cbe32659b06
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
@@ -0,0 +1,285 @@
+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.codecs.blocktreeords.Ords41PostingsFormat;
+import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
+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.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+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.
+
+ public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16);
+ }
+ }
+
+ @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);
+ }
+ }
+
+ 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);
+ }
+ }
+
+ public void testTermsEnumFixedWidth() throws Exception {
+ int numIterations = atLeast(1);
+ for (int i = 0; i < numIterations; i++) {
+ doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+ }
+ }
+
+ 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;
+ switch (random().nextInt(2)) {
+ case 0: pf = new Lucene41WithOrds();
+ break;
+ case 1: pf = new Ords41PostingsFormat();
+ break;
+ // TODO: these don't actually support ords!
+ //case 2: pf = new FSTOrdPostingsFormat();
+ // break;
+ default: throw new AssertionError();
+ }
+ 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 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 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 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) {
+ assertEquals(terms.size(), r.getSortedSetDocValues("dv").getValueCount());
+ TermsEnum expected = terms.iterator(null);
+ TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+ assertEquals(terms.size(), expected, actual);
+ }
+ }
+ 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(null);
+ 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());
+ }
+ }
+ }
+}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
index faf057c5bae..5fa38b2f957 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
@@ -2883,4 +2883,6 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
protected boolean codecSupportsSortedNumeric() {
return true;
}
+
+ // nocommit: ensure every type has a *mergeAwayAllValues test, and add simple constant tests
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index ef75a4848cf..0db3910b55b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -46,8 +46,8 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
import org.apache.lucene.codecs.lucene50.Lucene50Codec;
+import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@@ -761,7 +761,7 @@ public final class TestUtil {
* Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
*/
public static DocValuesFormat getDefaultDocValuesFormat() {
- return new Lucene410DocValuesFormat();
+ return new Lucene50DocValuesFormat();
}
// TODO: generalize all 'test-checks-for-crazy-codecs' to
From 18565522986f11993c28a1871443726e5b4850ce Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Thu, 16 Oct 2014 11:03:09 +0000
Subject: [PATCH 02/19] LUCENE-5969: add tests
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632273 13f79535-47bb-0310-9956-ffa450edef68
---
.../index/BaseDocValuesFormatTestCase.java | 81 ++++++++++++++++++-
1 file changed, 79 insertions(+), 2 deletions(-)
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
index 5fa38b2f957..b3d9d54903e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
@@ -541,6 +541,34 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
ireader.close();
directory.close();
}
+
+ public void testBytesMergeAwayAllValues() throws IOException {
+ Directory directory = newDirectory();
+ Analyzer analyzer = new MockAnalyzer(random());
+ IndexWriterConfig iwconfig = newIndexWriterConfig(analyzer);
+ iwconfig.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
+
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.NO));
+ iwriter.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.NO));
+ doc.add(new BinaryDocValuesField("field", new BytesRef("hi")));
+ iwriter.addDocument(doc);
+ iwriter.commit();
+ iwriter.deleteDocuments(new Term("id", "1"));
+ iwriter.forceMerge(1);
+
+ DirectoryReader ireader = iwriter.getReader();
+ iwriter.close();
+
+ BinaryDocValues dv = getOnlySegmentReader(ireader).getBinaryDocValues("field");
+ assertEquals(new BytesRef(), dv.get(0));
+
+ ireader.close();
+ directory.close();
+ }
public void testSortedBytes() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
@@ -2749,6 +2777,34 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
directory.close();
}
+ public void testNumberMergeAwayAllValues() throws IOException {
+ Directory directory = newDirectory();
+ Analyzer analyzer = new MockAnalyzer(random());
+ IndexWriterConfig iwconfig = newIndexWriterConfig(analyzer);
+ iwconfig.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
+
+ Document doc = new Document();
+ doc.add(new StringField("id", "0", Field.Store.NO));
+ iwriter.addDocument(doc);
+ doc = new Document();
+ doc.add(new StringField("id", "1", Field.Store.NO));
+ doc.add(new NumericDocValuesField("field", 5));
+ iwriter.addDocument(doc);
+ iwriter.commit();
+ iwriter.deleteDocuments(new Term("id", "1"));
+ iwriter.forceMerge(1);
+
+ DirectoryReader ireader = iwriter.getReader();
+ iwriter.close();
+
+ NumericDocValues dv = getOnlySegmentReader(ireader).getNumericDocValues("field");
+ assertEquals(0, dv.get(0));
+
+ ireader.close();
+ directory.close();
+ }
+
public void testTwoSortedNumber() throws IOException {
assumeTrue("Codec does not support SORTED_NUMERIC", codecSupportsSortedNumeric());
Directory directory = newDirectory();
@@ -2772,6 +2828,29 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
directory.close();
}
+ public void testTwoSortedNumberSameValue() throws IOException {
+ assumeTrue("Codec does not support SORTED_NUMERIC", codecSupportsSortedNumeric());
+ Directory directory = newDirectory();
+ RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
+ Document doc = new Document();
+ doc.add(new SortedNumericDocValuesField("dv", 11));
+ doc.add(new SortedNumericDocValuesField("dv", 11));
+ writer.addDocument(doc);
+ writer.close();
+
+ // Now search the index:
+ IndexReader reader = DirectoryReader.open(directory);
+ assert reader.leaves().size() == 1;
+ SortedNumericDocValues dv = reader.leaves().get(0).reader().getSortedNumericDocValues("dv");
+ dv.setDocument(0);
+ assertEquals(2, dv.count());
+ assertEquals(11, dv.valueAt(0));
+ assertEquals(11, dv.valueAt(1));
+
+ reader.close();
+ directory.close();
+ }
+
public void testTwoSortedNumberOneMissing() throws IOException {
assumeTrue("Codec does not support SORTED_NUMERIC", codecSupportsSortedNumeric());
Directory directory = newDirectory();
@@ -2883,6 +2962,4 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
protected boolean codecSupportsSortedNumeric() {
return true;
}
-
- // nocommit: ensure every type has a *mergeAwayAllValues test, and add simple constant tests
}
From 922d1edbff054c7551a2019de834c5abb06a53b6 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Fri, 17 Oct 2014 16:49:25 +0000
Subject: [PATCH 03/19] LUCENE-5969: use sparsebitset to expand sparse encoding
to cover more absurd cases
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632631 13f79535-47bb-0310-9956-ffa450edef68
---
.../lucene50/Lucene50NormsConsumer.java | 137 ++++++++++++++----
.../codecs/lucene50/Lucene50NormsFormat.java | 3 +
.../lucene50/Lucene50NormsProducer.java | 36 ++++-
.../lucene50/TestLucene50NormsFormat.java | 95 +++++++++++-
.../lucene/index/BaseNormsFormatTestCase.java | 29 ++++
5 files changed, 273 insertions(+), 27 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
index cc170e45e11..d1a8efa6faf 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
@@ -47,6 +47,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
static final byte CONST_COMPRESSED = 2;
static final byte UNCOMPRESSED = 3;
static final byte INDIRECT = 4;
+ static final byte PATCHED = 5;
static final int BLOCK_SIZE = 1 << 14;
// threshold for indirect encoding, computed as 1 - 1/log2(maxint)
@@ -82,6 +83,11 @@ class Lucene50NormsConsumer extends NormsConsumer {
@Override
public void addNormsField(FieldInfo field, Iterable values) throws IOException {
+ writeNormsField(field, values, 0);
+ }
+
+ private void writeNormsField(FieldInfo field, Iterable values, int level) throws IOException {
+ assert level <= 1; // we only "recurse" once in the indirect case
meta.writeVInt(field.number);
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
@@ -89,16 +95,12 @@ class Lucene50NormsConsumer extends NormsConsumer {
NormMap uniqueValues = new NormMap();
int count = 0;
- int missingCount = 0;
for (Number nv : values) {
if (nv == null) {
throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
}
final long v = nv.longValue();
- if (v == 0) {
- missingCount++;
- }
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
@@ -115,9 +117,15 @@ class Lucene50NormsConsumer extends NormsConsumer {
if (uniqueValues != null && uniqueValues.size == 1) {
// 0 bpv
addConstant(minValue);
- } else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
- // sparse encoding
- addIndirect(field, values, count, missingCount);
+ } else if (level == 0 && count > 256 && uniqueValues != null && uniqueValues.maxFreq() > count * INDIRECT_THRESHOLD) {
+ long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
+ if (commonValue == 0) {
+ // if the common value is missing, don't waste RAM on a bitset, since we won't be searching those docs
+ addIndirect(field, values, count, uniqueValues);
+ } else {
+ // otherwise, write a sparse bitset, where 1 indicates 'uncommon value'.
+ addPatched(field, values, count, uniqueValues);
+ }
} else if (uniqueValues != null) {
// small number of unique values: this is the typical case:
FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
@@ -200,10 +208,65 @@ class Lucene50NormsConsumer extends NormsConsumer {
writer.finish();
}
- private void addIndirect(FieldInfo field, final Iterable values, int count, int missingCount) throws IOException {
- meta.writeVInt(count - missingCount);
+ // encodes only uncommon values in a sparse bitset
+ // access is constant time, and the common case is predictable
+ // exceptions nest either to CONST (if there are only 2 values), or INDIRECT (if there are > 2 values)
+ private void addPatched(FieldInfo field, final Iterable values, int count, NormMap uniqueValues) throws IOException {
+ final long decodeTable[] = uniqueValues.getDecodeTable();
+ int commonCount = uniqueValues.maxFreq();
+ final long commonValue = decodeTable[uniqueValues.maxOrd()];
+
+ meta.writeVInt(count - commonCount);
+ meta.writeByte(PATCHED);
+ meta.writeLong(data.getFilePointer());
+
+ // write docs with value
+ writeDocsWithValue(values, commonValue);
+
+ // write exceptions: only two cases make sense
+ // bpv = 1 (folded into sparse bitset already)
+ // bpv > 1 (add indirect exception table)
+ meta.writeVInt(field.number);
+ if (uniqueValues.size == 2) {
+ // special case: implicit in bitset
+ int otherOrd = uniqueValues.maxOrd() == 0 ? 1 : 0;
+ addConstant(decodeTable[otherOrd]);
+ } else {
+ // exception table
+ addIndirect(field, values, count, uniqueValues);
+ }
+ }
+
+ // encodes values as sparse array: keys[] and values[]
+ // access is log(N) where N = keys.length (slow!)
+ // so this is only appropriate as an exception table for patched, or when common value is 0 (wont be accessed by searching)
+ private void addIndirect(FieldInfo field, final Iterable values, int count, NormMap uniqueValues) throws IOException {
+ int commonCount = uniqueValues.maxFreq();
+ final long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
+
+ meta.writeVInt(count - commonCount);
meta.writeByte(INDIRECT);
meta.writeLong(data.getFilePointer());
+
+ // write docs with value
+ writeDocsWithValue(values, commonValue);
+
+ // write actual values
+ writeNormsField(field, new Iterable() {
+ @Override
+ public Iterator iterator() {
+ return new FilterIterator(values.iterator()) {
+ @Override
+ protected boolean predicateFunction(Number value) {
+ return value.longValue() != commonValue;
+ }
+ };
+ }
+ }, 1);
+ }
+
+ private void writeDocsWithValue(final Iterable values, long commonValue) throws IOException {
+ data.writeLong(commonValue);
data.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE);
@@ -212,25 +275,12 @@ class Lucene50NormsConsumer extends NormsConsumer {
int doc = 0;
for (Number n : values) {
long v = n.longValue();
- if (v != 0) {
+ if (v != commonValue) {
writer.add(doc);
}
doc++;
}
writer.finish();
-
- // write actual values
- addNormsField(field, new Iterable() {
- @Override
- public Iterator iterator() {
- return new FilterIterator(values.iterator()) {
- @Override
- protected boolean predicateFunction(Number value) {
- return value.longValue() != 0;
- }
- };
- }
- });
}
@Override
@@ -259,6 +309,7 @@ class Lucene50NormsConsumer extends NormsConsumer {
static class NormMap {
// we use short: at most we will add 257 values to this map before its rejected as too big above.
final short[] singleByteRange = new short[256];
+ final int[] freqs = new int[257];
final Map other = new HashMap();
int size;
@@ -273,18 +324,24 @@ class Lucene50NormsConsumer extends NormsConsumer {
int index = (int) (l + 128);
short previous = singleByteRange[index];
if (previous < 0) {
- singleByteRange[index] = (short) size;
+ short slot = (short) size;
+ singleByteRange[index] = slot;
+ freqs[slot]++;
size++;
return true;
} else {
+ freqs[previous]++;
return false;
}
} else {
- if (!other.containsKey(l)) {
+ Short previous = other.get(l);
+ if (previous == null) {
+ freqs[size]++;
other.put(l, (short)size);
size++;
return true;
} else {
+ freqs[previous]++;
return false;
}
}
@@ -315,5 +372,35 @@ class Lucene50NormsConsumer extends NormsConsumer {
}
return decode;
}
+
+ // TODO: if we need more complicated frequency-driven optos, maybe add 'finish' to this api
+ // and sort all ords by frequency. we could then lower BPV and waste a value to represent 'patched',
+
+ /** retrieves frequency table for items (indexed by ordinal) */
+ public int[] getFreqs() {
+ return freqs;
+ }
+
+ /** sugar: returns max value over getFreqs() */
+ public int maxFreq() {
+ int max = 0;
+ for (int i = 0; i < size; i++) {
+ max = Math.max(max, freqs[i]);
+ }
+ return max;
+ }
+
+ /** sugar: returns ordinal with maxFreq() */
+ public int maxOrd() {
+ long max = 0;
+ int maxOrd = 0;
+ for (int i = 0; i < size; i++) {
+ if (freqs[i] > max) {
+ max = freqs[i];
+ maxOrd = i;
+ }
+ }
+ return maxOrd;
+ }
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
index 2542ab349f9..bd46680dc4b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
@@ -50,6 +50,9 @@ import org.apache.lucene.util.packed.PackedInts;
* Indirect: when norms are extremely sparse, missing values are omitted.
* Access to an individual value is slower, but missing norm values are never accessed
* by search code.
+ * Patched: when a single norm value dominates, a sparse bitset encodes docs with exceptions,
+ * so that access to the common value is still very fast. outliers fall thru to an exception
+ * handling mechanism (Indirect or Constant).
*
*
* Files:
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
index 8c54ffdb06d..16566efb9a7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.SparseFixedBitSet;
import org.apache.lucene.util.packed.BlockPackedReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
import org.apache.lucene.util.packed.PackedInts;
@@ -48,6 +49,7 @@ import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMP
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED;
/**
* Reader for {@link Lucene50NormsFormat}
@@ -63,6 +65,7 @@ class Lucene50NormsProducer extends NormsProducer {
private final AtomicLong ramBytesUsed;
private final AtomicInteger activeCount = new AtomicInteger();
+ private final int maxDoc;
private final boolean merging;
@@ -75,11 +78,13 @@ class Lucene50NormsProducer extends NormsProducer {
instancesInfo.putAll(original.instancesInfo);
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
activeCount.set(original.activeCount.get());
+ maxDoc = original.maxDoc;
merging = true;
}
Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
merging = false;
+ maxDoc = state.segmentInfo.getDocCount();
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
int version = -1;
@@ -146,6 +151,7 @@ class Lucene50NormsProducer extends NormsProducer {
case TABLE_COMPRESSED:
case DELTA_COMPRESSED:
break;
+ case PATCHED:
case INDIRECT:
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
@@ -254,6 +260,7 @@ class Lucene50NormsProducer extends NormsProducer {
}
case INDIRECT: {
data.seek(entry.offset);
+ final long common = data.readLong();
int packedIntsVersion = data.readVInt();
int blockSize = data.readVInt();
final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
@@ -279,7 +286,34 @@ class Lucene50NormsProducer extends NormsProducer {
return values.get(mid);
}
}
- return 0;
+ return common;
+ }
+ };
+ break;
+ }
+ case PATCHED: {
+ data.seek(entry.offset);
+ final long common = data.readLong();
+ int packedIntsVersion = data.readVInt();
+ int blockSize = data.readVInt();
+ MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, true);
+ final SparseFixedBitSet set = new SparseFixedBitSet(maxDoc);
+ for (int i = 0; i < live.size(); i++) {
+ int doc = (int) live.get(i);
+ set.set(doc);
+ }
+ LoadedNorms nestedInstance = loadNorms(entry.nested);
+ instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed;
+ instance.info = Accountables.namedAccountable("patched -> " + nestedInstance.info, instance.ramBytesUsed);
+ final NumericDocValues values = nestedInstance.norms;
+ instance.norms = new NumericDocValues() {
+ @Override
+ public long get(int docID) {
+ if (set.get(docID)) {
+ return values.get(docID);
+ } else {
+ return common;
+ }
}
};
break;
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java
index 0dfc22fa496..19939d9e22e 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50NormsFormat.java
@@ -17,7 +17,13 @@ package org.apache.lucene.codecs.lucene50;
* limitations under the License.
*/
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.NormMap;
import org.apache.lucene.index.BaseNormsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@@ -30,5 +36,92 @@ public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
@Override
protected Codec getCodec() {
return codec;
- }
+ }
+
+ // NormMap is rather complicated, doing domain encoding / tracking frequencies etc.
+ // test it directly some here...
+
+ public void testNormMapSimple() {
+ NormMap map = new NormMap();
+ map.add(10);
+ map.add(5);
+ map.add(4);
+ map.add(10);
+ assertEquals(3, map.size);
+
+ // first come, first serve ord assignment
+
+ // encode
+ assertEquals(0, map.getOrd(10));
+ assertEquals(1, map.getOrd(5));
+ assertEquals(2, map.getOrd(4));
+
+ // decode
+ long decode[] = map.getDecodeTable();
+ assertEquals(10, decode[0]);
+ assertEquals(5, decode[1]);
+ assertEquals(4, decode[2]);
+
+ // freqs
+ int freqs[] = map.getFreqs();
+ assertEquals(2, freqs[0]);
+ assertEquals(1, freqs[1]);
+ assertEquals(1, freqs[2]);
+
+ assertEquals(2, map.maxFreq());
+ }
+
+ public void testNormMapRandom() {
+ Map freqs = new HashMap<>();
+ Map ords = new HashMap<>();
+
+ Set uniqueValuesSet = new HashSet<>();
+ int numUniqValues = TestUtil.nextInt(random(), 1, 256);
+ for (int i = 0; i < numUniqValues; i++) {
+ if (random().nextBoolean()) {
+ uniqueValuesSet.add(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
+ } else {
+ uniqueValuesSet.add(TestUtil.nextLong(random(), Byte.MIN_VALUE, Byte.MAX_VALUE));
+ }
+ }
+
+ Long uniqueValues[] = uniqueValuesSet.toArray(new Long[uniqueValuesSet.size()]);
+
+ NormMap map = new NormMap();
+ int numdocs = TestUtil.nextInt(random(), 1, 100000);
+ for (int i = 0; i < numdocs; i++) {
+ long value = uniqueValues[random().nextInt(uniqueValues.length)];
+ // now add to both expected and actual
+ map.add(value);
+
+ Integer ord = ords.get(value);
+ if (ord == null) {
+ ord = ords.size();
+ ords.put(value, ord);
+ freqs.put(value, 1);
+ } else {
+ freqs.put(value, freqs.get(value)+1);
+ }
+ }
+
+ // value -> ord
+ assertEquals(ords.size(), map.size);
+ for (Map.Entry kv : ords.entrySet()) {
+ assertEquals(kv.getValue().intValue(), map.getOrd(kv.getKey()));
+ }
+
+ // ord -> value
+ Map reversed = new HashMap<>();
+ long table[] = map.getDecodeTable();
+ for (int i = 0; i < map.size; i++) {
+ reversed.put(table[i], i);
+ }
+ assertEquals(ords, reversed);
+
+ // freqs
+ int freqTable[] = map.getFreqs();
+ for (int i = 0; i < map.size; i++) {
+ assertEquals(freqs.get(table[i]).longValue(), freqTable[i]);
+ }
+ }
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
index 9eaeeef7f67..d3ff8895700 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
@@ -153,6 +153,35 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
}
}
+ public void testOutliers() throws Exception {
+ int iterations = atLeast(1);
+ final Random r = random();
+ for (int i = 0; i < iterations; i++) {
+ final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+ doTestNormsVersusStoredFields(new LongProducer() {
+ @Override
+ long next() {
+ return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
+ }
+ });
+ }
+ }
+
+ public void testOutliers2() throws Exception {
+ int iterations = atLeast(1);
+ final Random r = random();
+ for (int i = 0; i < iterations; i++) {
+ final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+ final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+ doTestNormsVersusStoredFields(new LongProducer() {
+ @Override
+ long next() {
+ return r.nextInt(100) == 0 ? uncommonValue : commonValue;
+ }
+ });
+ }
+ }
+
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
int numDocs = atLeast(500);
long norms[] = new long[numDocs];
From c9e2d7bead4b364e017c3730f03ec40444c05ee9 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Sat, 18 Oct 2014 00:19:30 +0000
Subject: [PATCH 04/19] LUCENE-5969: fix tests
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1632706 13f79535-47bb-0310-9956-ffa450edef68
---
.../solr/collection1/conf/schema_codec.xml | 2 +-
.../org/apache/solr/core/TestCodecSupport.java | 17 +++++++++--------
2 files changed, 10 insertions(+), 9 deletions(-)
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
index 439472e6fe9..3b4cfe3e561 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
@@ -21,7 +21,7 @@
-
+
diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
index 15254517527..e04a17f041c 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
@@ -22,6 +22,7 @@ import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.schema.SchemaField;
import org.junit.BeforeClass;
@@ -43,9 +44,9 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
assertEquals("SimpleText",
format.getPostingsFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_standard_f");
- assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
+ assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_f");
- assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
+ assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField(schemaField.getName()).getName());
}
public void testDocValuesFormats() {
@@ -53,12 +54,12 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
Map fields = h.getCore().getLatestSchema().getFields();
SchemaField schemaField = fields.get("string_disk_f");
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
- assertEquals("Lucene410", format.getDocValuesFormatForField(schemaField.getName()).getName());
+ assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_memory_f");
assertEquals("Memory",
format.getDocValuesFormatForField(schemaField.getName()).getName());
schemaField = fields.get("string_f");
- assertEquals("Lucene410",
+ assertEquals(TestUtil.getDefaultDocValuesFormat().getName(),
format.getDocValuesFormatForField(schemaField.getName()).getName());
}
@@ -70,16 +71,16 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
assertEquals("SimpleText", format.getPostingsFormatForField("bar_simple").getName());
assertEquals("Direct", format.getPostingsFormatForField("foo_direct").getName());
assertEquals("Direct", format.getPostingsFormatForField("bar_direct").getName());
- assertEquals("Lucene41", format.getPostingsFormatForField("foo_standard").getName());
- assertEquals("Lucene41", format.getPostingsFormatForField("bar_standard").getName());
+ assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("foo_standard").getName());
+ assertEquals(TestUtil.getDefaultPostingsFormat().getName(), format.getPostingsFormatForField("bar_standard").getName());
}
public void testDynamicFieldsDocValuesFormats() {
Codec codec = h.getCore().getCodec();
PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
- assertEquals("Lucene410", format.getDocValuesFormatForField("foo_disk").getName());
- assertEquals("Lucene410", format.getDocValuesFormatForField("bar_disk").getName());
+ assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("foo_disk").getName());
+ assertEquals(TestUtil.getDefaultDocValuesFormat().getName(), format.getDocValuesFormatForField("bar_disk").getName());
assertEquals("Memory", format.getDocValuesFormatForField("foo_memory").getName());
assertEquals("Memory", format.getDocValuesFormatForField("bar_memory").getName());
}
From eee7825cef1e500c2953d4b4d952b8cdcc5dc6da Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Mon, 20 Oct 2014 18:25:52 +0000
Subject: [PATCH 05/19] LUCENE-5969: move old postings back compat to
backward-codecs, cleanup PBF related stuff, add segment headers, etc
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1633196 13f79535-47bb-0310-9956-ffa450edef68
---
.../lucene40/Lucene40PostingsBaseFormat.java | 48 -
.../lucene40/Lucene40PostingsFormat.java | 8 +-
.../lucene40/Lucene40PostingsReader.java | 3 +-
.../lucene/codecs/lucene41/ForUtil.java | 247 +++
.../codecs/lucene41/IntBlockTermState.java | 45 +
.../lucene41/Lucene41PostingsFormat.java | 113 ++
.../lucene41/Lucene41PostingsReader.java | 44 +-
.../codecs/lucene41/Lucene41SkipReader.java | 28 +-
.../org.apache.lucene.codecs.PostingsFormat | 1 +
.../lucene40/Lucene40PostingsWriter.java | 2 +-
.../lucene41/Lucene41PostingsWriter.java | 77 +-
.../codecs/lucene41/Lucene41RWCodec.java | 7 +
.../lucene41/Lucene41RWPostingsFormat.java | 56 +
.../codecs/lucene41/Lucene41SkipWriter.java | 21 +-
.../codecs/lucene41/TestLucene41ForUtil.java | 94 ++
.../lucene41/TestLucene41PostingsFormat.java | 66 +
.../lucene41/TestLucene41PostingsFormat2.java | 132 ++
.../lucene41/TestLucene41PostingsFormat3.java | 521 +++++++
.../codecs/lucene410/Lucene410RWCodec.java | 9 +
.../TestLucene410DocValuesFormat.java | 14 +-
.../codecs/lucene42/Lucene42RWCodec.java | 9 +
.../codecs/lucene45/Lucene45RWCodec.java | 9 +
.../codecs/lucene46/Lucene46RWCodec.java | 9 +
.../codecs/lucene49/Lucene49RWCodec.java | 9 +
.../codecs/blockterms/BlockTermsReader.java | 2 +-
.../codecs/blockterms/BlockTermsWriter.java | 2 +-
....java => BlockTreeOrdsPostingsFormat.java} | 22 +-
.../OrdsBlockTreeTermsReader.java | 2 +-
.../OrdsBlockTreeTermsWriter.java | 2 +-
.../codecs/memory/DirectPostingsFormat.java | 8 +-
.../codecs/memory/FSTOrdPostingsFormat.java | 16 +-
.../codecs/memory/FSTOrdTermsReader.java | 2 +-
.../codecs/memory/FSTOrdTermsWriter.java | 2 +-
.../codecs/memory/FSTPostingsFormat.java | 16 +-
.../lucene/codecs/memory/FSTTermsReader.java | 2 +-
.../lucene/codecs/memory/FSTTermsWriter.java | 2 +-
.../org.apache.lucene.codecs.PostingsFormat | 2 +-
.../TestFixedGapPostingsFormat.java | 3 +-
...stVarGapDocFreqIntervalPostingsFormat.java | 4 +-
...TestVarGapFixedIntervalPostingsFormat.java | 4 +-
.../blocktreeords/TestOrdsBlockTree.java | 2 +-
.../lucene/codecs/PostingsBaseFormat.java | 55 -
.../lucene/codecs/PostingsReaderBase.java | 3 +-
.../lucene/codecs/PostingsWriterBase.java | 3 +-
.../lucene/codecs/PushPostingsWriterBase.java | 24 +-
.../blocktree/BlockTreeTermsReader.java | 27 +-
.../blocktree/BlockTreeTermsWriter.java | 2 +-
.../apache/lucene/codecs/blocktree/Stats.java | 6 +-
.../lucene41/Lucene41PostingsBaseFormat.java | 51 -
.../lucene/codecs/lucene41/package.html | 25 -
.../{lucene41 => lucene50}/ForUtil.java | 4 +-
.../lucene/codecs/lucene50/Lucene50Codec.java | 4 +-
.../Lucene50PostingsFormat.java} | 52 +-
.../lucene50/Lucene50PostingsReader.java | 1340 +++++++++++++++++
.../lucene50/Lucene50PostingsWriter.java | 525 +++++++
.../codecs/lucene50/Lucene50SkipReader.java | 198 +++
.../codecs/lucene50/Lucene50SkipWriter.java | 169 +++
.../lucene/codecs/lucene50/package.html | 16 +-
.../org.apache.lucene.codecs.PostingsFormat | 2 +-
.../TestBlockPostingsFormat.java | 4 +-
.../TestBlockPostingsFormat2.java | 14 +-
.../TestBlockPostingsFormat3.java | 8 +-
.../{lucene41 => lucene50}/TestForUtil.java | 10 +-
.../lucene50/TestLucene50DocValuesFormat.java | 14 +-
.../perfield/TestPerFieldPostingsFormat2.java | 6 +-
.../lucene/uninverting/TestDocTermOrds.java | 5 +-
.../idversion/IDVersionPostingsFormat.java | 4 +-
.../idversion/IDVersionPostingsReader.java | 7 +-
.../idversion/IDVersionPostingsWriter.java | 11 +-
.../VersionBlockTreeTermsReader.java | 2 +-
.../VersionBlockTreeTermsWriter.java | 2 +-
.../LuceneFixedGap.java} | 22 +-
.../LuceneVarGapDocFreqInterval.java} | 22 +-
.../LuceneVarGapFixedInterval.java} | 22 +-
.../package.html | 2 +-
.../lucene/codecs/lucene41ords/package.html | 25 -
.../mockrandom/MockRandomPostingsFormat.java | 19 +-
.../org/apache/lucene/index/RandomCodec.java | 18 +-
.../java/org/apache/lucene/util/TestUtil.java | 19 +-
.../org.apache.lucene.codecs.PostingsFormat | 6 +-
.../solr/collection1/conf/schema_codec.xml | 2 +-
81 files changed, 3800 insertions(+), 615 deletions(-)
delete mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/IntBlockTermState.java
create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
rename lucene/{core => backward-codecs}/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (97%)
rename lucene/{core => backward-codecs}/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java (84%)
rename lucene/{core/src/java => backward-codecs/src/test}/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java (87%)
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWPostingsFormat.java
rename lucene/{core/src/java => backward-codecs/src/test}/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java (86%)
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41ForUtil.java
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat.java
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat2.java
create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat3.java
rename lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/{Ords41PostingsFormat.java => BlockTreeOrdsPostingsFormat.java} (79%)
delete mode 100644 lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java
delete mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
delete mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
rename lucene/core/src/java/org/apache/lucene/codecs/{lucene41 => lucene50}/ForUtil.java (98%)
rename lucene/core/src/java/org/apache/lucene/codecs/{lucene41/Lucene41PostingsFormat.java => lucene50/Lucene50PostingsFormat.java} (91%)
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipReader.java
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
rename lucene/core/src/test/org/apache/lucene/codecs/{lucene41 => lucene50}/TestBlockPostingsFormat.java (97%)
rename lucene/core/src/test/org/apache/lucene/codecs/{lucene41 => lucene50}/TestBlockPostingsFormat2.java (91%)
rename lucene/core/src/test/org/apache/lucene/codecs/{lucene41 => lucene50}/TestBlockPostingsFormat3.java (98%)
rename lucene/core/src/test/org/apache/lucene/codecs/{lucene41 => lucene50}/TestForUtil.java (92%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene41ords/Lucene41WithOrds.java => blockterms/LuceneFixedGap.java} (84%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene41vargap/Lucene41VarGapDocFreqInterval.java => blockterms/LuceneVarGapDocFreqInterval.java} (83%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene41vargap/Lucene41VarGapFixedInterval.java => blockterms/LuceneVarGapFixedInterval.java} (83%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene41vargap => blockterms}/package.html (96%)
delete mode 100644 lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/package.html
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
deleted file mode 100644
index 5d278db4209..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
+++ /dev/null
@@ -1,48 +0,0 @@
-package org.apache.lucene.codecs.lucene40;
-
-/*
- * 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.PostingsBaseFormat;
-import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-
-/**
- * PostingsReaderBase for 4.0 segments
- * @deprecated Only for reading old 4.0 segments */
-@Deprecated
-final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
-
- /** Sole constructor. */
- Lucene40PostingsBaseFormat() {
- super("Lucene40");
- }
-
- @Override
- public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
- return new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
- }
-
- @Override
- public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
- throw new UnsupportedOperationException("this codec can only be used for reading");
- }
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
index e6e3eeaf53b..3256947cc4d 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
@@ -51,13 +51,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
boolean success = false;
try {
- FieldsProducer ret = new BlockTreeTermsReader(
- state.directory,
- state.fieldInfos,
- state.segmentInfo,
- postings,
- state.context,
- state.segmentSuffix);
+ FieldsProducer ret = new BlockTreeTermsReader(postings, state);
success = true;
return ret;
} finally {
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
index 05ffeda2591..d60b15b15f1 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
@@ -31,6 +31,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
@@ -104,7 +105,7 @@ final class Lucene40PostingsReader extends PostingsReaderBase {
}
@Override
- public void init(IndexInput termsIn) throws IOException {
+ public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
// Make sure we are talking to the matching past writer
CodecUtil.checkHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT);
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
new file mode 100644
index 00000000000..cf76197f9ee
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
@@ -0,0 +1,247 @@
+package org.apache.lucene.codecs.lucene41;
+/*
+ * 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.Arrays;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedInts.Decoder;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+
+/**
+ * Lucene 4.1 postings format.
+ * @deprecated only for reading old 4.x segments
+ */
+@Deprecated
+final class ForUtil {
+
+ /**
+ * Special number of bits per value used whenever all values to encode are equal.
+ */
+ private static final int ALL_VALUES_EQUAL = 0;
+
+ /**
+ * Upper limit of the number of bytes that might be required to stored
+ * BLOCK_SIZE
encoded values.
+ */
+ static final int MAX_ENCODED_SIZE = BLOCK_SIZE * 4;
+
+ /**
+ * Upper limit of the number of values that might be decoded in a single call to
+ * {@link #readBlock(IndexInput, byte[], int[])}. Although values after
+ * BLOCK_SIZE
are garbage, it is necessary to allocate value buffers
+ * whose size is >= MAX_DATA_SIZE to avoid {@link ArrayIndexOutOfBoundsException}s.
+ */
+ static final int MAX_DATA_SIZE;
+ static {
+ int maxDataSize = 0;
+ for(int version=PackedInts.VERSION_START;version<=PackedInts.VERSION_CURRENT;version++) {
+ for (PackedInts.Format format : PackedInts.Format.values()) {
+ for (int bpv = 1; bpv <= 32; ++bpv) {
+ if (!format.isSupported(bpv)) {
+ continue;
+ }
+ final PackedInts.Decoder decoder = PackedInts.getDecoder(format, version, bpv);
+ final int iterations = computeIterations(decoder);
+ maxDataSize = Math.max(maxDataSize, iterations * decoder.byteValueCount());
+ }
+ }
+ }
+ MAX_DATA_SIZE = maxDataSize;
+ }
+
+ /**
+ * Compute the number of iterations required to decode BLOCK_SIZE
+ * values with the provided {@link Decoder}.
+ */
+ private static int computeIterations(PackedInts.Decoder decoder) {
+ return (int) Math.ceil((float) BLOCK_SIZE / decoder.byteValueCount());
+ }
+
+ /**
+ * Compute the number of bytes required to encode a block of values that require
+ * bitsPerValue
bits per value with format format
.
+ */
+ private static int encodedSize(PackedInts.Format format, int packedIntsVersion, int bitsPerValue) {
+ final long byteCount = format.byteCount(packedIntsVersion, BLOCK_SIZE, bitsPerValue);
+ assert byteCount >= 0 && byteCount <= Integer.MAX_VALUE : byteCount;
+ return (int) byteCount;
+ }
+
+ private final int[] encodedSizes;
+ private final PackedInts.Encoder[] encoders;
+ private final PackedInts.Decoder[] decoders;
+ private final int[] iterations;
+
+ /**
+ * Create a new {@link ForUtil} instance and save state into out
.
+ */
+ ForUtil(float acceptableOverheadRatio, DataOutput out) throws IOException {
+ out.writeVInt(PackedInts.VERSION_CURRENT);
+ encodedSizes = new int[33];
+ encoders = new PackedInts.Encoder[33];
+ decoders = new PackedInts.Decoder[33];
+ iterations = new int[33];
+
+ for (int bpv = 1; bpv <= 32; ++bpv) {
+ final FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(
+ BLOCK_SIZE, bpv, acceptableOverheadRatio);
+ assert formatAndBits.format.isSupported(formatAndBits.bitsPerValue);
+ assert formatAndBits.bitsPerValue <= 32;
+ encodedSizes[bpv] = encodedSize(formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
+ encoders[bpv] = PackedInts.getEncoder(
+ formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
+ decoders[bpv] = PackedInts.getDecoder(
+ formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
+ iterations[bpv] = computeIterations(decoders[bpv]);
+
+ out.writeVInt(formatAndBits.format.getId() << 5 | (formatAndBits.bitsPerValue - 1));
+ }
+ }
+
+ /**
+ * Restore a {@link ForUtil} from a {@link DataInput}.
+ */
+ ForUtil(DataInput in) throws IOException {
+ int packedIntsVersion = in.readVInt();
+ PackedInts.checkVersion(packedIntsVersion);
+ encodedSizes = new int[33];
+ encoders = new PackedInts.Encoder[33];
+ decoders = new PackedInts.Decoder[33];
+ iterations = new int[33];
+
+ for (int bpv = 1; bpv <= 32; ++bpv) {
+ final int code = in.readVInt();
+ final int formatId = code >>> 5;
+ final int bitsPerValue = (code & 31) + 1;
+
+ final PackedInts.Format format = PackedInts.Format.byId(formatId);
+ assert format.isSupported(bitsPerValue);
+ encodedSizes[bpv] = encodedSize(format, packedIntsVersion, bitsPerValue);
+ encoders[bpv] = PackedInts.getEncoder(
+ format, packedIntsVersion, bitsPerValue);
+ decoders[bpv] = PackedInts.getDecoder(
+ format, packedIntsVersion, bitsPerValue);
+ iterations[bpv] = computeIterations(decoders[bpv]);
+ }
+ }
+
+ /**
+ * Write a block of data (For
format).
+ *
+ * @param data the data to write
+ * @param encoded a buffer to use to encode data
+ * @param out the destination output
+ * @throws IOException If there is a low-level I/O error
+ */
+ void writeBlock(int[] data, byte[] encoded, IndexOutput out) throws IOException {
+ if (isAllEqual(data)) {
+ out.writeByte((byte) ALL_VALUES_EQUAL);
+ out.writeVInt(data[0]);
+ return;
+ }
+
+ final int numBits = bitsRequired(data);
+ assert numBits > 0 && numBits <= 32 : numBits;
+ final PackedInts.Encoder encoder = encoders[numBits];
+ final int iters = iterations[numBits];
+ assert iters * encoder.byteValueCount() >= BLOCK_SIZE;
+ final int encodedSize = encodedSizes[numBits];
+ assert iters * encoder.byteBlockCount() >= encodedSize;
+
+ out.writeByte((byte) numBits);
+
+ encoder.encode(data, 0, encoded, 0, iters);
+ out.writeBytes(encoded, encodedSize);
+ }
+
+ /**
+ * Read the next block of data (For
format).
+ *
+ * @param in the input to use to read data
+ * @param encoded a buffer that can be used to store encoded data
+ * @param decoded where to write decoded data
+ * @throws IOException If there is a low-level I/O error
+ */
+ void readBlock(IndexInput in, byte[] encoded, int[] decoded) throws IOException {
+ final int numBits = in.readByte();
+ assert numBits <= 32 : numBits;
+
+ if (numBits == ALL_VALUES_EQUAL) {
+ final int value = in.readVInt();
+ Arrays.fill(decoded, 0, BLOCK_SIZE, value);
+ return;
+ }
+
+ final int encodedSize = encodedSizes[numBits];
+ in.readBytes(encoded, 0, encodedSize);
+
+ final PackedInts.Decoder decoder = decoders[numBits];
+ final int iters = iterations[numBits];
+ assert iters * decoder.byteValueCount() >= BLOCK_SIZE;
+
+ decoder.decode(encoded, 0, decoded, 0, iters);
+ }
+
+ /**
+ * Skip the next block of data.
+ *
+ * @param in the input where to read data
+ * @throws IOException If there is a low-level I/O error
+ */
+ void skipBlock(IndexInput in) throws IOException {
+ final int numBits = in.readByte();
+ if (numBits == ALL_VALUES_EQUAL) {
+ in.readVInt();
+ return;
+ }
+ assert numBits > 0 && numBits <= 32 : numBits;
+ final int encodedSize = encodedSizes[numBits];
+ in.seek(in.getFilePointer() + encodedSize);
+ }
+
+ private static boolean isAllEqual(final int[] data) {
+ final int v = data[0];
+ for (int i = 1; i < BLOCK_SIZE; ++i) {
+ if (data[i] != v) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Compute the number of bits required to serialize any of the longs in
+ * data
.
+ */
+ private static int bitsRequired(final int[] data) {
+ long or = 0;
+ for (int i = 0; i < BLOCK_SIZE; ++i) {
+ assert data[i] >= 0;
+ or |= data[i];
+ }
+ return PackedInts.bitsRequired(or);
+ }
+
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/IntBlockTermState.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/IntBlockTermState.java
new file mode 100644
index 00000000000..dc85d8fb176
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/IntBlockTermState.java
@@ -0,0 +1,45 @@
+package org.apache.lucene.codecs.lucene41;
+
+import org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.index.TermState;
+
+/**
+ * term state for Lucene 4.1 postings format
+ * @deprecated only for reading old 4.x segments
+ */
+@Deprecated
+final class IntBlockTermState extends BlockTermState {
+ long docStartFP = 0;
+ long posStartFP = 0;
+ long payStartFP = 0;
+ long skipOffset = -1;
+ long lastPosBlockOffset = -1;
+ // docid when there is a single pulsed posting, otherwise -1
+ // freq is always implicitly totalTermFreq in this case.
+ int singletonDocID = -1;
+
+ @Override
+ public IntBlockTermState clone() {
+ IntBlockTermState other = new IntBlockTermState();
+ other.copyFrom(this);
+ return other;
+ }
+
+ @Override
+ public void copyFrom(TermState _other) {
+ super.copyFrom(_other);
+ IntBlockTermState other = (IntBlockTermState) _other;
+ docStartFP = other.docStartFP;
+ posStartFP = other.posStartFP;
+ payStartFP = other.payStartFP;
+ lastPosBlockOffset = other.lastPosBlockOffset;
+ skipOffset = other.skipOffset;
+ singletonDocID = other.singletonDocID;
+ }
+
+
+ @Override
+ public String toString() {
+ return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
+ }
+}
\ No newline at end of file
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
new file mode 100644
index 00000000000..c18ae8d94cc
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
@@ -0,0 +1,113 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Lucene 4.1 postings format.
+ * @deprecated only for reading old 4.x segments
+ */
+@Deprecated
+public class Lucene41PostingsFormat extends PostingsFormat {
+ /**
+ * Filename extension for document number, frequencies, and skip data.
+ * See chapter: Frequencies and Skip Data
+ */
+ public static final String DOC_EXTENSION = "doc";
+
+ /**
+ * Filename extension for positions.
+ * See chapter: Positions
+ */
+ public static final String POS_EXTENSION = "pos";
+
+ /**
+ * Filename extension for payloads and offsets.
+ * See chapter: Payloads and Offsets
+ */
+ public static final String PAY_EXTENSION = "pay";
+
+ /**
+ * Expert: The maximum number of skip levels. Smaller values result in
+ * slightly smaller indexes, but slower skipping in big posting lists.
+ */
+ static final int maxSkipLevels = 10;
+
+ final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
+ final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
+ final static String POS_CODEC = "Lucene41PostingsWriterPos";
+ final static String PAY_CODEC = "Lucene41PostingsWriterPay";
+
+ // Increment version to change it
+ final static int VERSION_START = 0;
+ final static int VERSION_META_ARRAY = 1;
+ final static int VERSION_CHECKSUM = 2;
+ final static int VERSION_CURRENT = VERSION_CHECKSUM;
+
+ /**
+ * Fixed packed block size, number of integers encoded in
+ * a single packed block.
+ */
+ // NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
+ public final static int BLOCK_SIZE = 128;
+
+ /** Creates {@code Lucene41PostingsFormat} with default
+ * settings. */
+ public Lucene41PostingsFormat() {
+ super("Lucene41");
+ }
+
+ @Override
+ public String toString() {
+ return getName() + "(blocksize=" + BLOCK_SIZE + ")";
+ }
+
+ @Override
+ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+
+ @Override
+ public final FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+ PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
+ state.fieldInfos,
+ state.segmentInfo,
+ state.context,
+ state.segmentSuffix);
+ boolean success = false;
+ try {
+ FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
+ success = true;
+ return ret;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(postingsReader);
+ }
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
similarity index 97%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
index 26cb34b8823..2b7eb57616f 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
@@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene41;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
-import static org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter.IntBlockTermState;
import java.io.IOException;
import java.util.Arrays;
@@ -32,6 +31,7 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
@@ -48,12 +48,10 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/**
- * Concrete class that reads docId(maybe frq,pos,offset,payloads) list
- * with postings format.
- *
- * @see Lucene41SkipReader for details
- * @lucene.experimental
+ * Lucene 4.1 postings format.
+ * @deprecated only for reading old 4.x segments
*/
+@Deprecated
public final class Lucene41PostingsReader extends PostingsReaderBase {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41PostingsReader.class);
@@ -77,12 +75,12 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
ioContext);
version = CodecUtil.checkHeader(docIn,
- Lucene41PostingsWriter.DOC_CODEC,
- Lucene41PostingsWriter.VERSION_START,
- Lucene41PostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsFormat.DOC_CODEC,
+ Lucene41PostingsFormat.VERSION_START,
+ Lucene41PostingsFormat.VERSION_CURRENT);
forUtil = new ForUtil(docIn);
- if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
+ if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// 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
@@ -93,9 +91,9 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (fieldInfos.hasProx()) {
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
ioContext);
- CodecUtil.checkHeader(posIn, Lucene41PostingsWriter.POS_CODEC, version, version);
+ CodecUtil.checkHeader(posIn, Lucene41PostingsFormat.POS_CODEC, version, version);
- if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
+ if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// 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
@@ -106,9 +104,9 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
ioContext);
- CodecUtil.checkHeader(payIn, Lucene41PostingsWriter.PAY_CODEC, version, version);
+ CodecUtil.checkHeader(payIn, Lucene41PostingsFormat.PAY_CODEC, version, version);
- if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
+ if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// 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
@@ -130,12 +128,12 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
}
@Override
- public void init(IndexInput termsIn) throws IOException {
+ public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
- Lucene41PostingsWriter.TERMS_CODEC,
- Lucene41PostingsWriter.VERSION_START,
- Lucene41PostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsFormat.TERMS_CODEC,
+ Lucene41PostingsFormat.VERSION_START,
+ Lucene41PostingsFormat.VERSION_CURRENT);
final int indexBlockSize = termsIn.readVInt();
if (indexBlockSize != BLOCK_SIZE) {
throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");
@@ -187,7 +185,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
termState.posStartFP = 0;
termState.payStartFP = 0;
}
- if (version < Lucene41PostingsWriter.VERSION_META_ARRAY) { // backward compatibility
+ if (version < Lucene41PostingsFormat.VERSION_META_ARRAY) { // backward compatibility
_decodeTerm(in, fieldInfo, termState);
return;
}
@@ -488,7 +486,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (skipper == null) {
// Lazy init: first time this enum has ever been used for skipping
skipper = new Lucene41SkipReader(docIn.clone(),
- Lucene41PostingsWriter.maxSkipLevels,
+ Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
indexHasPos,
indexHasOffsets,
@@ -821,7 +819,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
// System.out.println(" create skipper");
// }
skipper = new Lucene41SkipReader(docIn.clone(),
- Lucene41PostingsWriter.maxSkipLevels,
+ Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@@ -1347,7 +1345,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
// System.out.println(" create skipper");
// }
skipper = new Lucene41SkipReader(docIn.clone(),
- Lucene41PostingsWriter.maxSkipLevels,
+ Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@@ -1590,7 +1588,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
@Override
public void checkIntegrity() throws IOException {
- if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
+ if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
if (docIn != null) {
CodecUtil.checksumEntireFile(docIn);
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
similarity index 84%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
index 483b0ec21df..4f57430bdc7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
@@ -24,32 +24,10 @@ import org.apache.lucene.codecs.MultiLevelSkipListReader;
import org.apache.lucene.store.IndexInput;
/**
- * Implements the skip list reader for block postings format
- * that stores positions and payloads.
- *
- * Although this skipper uses MultiLevelSkipListReader as an interface,
- * its definition of skip position will be a little different.
- *
- * For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6,
- *
- * 0 1 2 3 4 5
- * d d d d d d (posting list)
- * ^ ^ (skip point in MultiLeveSkipWriter)
- * ^ (skip point in Lucene41SkipWriter)
- *
- * In this case, MultiLevelSkipListReader will use the last document as a skip point,
- * while Lucene41SkipReader should assume no skip point will comes.
- *
- * If we use the interface directly in Lucene41SkipReader, it may silly try to read
- * another skip data after the only skip point is loaded.
- *
- * To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId,
- * and numSkipped+blockSize== df, the MultiLevelSkipListReader will assume the skip list
- * isn't exhausted yet, and try to load a non-existed skip point
- *
- * Therefore, we'll trim df before passing it to the interface. see trim(int)
- *
+ * Lucene 4.1 skiplist format.
+ * @deprecated only for reading old 4.x segments
*/
+@Deprecated
final class Lucene41SkipReader extends MultiLevelSkipListReader {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private final int blockSize;
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index 112a1698302..023d9c9e1a6 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
@@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
+org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
index cff96cf38bf..bac0e4bd6f1 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
@@ -129,7 +129,7 @@ final class Lucene40PostingsWriter extends PushPostingsWriterBase {
}
@Override
- public void init(IndexOutput termsOut) throws IOException {
+ public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
termsOut.writeInt(skipInterval); // write skipInterval
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
similarity index 87%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
index 40147176584..41d735aaad5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
@@ -26,7 +26,6 @@ import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@@ -38,35 +37,13 @@ import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
-
/**
- * Concrete class that writes docId(maybe frq,pos,offset,payloads) list
- * with postings format.
- *
- * Postings list for each term will be stored separately.
- *
- * @see Lucene41SkipWriter for details about skipping setting and postings layout.
- * @lucene.experimental
+ * Writes 4.1 postings for testing
+ * @deprecated for test purposes only
*/
+@Deprecated
public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
- /**
- * Expert: The maximum number of skip levels. Smaller values result in
- * slightly smaller indexes, but slower skipping in big posting lists.
- */
- static final int maxSkipLevels = 10;
-
- final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
- final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
- final static String POS_CODEC = "Lucene41PostingsWriterPos";
- final static String PAY_CODEC = "Lucene41PostingsWriterPay";
-
- // Increment version to change it
- final static int VERSION_START = 0;
- final static int VERSION_META_ARRAY = 1;
- final static int VERSION_CHECKSUM = 2;
- final static int VERSION_CURRENT = VERSION_CHECKSUM;
-
IndexOutput docOut;
IndexOutput posOut;
IndexOutput payOut;
@@ -119,13 +96,13 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
IndexOutput payOut = null;
boolean success = false;
try {
- CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
+ CodecUtil.writeHeader(docOut, Lucene41PostingsFormat.DOC_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
forUtil = new ForUtil(acceptableOverheadRatio, docOut);
if (state.fieldInfos.hasProx()) {
posDeltaBuffer = new int[MAX_DATA_SIZE];
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
state.context);
- CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
+ CodecUtil.writeHeader(posOut, Lucene41PostingsFormat.POS_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
if (state.fieldInfos.hasPayloads()) {
payloadBytes = new byte[128];
@@ -146,7 +123,7 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
state.context);
- CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
+ CodecUtil.writeHeader(payOut, Lucene41PostingsFormat.PAY_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
}
} else {
posDeltaBuffer = null;
@@ -168,7 +145,7 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
freqBuffer = new int[MAX_DATA_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
- skipWriter = new Lucene41SkipWriter(maxSkipLevels,
+ skipWriter = new Lucene41SkipWriter(Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
state.segmentInfo.getDocCount(),
docOut,
@@ -183,50 +160,14 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
this(state, PackedInts.COMPACT);
}
- final static class IntBlockTermState extends BlockTermState {
- long docStartFP = 0;
- long posStartFP = 0;
- long payStartFP = 0;
- long skipOffset = -1;
- long lastPosBlockOffset = -1;
- // docid when there is a single pulsed posting, otherwise -1
- // freq is always implicitly totalTermFreq in this case.
- int singletonDocID = -1;
-
- @Override
- public IntBlockTermState clone() {
- IntBlockTermState other = new IntBlockTermState();
- other.copyFrom(this);
- return other;
- }
-
- @Override
- public void copyFrom(TermState _other) {
- super.copyFrom(_other);
- IntBlockTermState other = (IntBlockTermState) _other;
- docStartFP = other.docStartFP;
- posStartFP = other.posStartFP;
- payStartFP = other.payStartFP;
- lastPosBlockOffset = other.lastPosBlockOffset;
- skipOffset = other.skipOffset;
- singletonDocID = other.singletonDocID;
- }
-
-
- @Override
- public String toString() {
- return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
- }
- }
-
@Override
public IntBlockTermState newTermState() {
return new IntBlockTermState();
}
@Override
- public void init(IndexOutput termsOut) throws IOException {
- CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
+ public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
+ CodecUtil.writeHeader(termsOut, Lucene41PostingsFormat.TERMS_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
termsOut.writeVInt(BLOCK_SIZE);
}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
index d07e370ad57..16779c86e14 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
@@ -3,6 +3,7 @@ package org.apache.lucene.codecs.lucene41;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
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;
@@ -40,6 +41,12 @@ public final class Lucene41RWCodec extends Lucene41Codec {
private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
private final NormsFormat norms = new Lucene40RWNormsFormat();
private final TermVectorsFormat vectors = new Lucene40RWTermVectorsFormat();
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
@Override
public FieldInfosFormat fieldInfosFormat() {
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWPostingsFormat.java
new file mode 100644
index 00000000000..2679ef416ae
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWPostingsFormat.java
@@ -0,0 +1,56 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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.FieldsConsumer;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Read-write version of 4.1 postings format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public class Lucene41RWPostingsFormat extends Lucene41PostingsFormat {
+
+ static final int MIN_BLOCK_SIZE = 25;
+ static final int MAX_BLOCK_SIZE = 48;
+
+ @Override
+ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+
+ boolean success = false;
+ try {
+ FieldsConsumer ret = new BlockTreeTermsWriter(state,
+ postingsWriter,
+ MIN_BLOCK_SIZE,
+ MAX_BLOCK_SIZE);
+ success = true;
+ return ret;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(postingsWriter);
+ }
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
similarity index 86%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
index fdb65d73d8b..fd1b61c8928 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
@@ -24,25 +24,10 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
/**
- * Write skip lists with multiple levels, and support skip within block ints.
- *
- * Assume that docFreq = 28, skipInterval = blockSize = 12
- *
- * | block#0 | | block#1 | |vInts|
- * d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list)
- * ^ ^ (level 0 skip point)
- *
- * Note that skipWriter will ignore first document in block#0, since
- * it is useless as a skip point. Also, we'll never skip into the vInts
- * block, only record skip data at the start its start point(if it exist).
- *
- * For each skip point, we will record:
- * 1. docID in former position, i.e. for position 12, record docID[11], etc.
- * 2. its related file points(position, payload),
- * 3. related numbers or uptos(position, payload).
- * 4. start offset.
- *
+ * Writes 4.1 skiplists for testing
+ * @deprecated for test purposes only
*/
+@Deprecated
final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41ForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41ForUtil.java
new file mode 100644
index 00000000000..437758d777b
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41ForUtil.java
@@ -0,0 +1,94 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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 static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+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.store.RAMDirectory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.packed.PackedInts;
+
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
+
+public class TestLucene41ForUtil extends LuceneTestCase {
+
+ public void testEncodeDecode() throws IOException {
+ final int iterations = RandomInts.randomIntBetween(random(), 1, 1000);
+ final float acceptableOverheadRatio = random().nextFloat();
+ final int[] values = new int[(iterations - 1) * BLOCK_SIZE + ForUtil.MAX_DATA_SIZE];
+ for (int i = 0; i < iterations; ++i) {
+ final int bpv = random().nextInt(32);
+ if (bpv == 0) {
+ final int value = RandomInts.randomIntBetween(random(), 0, Integer.MAX_VALUE);
+ for (int j = 0; j < BLOCK_SIZE; ++j) {
+ values[i * BLOCK_SIZE + j] = value;
+ }
+ } else {
+ for (int j = 0; j < BLOCK_SIZE; ++j) {
+ values[i * BLOCK_SIZE + j] = RandomInts.randomIntBetween(random(),
+ 0, (int) PackedInts.maxValue(bpv));
+ }
+ }
+ }
+
+ final Directory d = new RAMDirectory();
+ final long endPointer;
+
+ {
+ // encode
+ IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
+ final ForUtil forUtil = new ForUtil(acceptableOverheadRatio, out);
+
+ for (int i = 0; i < iterations; ++i) {
+ forUtil.writeBlock(
+ Arrays.copyOfRange(values, i * BLOCK_SIZE, values.length),
+ new byte[MAX_ENCODED_SIZE], out);
+ }
+ endPointer = out.getFilePointer();
+ out.close();
+ }
+
+ {
+ // decode
+ IndexInput in = d.openInput("test.bin", IOContext.READONCE);
+ final ForUtil forUtil = new ForUtil(in);
+ for (int i = 0; i < iterations; ++i) {
+ if (random().nextBoolean()) {
+ forUtil.skipBlock(in);
+ continue;
+ }
+ final int[] restored = new int[MAX_DATA_SIZE];
+ forUtil.readBlock(in, new byte[MAX_ENCODED_SIZE], restored);
+ assertArrayEquals(Arrays.copyOfRange(values, i * BLOCK_SIZE, (i + 1) * BLOCK_SIZE),
+ Arrays.copyOf(restored, BLOCK_SIZE));
+ }
+ assertEquals(endPointer, in.getFilePointer());
+ in.close();
+ }
+ }
+
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat.java
new file mode 100644
index 00000000000..0cf41e99bf4
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat.java
@@ -0,0 +1,66 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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 org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.blocktree.FieldReader;
+import org.apache.lucene.codecs.blocktree.Stats;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.BasePostingsFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Tests BlockPostingsFormat
+ */
+public class TestLucene41PostingsFormat extends BasePostingsFormatTestCase {
+ private final Codec codec = new Lucene41RWCodec();
+
+ @Override
+ protected Codec getCodec() {
+ return codec;
+ }
+
+ /** Make sure the final sub-block(s) are not skipped. */
+ public void testFinalBlock() throws Exception {
+ Directory d = newDirectory();
+ IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
+ for(int i=0;i<25;i++) {
+ Document doc = new Document();
+ doc.add(newStringField("field", Character.toString((char) (97+i)), Field.Store.NO));
+ doc.add(newStringField("field", "z" + Character.toString((char) (97+i)), Field.Store.NO));
+ w.addDocument(doc);
+ }
+ w.forceMerge(1);
+
+ DirectoryReader r = DirectoryReader.open(w, true);
+ assertEquals(1, r.leaves().size());
+ FieldReader field = (FieldReader) r.leaves().get(0).reader().fields().terms("field");
+ // We should see exactly two blocks: one root block (prefix empty string) and one block for z* terms (prefix z):
+ Stats stats = field.computeStats();
+ assertEquals(0, stats.floorBlockCount);
+ assertEquals(2, stats.nonFloorBlockCount);
+ r.close();
+ w.close();
+ d.close();
+ }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat2.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat2.java
new file mode 100644
index 00000000000..8ebd506a4da
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat2.java
@@ -0,0 +1,132 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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 org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests special cases of BlockPostingsFormat
+ */
+
+public class TestLucene41PostingsFormat2 extends LuceneTestCase {
+ Directory dir;
+ RandomIndexWriter iw;
+
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ dir = newFSDirectory(createTempDir("testDFBlockSize"));
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+ iwc.setCodec(new Lucene41RWCodec());
+ iw = new RandomIndexWriter(random(), dir, iwc);
+ iw.setDoRandomForceMerge(false); // we will ourselves
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ iw.close();
+ TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+ iwc.setCodec(new Lucene41RWCodec());
+ iwc.setOpenMode(OpenMode.APPEND);
+ IndexWriter iw = new IndexWriter(dir, iwc);
+ iw.forceMerge(1);
+ iw.close();
+ dir.close(); // just force a checkindex for now
+ super.tearDown();
+ }
+
+ private Document newDocument() {
+ Document doc = new Document();
+ for (IndexOptions option : FieldInfo.IndexOptions.values()) {
+ FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+ // turn on tvs for a cross-check, since we rely upon checkindex in this test (for now)
+ ft.setStoreTermVectors(true);
+ ft.setStoreTermVectorOffsets(true);
+ ft.setStoreTermVectorPositions(true);
+ ft.setStoreTermVectorPayloads(true);
+ ft.setIndexOptions(option);
+ doc.add(new Field(option.toString(), "", ft));
+ }
+ return doc;
+ }
+
+ /** tests terms with df = blocksize */
+ public void testDFBlockSize() throws Exception {
+ Document doc = newDocument();
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
+ for (Field f : doc.getFields()) {
+ f.setStringValue(f.name() + " " + f.name() + "_2");
+ }
+ iw.addDocument(doc);
+ }
+ }
+
+ /** tests terms with df % blocksize = 0 */
+ public void testDFBlockSizeMultiple() throws Exception {
+ Document doc = newDocument();
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
+ for (Field f : doc.getFields()) {
+ f.setStringValue(f.name() + " " + f.name() + "_2");
+ }
+ iw.addDocument(doc);
+ }
+ }
+
+ /** tests terms with ttf = blocksize */
+ public void testTTFBlockSize() throws Exception {
+ Document doc = newDocument();
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
+ for (Field f : doc.getFields()) {
+ f.setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
+ }
+ iw.addDocument(doc);
+ }
+ }
+
+ /** tests terms with ttf % blocksize = 0 */
+ public void testTTFBlockSizeMultiple() throws Exception {
+ Document doc = newDocument();
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
+ for (Field f : doc.getFields()) {
+ String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " "
+ + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");
+ StringBuilder val = new StringBuilder();
+ for (int j = 0; j < 16; j++) {
+ val.append(proto);
+ val.append(" ");
+ }
+ f.setStringValue(val.toString());
+ }
+ iw.addDocument(doc);
+ }
+ }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat3.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat3.java
new file mode 100644
index 00000000000..30a92734b9f
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat3.java
@@ -0,0 +1,521 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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.HashSet;
+import java.util.Random;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockFixedLengthPayloadFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.MockVariableLengthPayloadFilter;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
+
+/**
+ * Tests partial enumeration (only pulling a subset of the indexed data)
+ */
+public class TestLucene41PostingsFormat3 extends LuceneTestCase {
+ static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
+
+ // creates 8 fields with different options and does "duels" of fields against each other
+ public void test() throws Exception {
+ Directory dir = newDirectory();
+ Analyzer analyzer = new Analyzer(Analyzer.PER_FIELD_REUSE_STRATEGY) {
+ @Override
+ protected TokenStreamComponents createComponents(String fieldName) {
+ Tokenizer tokenizer = new MockTokenizer();
+ if (fieldName.contains("payloadsFixed")) {
+ TokenFilter filter = new MockFixedLengthPayloadFilter(new Random(0), tokenizer, 1);
+ return new TokenStreamComponents(tokenizer, filter);
+ } else if (fieldName.contains("payloadsVariable")) {
+ TokenFilter filter = new MockVariableLengthPayloadFilter(new Random(0), tokenizer);
+ return new TokenStreamComponents(tokenizer, filter);
+ } else {
+ return new TokenStreamComponents(tokenizer);
+ }
+ }
+ };
+ IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+ iwc.setCodec(new Lucene41RWCodec());
+ // TODO we could actually add more fields implemented with different PFs
+ // or, just put this test into the usual rotation?
+ RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+ Document doc = new Document();
+ FieldType docsOnlyType = new FieldType(TextField.TYPE_NOT_STORED);
+ // turn this on for a cross-check
+ docsOnlyType.setStoreTermVectors(true);
+ docsOnlyType.setIndexOptions(IndexOptions.DOCS_ONLY);
+
+ FieldType docsAndFreqsType = new FieldType(TextField.TYPE_NOT_STORED);
+ // turn this on for a cross-check
+ docsAndFreqsType.setStoreTermVectors(true);
+ docsAndFreqsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+
+ FieldType positionsType = new FieldType(TextField.TYPE_NOT_STORED);
+ // turn these on for a cross-check
+ positionsType.setStoreTermVectors(true);
+ positionsType.setStoreTermVectorPositions(true);
+ positionsType.setStoreTermVectorOffsets(true);
+ positionsType.setStoreTermVectorPayloads(true);
+ FieldType offsetsType = new FieldType(positionsType);
+ offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+ Field field1 = new Field("field1docs", "", docsOnlyType);
+ Field field2 = new Field("field2freqs", "", docsAndFreqsType);
+ Field field3 = new Field("field3positions", "", positionsType);
+ Field field4 = new Field("field4offsets", "", offsetsType);
+ Field field5 = new Field("field5payloadsFixed", "", positionsType);
+ Field field6 = new Field("field6payloadsVariable", "", positionsType);
+ Field field7 = new Field("field7payloadsFixedOffsets", "", offsetsType);
+ Field field8 = new Field("field8payloadsVariableOffsets", "", offsetsType);
+ doc.add(field1);
+ doc.add(field2);
+ doc.add(field3);
+ doc.add(field4);
+ doc.add(field5);
+ doc.add(field6);
+ doc.add(field7);
+ doc.add(field8);
+ for (int i = 0; i < MAXDOC; i++) {
+ String stringValue = Integer.toString(i) + " verycommon " + English.intToEnglish(i).replace('-', ' ') + " " + TestUtil.randomSimpleString(random());
+ field1.setStringValue(stringValue);
+ field2.setStringValue(stringValue);
+ field3.setStringValue(stringValue);
+ field4.setStringValue(stringValue);
+ field5.setStringValue(stringValue);
+ field6.setStringValue(stringValue);
+ field7.setStringValue(stringValue);
+ field8.setStringValue(stringValue);
+ iw.addDocument(doc);
+ }
+ iw.close();
+ verify(dir);
+ TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
+ iwc = newIndexWriterConfig(analyzer);
+ iwc.setCodec(new Lucene41RWCodec());
+ iwc.setOpenMode(OpenMode.APPEND);
+ IndexWriter iw2 = new IndexWriter(dir, iwc);
+ iw2.forceMerge(1);
+ iw2.close();
+ verify(dir);
+ dir.close();
+ }
+
+ private void verify(Directory dir) throws Exception {
+ DirectoryReader ir = DirectoryReader.open(dir);
+ for (LeafReaderContext leaf : ir.leaves()) {
+ LeafReader leafReader = leaf.reader();
+ assertTerms(leafReader.terms("field1docs"), leafReader.terms("field2freqs"), true);
+ assertTerms(leafReader.terms("field3positions"), leafReader.terms("field4offsets"), true);
+ assertTerms(leafReader.terms("field4offsets"), leafReader.terms("field5payloadsFixed"), true);
+ assertTerms(leafReader.terms("field5payloadsFixed"), leafReader.terms("field6payloadsVariable"), true);
+ assertTerms(leafReader.terms("field6payloadsVariable"), leafReader.terms("field7payloadsFixedOffsets"), true);
+ assertTerms(leafReader.terms("field7payloadsFixedOffsets"), leafReader.terms("field8payloadsVariableOffsets"), true);
+ }
+ ir.close();
+ }
+
+ // following code is almost an exact dup of code from TestDuelingCodecs: sorry!
+
+ public void assertTerms(Terms leftTerms, Terms rightTerms, boolean deep) throws Exception {
+ if (leftTerms == null || rightTerms == null) {
+ assertNull(leftTerms);
+ assertNull(rightTerms);
+ return;
+ }
+ assertTermsStatistics(leftTerms, rightTerms);
+
+ // NOTE: we don't assert hasOffsets/hasPositions/hasPayloads because they are allowed to be different
+
+ TermsEnum leftTermsEnum = leftTerms.iterator(null);
+ TermsEnum rightTermsEnum = rightTerms.iterator(null);
+ assertTermsEnum(leftTermsEnum, rightTermsEnum, true);
+
+ assertTermsSeeking(leftTerms, rightTerms);
+
+ if (deep) {
+ int numIntersections = atLeast(3);
+ for (int i = 0; i < numIntersections; i++) {
+ String re = AutomatonTestUtil.randomRegexp(random());
+ CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toAutomaton());
+ if (automaton.type == CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+ // TODO: test start term too
+ TermsEnum leftIntersection = leftTerms.intersect(automaton, null);
+ TermsEnum rightIntersection = rightTerms.intersect(automaton, null);
+ assertTermsEnum(leftIntersection, rightIntersection, rarely());
+ }
+ }
+ }
+ }
+
+ private void assertTermsSeeking(Terms leftTerms, Terms rightTerms) throws Exception {
+ TermsEnum leftEnum = null;
+ TermsEnum rightEnum = null;
+
+ // just an upper bound
+ int numTests = atLeast(20);
+ Random random = random();
+
+ // collect this number of terms from the left side
+ HashSet tests = new HashSet<>();
+ int numPasses = 0;
+ while (numPasses < 10 && tests.size() < numTests) {
+ leftEnum = leftTerms.iterator(leftEnum);
+ BytesRef term = null;
+ while ((term = leftEnum.next()) != null) {
+ int code = random.nextInt(10);
+ if (code == 0) {
+ // the term
+ tests.add(BytesRef.deepCopyOf(term));
+ } else if (code == 1) {
+ // truncated subsequence of term
+ term = BytesRef.deepCopyOf(term);
+ if (term.length > 0) {
+ // truncate it
+ term.length = random.nextInt(term.length);
+ }
+ } else if (code == 2) {
+ // term, but ensure a non-zero offset
+ byte newbytes[] = new byte[term.length+5];
+ System.arraycopy(term.bytes, term.offset, newbytes, 5, term.length);
+ tests.add(new BytesRef(newbytes, 5, term.length));
+ }
+ }
+ numPasses++;
+ }
+
+ ArrayList shuffledTests = new ArrayList<>(tests);
+ Collections.shuffle(shuffledTests, random);
+
+ for (BytesRef b : shuffledTests) {
+ leftEnum = leftTerms.iterator(leftEnum);
+ rightEnum = rightTerms.iterator(rightEnum);
+
+ assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
+ assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
+
+ SeekStatus leftStatus;
+ SeekStatus rightStatus;
+
+ leftStatus = leftEnum.seekCeil(b);
+ rightStatus = rightEnum.seekCeil(b);
+ assertEquals(leftStatus, rightStatus);
+ if (leftStatus != SeekStatus.END) {
+ assertEquals(leftEnum.term(), rightEnum.term());
+ }
+
+ leftStatus = leftEnum.seekCeil(b);
+ rightStatus = rightEnum.seekCeil(b);
+ assertEquals(leftStatus, rightStatus);
+ if (leftStatus != SeekStatus.END) {
+ assertEquals(leftEnum.term(), rightEnum.term());
+ }
+ }
+ }
+
+ /**
+ * checks collection-level statistics on Terms
+ */
+ public void assertTermsStatistics(Terms leftTerms, Terms rightTerms) throws Exception {
+ if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) {
+ assertEquals(leftTerms.getDocCount(), rightTerms.getDocCount());
+ }
+ if (leftTerms.getSumDocFreq() != -1 && rightTerms.getSumDocFreq() != -1) {
+ assertEquals(leftTerms.getSumDocFreq(), rightTerms.getSumDocFreq());
+ }
+ if (leftTerms.getSumTotalTermFreq() != -1 && rightTerms.getSumTotalTermFreq() != -1) {
+ assertEquals(leftTerms.getSumTotalTermFreq(), rightTerms.getSumTotalTermFreq());
+ }
+ if (leftTerms.size() != -1 && rightTerms.size() != -1) {
+ assertEquals(leftTerms.size(), rightTerms.size());
+ }
+ }
+
+ /**
+ * checks the terms enum sequentially
+ * if deep is false, it does a 'shallow' test that doesnt go down to the docsenums
+ */
+ public void assertTermsEnum(TermsEnum leftTermsEnum, TermsEnum rightTermsEnum, boolean deep) throws Exception {
+ BytesRef term;
+ Bits randomBits = new RandomBits(MAXDOC, random().nextDouble(), random());
+ DocsAndPositionsEnum leftPositions = null;
+ DocsAndPositionsEnum rightPositions = null;
+ DocsEnum leftDocs = null;
+ DocsEnum rightDocs = null;
+
+ while ((term = leftTermsEnum.next()) != null) {
+ assertEquals(term, rightTermsEnum.next());
+ assertTermStats(leftTermsEnum, rightTermsEnum);
+ if (deep) {
+ // with payloads + off
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
+
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
+ // with payloads only
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
+
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
+
+ // with offsets only
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
+
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
+
+ // with positions only
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
+ assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
+
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
+ rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
+ assertPositionsSkipping(leftTermsEnum.docFreq(),
+ leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
+ rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
+
+ // with freqs:
+ assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs),
+ rightDocs = rightTermsEnum.docs(null, rightDocs));
+ assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
+ rightDocs = rightTermsEnum.docs(randomBits, rightDocs));
+
+ // w/o freqs:
+ assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+ rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
+ assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+ rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
+
+ // with freqs:
+ assertDocsSkipping(leftTermsEnum.docFreq(),
+ leftDocs = leftTermsEnum.docs(null, leftDocs),
+ rightDocs = rightTermsEnum.docs(null, rightDocs));
+ assertDocsSkipping(leftTermsEnum.docFreq(),
+ leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
+ rightDocs = rightTermsEnum.docs(randomBits, rightDocs));
+
+ // w/o freqs:
+ assertDocsSkipping(leftTermsEnum.docFreq(),
+ leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+ rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
+ assertDocsSkipping(leftTermsEnum.docFreq(),
+ leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+ rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
+ }
+ }
+ assertNull(rightTermsEnum.next());
+ }
+
+ /**
+ * checks term-level statistics
+ */
+ public void assertTermStats(TermsEnum leftTermsEnum, TermsEnum rightTermsEnum) throws Exception {
+ assertEquals(leftTermsEnum.docFreq(), rightTermsEnum.docFreq());
+ if (leftTermsEnum.totalTermFreq() != -1 && rightTermsEnum.totalTermFreq() != -1) {
+ assertEquals(leftTermsEnum.totalTermFreq(), rightTermsEnum.totalTermFreq());
+ }
+ }
+
+ /**
+ * checks docs + freqs + positions + payloads, sequentially
+ */
+ public void assertDocsAndPositionsEnum(DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws Exception {
+ if (leftDocs == null || rightDocs == null) {
+ assertNull(leftDocs);
+ assertNull(rightDocs);
+ return;
+ }
+ assertEquals(-1, leftDocs.docID());
+ assertEquals(-1, rightDocs.docID());
+ int docid;
+ while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ assertEquals(docid, rightDocs.nextDoc());
+ int freq = leftDocs.freq();
+ assertEquals(freq, rightDocs.freq());
+ for (int i = 0; i < freq; i++) {
+ assertEquals(leftDocs.nextPosition(), rightDocs.nextPosition());
+ // we don't assert offsets/payloads, they are allowed to be different
+ }
+ }
+ assertEquals(DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
+ }
+
+ /**
+ * checks docs + freqs, sequentially
+ */
+ public void assertDocsEnum(DocsEnum leftDocs, DocsEnum rightDocs) throws Exception {
+ if (leftDocs == null) {
+ assertNull(rightDocs);
+ return;
+ }
+ assertEquals(-1, leftDocs.docID());
+ assertEquals(-1, rightDocs.docID());
+ int docid;
+ while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ assertEquals(docid, rightDocs.nextDoc());
+ // we don't assert freqs, they are allowed to be different
+ }
+ assertEquals(DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
+ }
+
+ /**
+ * checks advancing docs
+ */
+ public void assertDocsSkipping(int docFreq, DocsEnum leftDocs, DocsEnum rightDocs) throws Exception {
+ if (leftDocs == null) {
+ assertNull(rightDocs);
+ return;
+ }
+ int docid = -1;
+ int averageGap = MAXDOC / (1+docFreq);
+ int skipInterval = 16;
+
+ while (true) {
+ if (random().nextBoolean()) {
+ // nextDoc()
+ docid = leftDocs.nextDoc();
+ assertEquals(docid, rightDocs.nextDoc());
+ } else {
+ // advance()
+ int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
+ docid = leftDocs.advance(skip);
+ assertEquals(docid, rightDocs.advance(skip));
+ }
+
+ if (docid == DocIdSetIterator.NO_MORE_DOCS) {
+ return;
+ }
+ // we don't assert freqs, they are allowed to be different
+ }
+ }
+
+ /**
+ * checks advancing docs + positions
+ */
+ public void assertPositionsSkipping(int docFreq, DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws Exception {
+ if (leftDocs == null || rightDocs == null) {
+ assertNull(leftDocs);
+ assertNull(rightDocs);
+ return;
+ }
+
+ int docid = -1;
+ int averageGap = MAXDOC / (1+docFreq);
+ int skipInterval = 16;
+
+ while (true) {
+ if (random().nextBoolean()) {
+ // nextDoc()
+ docid = leftDocs.nextDoc();
+ assertEquals(docid, rightDocs.nextDoc());
+ } else {
+ // advance()
+ int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
+ docid = leftDocs.advance(skip);
+ assertEquals(docid, rightDocs.advance(skip));
+ }
+
+ if (docid == DocIdSetIterator.NO_MORE_DOCS) {
+ return;
+ }
+ int freq = leftDocs.freq();
+ assertEquals(freq, rightDocs.freq());
+ for (int i = 0; i < freq; i++) {
+ assertEquals(leftDocs.nextPosition(), rightDocs.nextPosition());
+ // we don't compare the payloads, its allowed that one is empty etc
+ }
+ }
+ }
+
+ private static class RandomBits implements Bits {
+ FixedBitSet bits;
+
+ RandomBits(int maxDoc, double pctLive, Random random) {
+ bits = new FixedBitSet(maxDoc);
+ for (int i = 0; i < maxDoc; i++) {
+ if (random.nextDouble() <= pctLive) {
+ bits.set(i);
+ }
+ }
+ }
+
+ @Override
+ public boolean get(int index) {
+ return bits.get(index);
+ }
+
+ @Override
+ public int length() {
+ return bits.length();
+ }
+ }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
index 22f891d510e..c8da82d5c27 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/Lucene410RWCodec.java
@@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene410;
import org.apache.lucene.codecs.DocValuesFormat;
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.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@@ -34,6 +36,13 @@ import org.apache.lucene.codecs.lucene49.Lucene49RWNormsFormat;
@Deprecated
public final class Lucene410RWCodec extends Lucene410Codec {
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
+
private static final DocValuesFormat docValues = new Lucene410RWDocValuesFormat();
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
index f612f4a975f..baa0eb9bd7f 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java
@@ -26,8 +26,6 @@ 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.codecs.blocktreeords.Ords41PostingsFormat;
-import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
@@ -121,17 +119,7 @@ public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormat
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
- final PostingsFormat pf;
- switch (random().nextInt(2)) {
- case 0: pf = new Lucene41WithOrds();
- break;
- case 1: pf = new Ords41PostingsFormat();
- break;
- // TODO: these don't actually support ords!
- //case 2: pf = new FSTOrdPostingsFormat();
- // break;
- default: throw new AssertionError();
- }
+ final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene410RWDocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
index c708d114a21..9c465714c3f 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
@@ -20,10 +20,12 @@ package org.apache.lucene.codecs.lucene42;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
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.lucene40.Lucene40RWSegmentInfoFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
/**
@@ -37,6 +39,13 @@ public final class Lucene42RWCodec extends Lucene42Codec {
private static final NormsFormat norms = new Lucene42RWNormsFormat();
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
+
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
index 6752eb1c7b4..6f42569abc5 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
@@ -20,10 +20,12 @@ package org.apache.lucene.codecs.lucene45;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
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.lucene40.Lucene40RWSegmentInfoFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWFieldInfosFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
@@ -36,6 +38,13 @@ import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
@Deprecated
public final class Lucene45RWCodec extends Lucene45Codec {
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
+
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java
index ea9e00be014..62631c9e539 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java
@@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene46;
import org.apache.lucene.codecs.DocValuesFormat;
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.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
@@ -34,6 +36,13 @@ import org.apache.lucene.codecs.lucene45.Lucene45RWDocValuesFormat;
@Deprecated
public final class Lucene46RWCodec extends Lucene46Codec {
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
+
private static final DocValuesFormat docValues = new Lucene45RWDocValuesFormat();
@Override
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java
index 3f24c13b3f8..c000488ef0b 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java
@@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene49;
import org.apache.lucene.codecs.DocValuesFormat;
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.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@@ -33,6 +35,13 @@ import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@Deprecated
public final class Lucene49RWCodec extends Lucene49Codec {
+ private final PostingsFormat postings = new Lucene41RWPostingsFormat();
+
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return postings;
+ }
+
private static final DocValuesFormat docValues = new Lucene49RWDocValuesFormat();
@Override
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
index fadb54b6672..635655da84f 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
@@ -120,7 +120,7 @@ public class BlockTermsReader extends FieldsProducer {
state.segmentInfo.getId(), state.segmentSuffix);
// Have PostingsReader init itself
- postingsReader.init(in);
+ postingsReader.init(in, state);
// 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
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
index c504751d333..868f6cec4ba 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
@@ -117,7 +117,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
//System.out.println("BTW.init seg=" + state.segmentName);
- postingsWriter.init(out); // have consumer write its format/header
+ postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/Ords41PostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java
similarity index 79%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/Ords41PostingsFormat.java
rename to lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java
index 797fe6b5e73..61e8f84cbcb 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/Ords41PostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java
@@ -24,14 +24,14 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
-/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene41PostingsWriter}. */
-public class Ords41PostingsFormat extends PostingsFormat {
+/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene50PostingsWriter}. */
+public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
private final int minTermBlockSize;
private final int maxTermBlockSize;
@@ -45,7 +45,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
/** Creates {@code Lucene41PostingsFormat} with default
* settings. */
- public Ords41PostingsFormat() {
+ public BlockTreeOrdsPostingsFormat() {
this(OrdsBlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, OrdsBlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
@@ -53,8 +53,8 @@ public class Ords41PostingsFormat extends PostingsFormat {
* values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
* @see OrdsBlockTreeTermsWriter#OrdsBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
- public Ords41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
- super("OrdsLucene41");
+ public BlockTreeOrdsPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+ super("BlockTreeOrds");
this.minTermBlockSize = minTermBlockSize;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@@ -68,7 +68,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+ PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@@ -87,11 +87,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
- state.fieldInfos,
- state.segmentInfo,
- state.context,
- state.segmentSuffix);
+ PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java
index b45b90bb618..0cbc543a145 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsReader.java
@@ -94,7 +94,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
CodecUtil.checksumEntireFile(indexIn);
// Have PostingsReader init itself
- postingsReader.init(in);
+ postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
index ffa991044c4..c8b1f8b4135 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
@@ -214,7 +214,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("BTW.init seg=" + state.segmentName);
- postingsWriter.init(out); // have consumer write its format/header
+ postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
index 418ae8c6834..d2a7624ce11 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
@@ -26,7 +26,7 @@ import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.Transition;
// - build depth-N prefix hash?
// - or: longer dense skip lists than just next byte?
-/** Wraps {@link Lucene41PostingsFormat} format for on-disk
+/** Wraps {@link Lucene50PostingsFormat} format for on-disk
* storage, but then at read time loads and stores all
* terms & postings directly in RAM as byte[], int[].
*
@@ -102,12 +102,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- return PostingsFormat.forName("Lucene41").fieldsConsumer(state);
+ return PostingsFormat.forName("Lucene50").fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- FieldsProducer postings = PostingsFormat.forName("Lucene41").fieldsProducer(state);
+ FieldsProducer postings = PostingsFormat.forName("Lucene50").fieldsProducer(state);
if (state.context.context != IOContext.Context.MERGE) {
FieldsProducer loadedPostings;
try {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java
index 516ee84d132..98a293bbbd8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java
@@ -25,19 +25,19 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
- * FSTOrd term dict + Lucene41PBF
+ * FSTOrd term dict + Lucene50PBF
*/
public final class FSTOrdPostingsFormat extends PostingsFormat {
public FSTOrdPostingsFormat() {
- super("FSTOrd41");
+ super("FSTOrd50");
}
@Override
@@ -47,7 +47,7 @@ public final class FSTOrdPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+ PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@@ -63,11 +63,7 @@ public final class FSTOrdPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
- state.fieldInfos,
- state.segmentInfo,
- state.context,
- state.segmentSuffix);
+ PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTOrdTermsReader(state, postingsReader);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
index 6537794d06b..c9f2696089f 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java
@@ -103,7 +103,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
CodecUtil.checksumEntireFile(blockIn);
- this.postingsReader.init(blockIn);
+ this.postingsReader.init(blockIn, state);
seekDir(blockIn);
final FieldInfos fieldInfos = state.fieldInfos;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
index 5d495bc3836..104e26d8bf0 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
@@ -178,7 +178,7 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.writeSegmentHeader(blockOut, TERMS_CODEC_NAME, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
- this.postingsWriter.init(blockOut);
+ this.postingsWriter.init(blockOut, state);
success = true;
} finally {
if (!success) {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java
index 4b317830265..f25f3337940 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java
@@ -25,19 +25,19 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
- * FST term dict + Lucene41PBF
+ * FST term dict + Lucene50PBF
*/
public final class FSTPostingsFormat extends PostingsFormat {
public FSTPostingsFormat() {
- super("FST41");
+ super("FST50");
}
@Override
@@ -47,7 +47,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+ PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@@ -63,11 +63,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
- state.fieldInfos,
- state.segmentInfo,
- state.context,
- state.segmentSuffix);
+ PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTTermsReader(state, postingsReader);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
index 4297550e635..d5379745543 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
@@ -86,7 +86,7 @@ public class FSTTermsReader extends FieldsProducer {
FSTTermsWriter.TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.checksumEntireFile(in);
- this.postingsReader.init(in);
+ this.postingsReader.init(in, state);
seekDir(in);
final FieldInfos fieldInfos = state.fieldInfos;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
index 2d908e5caeb..c423ac1719e 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
@@ -145,7 +145,7 @@ public class FSTTermsWriter extends FieldsConsumer {
CodecUtil.writeSegmentHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
- this.postingsWriter.init(out);
+ this.postingsWriter.init(out, state);
success = true;
} finally {
if (!success) {
diff --git a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index a7cf270af48..726d55e5704 100644
--- a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
@@ -13,7 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat
+org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
index 4a4a7c20da6..9cd5e85cb30 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@@ -26,7 +25,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using FixedGap terms dictionary
*/
public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
- private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41WithOrds(TestUtil.nextInt(random(), 1, 1000)));
+ private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneFixedGap(TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapDocFreqIntervalPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapDocFreqIntervalPostingsFormat.java
index 59608f86a16..b70dc6ec574 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapDocFreqIntervalPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapDocFreqIntervalPostingsFormat.java
@@ -18,7 +18,7 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
+import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@@ -26,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using VariableGap terms dictionary (fixed interval)
*/
public class TestVarGapDocFreqIntervalPostingsFormat extends BasePostingsFormatTestCase {
- private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41VarGapFixedInterval(TestUtil.nextInt(random(), 1, 1000)));
+ private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneVarGapFixedInterval(TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java
index d0935a1c33b..8ea4d1755c0 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java
@@ -18,7 +18,7 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
+import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@@ -26,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using VariableGap terms dictionary (fixed interval, docFreq threshold)
*/
public class TestVarGapFixedIntervalPostingsFormat extends BasePostingsFormatTestCase {
- private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41VarGapDocFreqInterval(TestUtil.nextInt(random(), 1, 100), TestUtil.nextInt(random(), 1, 1000)));
+ private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneVarGapDocFreqInterval(TestUtil.nextInt(random(), 1, 100), TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java b/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java
index 9c418c55038..341b8a39e93 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java
@@ -39,7 +39,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
public class TestOrdsBlockTree extends BasePostingsFormatTestCase {
- private final Codec codec = TestUtil.alwaysPostingsFormat(new Ords41PostingsFormat());
+ private final Codec codec = TestUtil.alwaysPostingsFormat(new BlockTreeOrdsPostingsFormat());
@Override
protected Codec getCodec() {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java
deleted file mode 100644
index 198c2082bec..00000000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package org.apache.lucene.codecs;
-
-/*
- * 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.index.SegmentWriteState;
-import org.apache.lucene.index.SegmentReadState;
-
-/**
- * Provides a {@link PostingsReaderBase} and {@link
- * PostingsWriterBase}.
- *
- * @lucene.experimental */
-
-// TODO: find a better name; this defines the API that the
-// terms dict impls use to talk to a postings impl.
-// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
-
-// can we clean this up and do this some other way?
-// refactor some of these classes and use covariant return?
-public abstract class PostingsBaseFormat {
-
- /** Unique name that's used to retrieve this codec when
- * reading the index */
- public final String name;
-
- /** Sole constructor. */
- protected PostingsBaseFormat(String name) {
- this.name = name;
- }
-
- /** Creates the {@link PostingsReaderBase} for this
- * format. */
- public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
-
- /** Creates the {@link PostingsWriterBase} for this
- * format. */
- public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
index 7145fe9a810..5681c1904bc 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
@@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
@@ -53,7 +54,7 @@ public abstract class PostingsReaderBase implements Closeable, Accountable {
/** Performs any initialization, such as reading and
* verifying the header from the provided terms
* dictionary {@link IndexInput}. */
- public abstract void init(IndexInput termsIn) throws IOException;
+ public abstract void init(IndexInput termsIn, SegmentReadState state) throws IOException;
/** Return a newly created empty TermState */
public abstract BlockTermState newTermState() throws IOException;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java
index 6e083164bec..0dc7bb5b41c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java
@@ -24,6 +24,7 @@ import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
@@ -50,7 +51,7 @@ public abstract class PostingsWriterBase implements Closeable {
/** Called once after startup, before any terms have been
* added. Implementations typically write a header to
* the provided {@code termsOut}. */
- public abstract void init(IndexOutput termsOut) throws IOException;
+ public abstract void init(IndexOutput termsOut, SegmentWriteState state) throws IOException;
/** Write all postings for one term; use the provided
* {@link TermsEnum} to pull a {@link DocsEnum} or {@link
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
index a310cf65abf..0ce70324612 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
@@ -24,8 +24,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@@ -72,11 +70,6 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
protected PushPostingsWriterBase() {
}
- /** Called once after startup, before any terms have been
- * added. Implementations typically write a header to
- * the provided {@code termsOut}. */
- public abstract void init(IndexOutput termsOut) throws IOException;
-
/** Return a newly created empty TermState */
public abstract BlockTermState newTermState() throws IOException;
@@ -90,26 +83,11 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
* and will holds metadata from PBF when returned */
public abstract void finishTerm(BlockTermState state) throws IOException;
- /**
- * Encode metadata as long[] and byte[]. {@code absolute} controls whether
- * current term is delta encoded according to latest term.
- * Usually elements in {@code longs} are file pointers, so each one always
- * increases when a new term is consumed. {@code out} is used to write generic
- * bytes, which are not monotonic.
- *
- * NOTE: sometimes long[] might contain "don't care" values that are unused, e.g.
- * the pointer to postings list may not be defined for some terms but is defined
- * for others, if it is designed to inline some postings data in term dictionary.
- * In this case, the postings writer should always use the last value, so that each
- * element in metadata long[] remains monotonic.
- */
- public abstract void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
-
/**
* Sets the current field for writing, and returns the
* fixed length of long[] metadata (which is fixed per
* field), called when the writing switches to another field. */
- // TODO: better name?
+ @Override
public int setField(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.getIndexOptions();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
index b72d086b6f5..07f3c8cb02f 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
@@ -30,12 +30,9 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
@@ -96,24 +93,22 @@ public final class BlockTreeTermsReader extends FieldsProducer {
private final int version;
/** Sole constructor. */
- public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
- PostingsReaderBase postingsReader, IOContext ioContext,
- String segmentSuffix)
+ public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state)
throws IOException {
this.postingsReader = postingsReader;
- this.segment = info.name;
- in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
- ioContext);
+ this.segment = state.segmentInfo.name;
+ String termsFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION);
+ in = state.directory.openInput(termsFileName, state.context);
boolean success = false;
IndexInput indexIn = null;
try {
version = readHeader(in);
- indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
- ioContext);
+ String indexFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
+ indexIn = state.directory.openInput(indexFileName, state.context);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion, indexIn);
@@ -125,7 +120,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
}
// Have PostingsReader init itself
- postingsReader.init(in);
+ postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
@@ -158,7 +153,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
final BytesRef rootCode = new BytesRef(new byte[numBytes]);
in.readBytes(rootCode.bytes, 0, numBytes);
rootCode.length = numBytes;
- final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+ final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
throw new CorruptIndexException("invalid field number: " + field, in);
}
@@ -176,8 +171,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
} else {
minTerm = maxTerm = null;
}
- if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
- throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount(), in);
+ if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
+ throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
index 39a1becfd01..c7be417d575 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
@@ -329,7 +329,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("BTW.init seg=" + state.segmentName);
- postingsWriter.init(out); // have consumer write its format/header
+ postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
index a4326412737..6780e2a0dce 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
@@ -22,7 +22,7 @@ import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.util.Locale;
-import org.apache.lucene.codecs.PostingsBaseFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@@ -81,11 +81,11 @@ public class Stats {
public long totalBlockSuffixBytes;
/** Total number of bytes used to store term stats (not
- * including what the {@link PostingsBaseFormat}
+ * including what the {@link PostingsReaderBase}
* stores. */
public long totalBlockStatsBytes;
- /** Total bytes stored by the {@link PostingsBaseFormat},
+ /** Total bytes stored by the {@link PostingsReaderBase},
* plus the other few vInts stored in the frame. */
public long totalBlockOtherBytes;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
deleted file mode 100644
index c6931623640..00000000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.lucene.codecs.lucene41;
-
-/*
- * 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.PostingsBaseFormat;
-import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-
-/**
- * Provides a {@link PostingsReaderBase} and {@link
- * PostingsWriterBase}.
- *
- * @lucene.experimental */
-
-// TODO: should these also be named / looked up via SPI?
-public final class Lucene41PostingsBaseFormat extends PostingsBaseFormat {
-
- /** Sole constructor. */
- public Lucene41PostingsBaseFormat() {
- super("Lucene41");
- }
-
- @Override
- public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
- return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
- }
-
- @Override
- public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
- return new Lucene41PostingsWriter(state);
- }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
deleted file mode 100644
index abea0c2767e..00000000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-
-
-
-
-
-
-
-Lucene 4.1 file format.
-
-
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/ForUtil.java
similarity index 98%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene50/ForUtil.java
index 64178590c57..97b99982e3e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/ForUtil.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene41;
+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
@@ -27,7 +27,7 @@ import org.apache.lucene.util.packed.PackedInts.Decoder;
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
import org.apache.lucene.util.packed.PackedInts;
-import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
/**
* Encode all values in normal area with fixed bit width,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
index b74c4bebd2d..80417ef707e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
@@ -106,7 +106,7 @@ public class Lucene50Codec extends Codec {
/** Returns the postings format that should be used for writing
* new segments of field
.
*
- * The default implementation always returns "Lucene41"
+ * The default implementation always returns "Lucene50"
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
@@ -126,7 +126,7 @@ public class Lucene50Codec extends Codec {
return docValuesFormat;
}
- private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
private final NormsFormat normsFormat = new Lucene50NormsFormat();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
similarity index 91%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
index c0861ae3a78..7a27c2283bb 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene41;
+package org.apache.lucene.codecs.lucene50;
/*
@@ -38,12 +38,9 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
- * Lucene 4.1 postings format, which encodes postings in packed integer blocks
+ * Lucene 5.0 postings format, which encodes postings in packed integer blocks
* for fast decode.
*
- * NOTE: this format is still experimental and
- * subject to change without backwards compatibility.
- *
*
* Basic idea:
*
@@ -58,7 +55,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* -
* Block structure:
- *
When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data
+ *
When the postings are long enough, Lucene50PostingsFormat will try to encode most integer data
* as a packed block.
* Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed
* blocks, while the remaining 3 are encoded as one VInt block.
@@ -129,14 +126,14 @@ import org.apache.lucene.util.packed.PackedInts;
* - PostingsHeader --> Header, PackedBlockSize
* - TermMetadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?,
* SkipFPDelta?
- * - Header, --> {@link CodecUtil#writeHeader CodecHeader}
+ * - Header, --> {@link CodecUtil#writeSegmentHeader SegmentHeader}
* - PackedBlockSize, SingletonDocID --> {@link DataOutput#writeVInt VInt}
* - DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --> {@link DataOutput#writeVLong VLong}
* - Footer --> {@link CodecUtil#writeFooter CodecFooter}
*
* Notes:
*
- * - Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information
+ *
- Header is a {@link CodecUtil#writeSegmentHeader SegmentHeader} storing the version information
* for the postings.
* - PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width is
* determined by the largest integer. Smaller block size result in smaller variance among width
@@ -162,7 +159,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
- SkipFPDelta determines the position of this term's SkipData within the .doc
* file. In particular, it is the length of the TermFreq data.
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
- * (i.e. 128 in Lucene41PostingsFormat).
+ * (i.e. 128 in Lucene50PostingsFormat).
* SingletonDocID is an optimization when a term only appears in one document. In this case, instead
* of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
* single document ID is written to the term dictionary.
@@ -192,7 +189,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
*
* - docFile(.doc) --> Header, <TermFreqs, SkipData?>TermCount, Footer
- * - Header --> {@link CodecUtil#writeHeader CodecHeader}
+ * - Header --> {@link CodecUtil#writeSegmentHeader SegmentHeader}
* - TermFreqs --> <PackedBlock> PackedDocBlockNum,
* VIntBlock?
* - PackedBlock --> PackedDocDeltaBlock, PackedFreqBlock?
@@ -243,10 +240,10 @@ import org.apache.lucene.util.packed.PackedInts;
* We use this trick since the definition of skip entry is a little different from base interface.
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
* skipIntervalth, 2*skipIntervalth ... posting in the list. However,
- * in Lucene41PostingsFormat, the skip data is saved for skipInterval+1th,
+ * in Lucene50PostingsFormat, the skip data is saved for skipInterval+1th,
* 2*skipInterval+1th ... posting (skipInterval==PackedBlockSize in this case).
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
- * more skip data than Lucene41SkipWriter.
+ * more skip data than Lucene50SkipWriter.
* SkipDatum is the metadata of one skip entry.
* For the first block (no matter packed or VInt), it is omitted.
* DocSkip records the document number of every PackedBlockSizeth document number in
@@ -276,7 +273,7 @@ import org.apache.lucene.util.packed.PackedInts;
* sometimes stores part of payloads and offsets for speedup.
*