From b6f7024afbd46cb3cbf82eddadcb1298929a2dde Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Fri, 12 Oct 2012 00:52:57 +0000
Subject: [PATCH 01/20] Make branch to switch default index format
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397400 13f79535-47bb-0310-9956-ffa450edef68
From 54ff47eff00694cb4b8df366ee0188c2bba678af Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Fri, 12 Oct 2012 02:00:19 +0000
Subject: [PATCH 02/20] LUCENE-4446: quick stab at a start... I think core
tests pass but all else is TODO/untested
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397416 13f79535-47bb-0310-9956-ffa450edef68
---
.../apache/lucene/codecs/block/package.html | 25 --
...rmat.java => Pulsing41PostingsFormat.java} | 16 +-
.../org.apache.lucene.codecs.PostingsFormat | 3 +-
.../TestFixedGapPostingsFormat.java | 4 +-
.../codecs/bloom/TestBloomPostingsFormat.java | 2 +-
.../codecs/pulsing/Test10KPulsings.java | 5 +-
.../pulsing/TestPulsingPostingsFormat.java | 6 +-
.../codecs/pulsing/TestPulsingReuse.java | 2 +-
.../lucene/codecs/lucene41}/ForUtil.java | 4 +-
.../lucene/codecs/lucene41/Lucene41Codec.java | 122 ++++++
.../lucene41/Lucene41PostingsBaseFormat.java | 51 +++
.../lucene41/Lucene41PostingsFormat.java} | 24 +-
.../lucene41/Lucene41PostingsReader.java} | 78 ++--
.../lucene41/Lucene41PostingsWriter.java} | 38 +-
.../codecs/lucene41/Lucene41SkipReader.java} | 16 +-
.../codecs/lucene41/Lucene41SkipWriter.java} | 8 +-
.../lucene/codecs/lucene41/package.html | 396 ++++++++++++++++++
.../services/org.apache.lucene.codecs.Codec | 1 +
.../org.apache.lucene.codecs.PostingsFormat | 1 +
.../org/apache/lucene/TestExternalCodecs.java | 8 +-
.../lucene41}/TestBlockPostingsFormat.java | 13 +-
.../lucene41}/TestBlockPostingsFormat2.java | 15 +-
.../lucene41}/TestBlockPostingsFormat3.java | 10 +-
.../lucene/codecs/lucene41}/TestForUtil.java | 8 +-
.../perfield/TestPerFieldPostingsFormat2.java | 30 +-
.../apache/lucene/index/TestAddIndexes.java | 12 +-
.../TestAllFilesHaveCodecHeader.java | 6 +-
.../lucene/index/TestDirectoryReader.java | 6 +-
.../apache/lucene/index/TestDocTermOrds.java | 4 +-
.../lucene/index/TestDuelingCodecs.java | 2 +-
.../org/apache/lucene/index/TestFlex.java | 4 +-
.../lucene/index/TestMultiLevelSkipList.java | 4 +-
.../lucene/index/TestSegmentTermEnum.java | 4 +-
.../lucene/util/TestNamedSPILoader.java | 6 +-
...=> TestBloomFilteredLucene41Postings.java} | 12 +-
.../Lucene41WithOrds.java} | 20 +-
.../package.html | 0
.../org/apache/lucene/index/RandomCodec.java | 24 +-
.../util/TestRuleSetupAndRestoreClassEnv.java | 17 +-
.../org.apache.lucene.codecs.PostingsFormat | 4 +-
40 files changed, 771 insertions(+), 240 deletions(-)
delete mode 100644 lucene/codecs/src/java/org/apache/lucene/codecs/block/package.html
rename lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/{Pulsing40PostingsFormat.java => Pulsing41PostingsFormat.java} (75%)
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block => core/src/java/org/apache/lucene/codecs/lucene41}/ForUtil.java (98%)
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java => core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java} (96%)
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java => core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java} (95%)
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java => core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java} (93%)
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java => core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java} (91%)
rename lucene/{codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java => core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java} (94%)
create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
rename lucene/{codecs/src/test/org/apache/lucene/codecs/block => core/src/test/org/apache/lucene/codecs/lucene41}/TestBlockPostingsFormat.java (71%)
rename lucene/{codecs/src/test/org/apache/lucene/codecs/block => core/src/test/org/apache/lucene/codecs/lucene41}/TestBlockPostingsFormat2.java (91%)
rename lucene/{codecs/src/test/org/apache/lucene/codecs/block => core/src/test/org/apache/lucene/codecs/lucene41}/TestBlockPostingsFormat3.java (98%)
rename lucene/{codecs/src/test/org/apache/lucene/codecs/block => core/src/test/org/apache/lucene/codecs/lucene41}/TestForUtil.java (92%)
rename lucene/core/src/test/org/apache/lucene/{codecs/lucene40 => index}/TestAllFilesHaveCodecHeader.java (95%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/{TestBloomFilteredLucene40Postings.java => TestBloomFilteredLucene41Postings.java} (87%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene40ords/Lucene40WithOrds.java => lucene41ords/Lucene41WithOrds.java} (89%)
rename lucene/test-framework/src/java/org/apache/lucene/codecs/{lucene40ords => lucene41ords}/package.html (100%)
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/package.html b/lucene/codecs/src/java/org/apache/lucene/codecs/block/package.html
deleted file mode 100644
index c4fe9c6a8ea..00000000000
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-
-
-
-
-
-
-
-BlockPostingsFormat file format.
-
-
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
similarity index 75%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java
rename to lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
index faf8df2bbf8..7fd7fb0504a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
@@ -18,28 +18,28 @@ package org.apache.lucene.codecs.pulsing;
*/
import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsBaseFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
/**
- * Concrete pulsing implementation over {@link Lucene40PostingsFormat}.
+ * Concrete pulsing implementation over {@link Lucene41PostingsFormat}.
*
* @lucene.experimental
*/
-public class Pulsing40PostingsFormat extends PulsingPostingsFormat {
+public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
/** Inlines docFreq=1 terms, otherwise uses the normal "Lucene40" format. */
- public Pulsing40PostingsFormat() {
+ public Pulsing41PostingsFormat() {
this(1);
}
/** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
- public Pulsing40PostingsFormat(int freqCutoff) {
+ public Pulsing41PostingsFormat(int freqCutoff) {
this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
- public Pulsing40PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
- super("Pulsing40", new Lucene40PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
+ public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
+ super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
}
}
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 72b05c5e74e..22062983d91 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,9 +13,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat
+org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat
org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
org.apache.lucene.codecs.memory.MemoryPostingsFormat
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
-org.apache.lucene.codecs.block.BlockPostingsFormat
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 d45b6828d1a..055bc21e2fb 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
@@ -20,7 +20,7 @@ package org.apache.lucene.codecs.blockterms;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
-import org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds;
+import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
// TODO: we should add an instantiation for VarGap too to TestFramework, and a test in this package
// TODO: ensure both of these are also in rotation in RandomCodec
public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
- private final PostingsFormat postings = new Lucene40WithOrds();
+ private final PostingsFormat postings = new Lucene41WithOrds();
private final Codec codec = new Lucene40Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
index 3bd9a90935c..143163feead 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
@@ -26,7 +26,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
* Basic tests for BloomPostingsFormat
*/
public class TestBloomPostingsFormat extends BasePostingsFormatTestCase {
- private final PostingsFormat postings = new TestBloomFilteredLucene40Postings();
+ private final PostingsFormat postings = new TestBloomFilteredLucene41Postings();
private final Codec codec = new Lucene40Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
index 3e47dc549cc..6e53a631ea3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
@@ -25,7 +25,6 @@ import java.util.Locale;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -52,7 +51,7 @@ import org.apache.lucene.util._TestUtil;
public class Test10KPulsings extends LuceneTestCase {
public void test10kPulsed() throws Exception {
// we always run this test with pulsing codec.
- Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1));
+ Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1));
File f = _TestUtil.getTempDir("10kpulsed");
BaseDirectoryWrapper dir = newFSDirectory(f);
@@ -103,7 +102,7 @@ public class Test10KPulsings extends LuceneTestCase {
public void test10kNotPulsed() throws Exception {
// we always run this test with pulsing codec.
int freqCutoff = _TestUtil.nextInt(random(), 1, 10);
- Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(freqCutoff));
+ Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(freqCutoff));
File f = _TestUtil.getTempDir("10knotpulsed");
BaseDirectoryWrapper dir = newFSDirectory(f);
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
index 3156323df61..75271d33d17 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.pulsing;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -27,8 +27,8 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
*/
public class TestPulsingPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize cutoff
- private final PostingsFormat postings = new Pulsing40PostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final PostingsFormat postings = new Pulsing41PostingsFormat();
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
index 488fca34baf..cfa520aef02 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
@@ -45,7 +45,7 @@ public class TestPulsingReuse extends LuceneTestCase {
// TODO: this is a basic test. this thing is complicated, add more
public void testSophisticatedReuse() throws Exception {
// we always run this test with pulsing codec.
- Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1));
+ Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1));
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
similarity index 98%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
index fc52520f551..88f70a249c1 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+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
@@ -28,7 +28,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.block.BlockPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
/**
* Encode all values in normal area with fixed bit width,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
new file mode 100644
index 00000000000..48219582b75
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
@@ -0,0 +1,122 @@
+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.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 4.1 index format, with configurable per-field postings formats.
+ *
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene41 package documentation for file format details.
+ * @lucene.experimental
+ */
+// NOTE: if we make largish changes in a minor release, easier to just make Lucene42Codec or whatever
+// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
+// (it writes a minor version, etc).
+public class Lucene41Codec extends Codec {
+ private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
+ private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
+ private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
+ private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
+ private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
+ private final NormsFormat normsFormat = new Lucene40NormsFormat();
+ private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+
+ private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return Lucene41Codec.this.getPostingsFormatForField(field);
+ }
+ };
+
+ /** Sole constructor. */
+ public Lucene41Codec() {
+ super("Lucene41");
+ }
+
+ @Override
+ public final StoredFieldsFormat storedFieldsFormat() {
+ return fieldsFormat;
+ }
+
+ @Override
+ public final TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public final DocValuesFormat docValuesFormat() {
+ return docValuesFormat;
+ }
+
+ @Override
+ public final PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ @Override
+ public final FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
+ public final SegmentInfoFormat segmentInfoFormat() {
+ return infosFormat;
+ }
+
+ @Override
+ public final NormsFormat normsFormat() {
+ return normsFormat;
+ }
+
+ @Override
+ public final LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
+
+ /** Returns the postings format that should be used for writing
+ * new segments of field
.
+ *
+ * The default implementation always returns "Lucene41"
+ */
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return defaultFormat;
+ }
+
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+}
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
new file mode 100644
index 00000000000..0360c0d2709
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
@@ -0,0 +1,51 @@
+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.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ }
+
+ @Override
+ public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
+ return new Lucene41PostingsWriter(state);
+ }
+}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
similarity index 96%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
index 73ce8df62c4..d1c21ed2846 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
@@ -38,7 +38,7 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
- * Block postings format, which encodes postings in packed integer blocks
+ * Lucene 4.1 postings format, which encodes postings in packed integer blocks
* for fast decode.
*
*
NOTE: this format is still experimental and
@@ -58,7 +58,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
*
* Block structure:
- * When the postings are long enough, BlockPostingsFormat will try to encode most integer data
+ *
When the postings are long enough, Lucene41PostingsFormat 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.
@@ -161,7 +161,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. 8 in BlockPostingsFormat).
+ * (i.e. 8 in Lucene41PostingsFormat).
*
*
*
@@ -238,10 +238,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 BlockPostingsFormat, the skip data is saved for skipInterval+1th,
+ * in Lucene41PostingsFormat, 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 BlockSkipWriter.
+ * more skip data than Lucene41SkipWriter.
* 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
@@ -351,7 +351,7 @@ import org.apache.lucene.util.packed.PackedInts;
* @lucene.experimental
*/
-public final class BlockPostingsFormat extends PostingsFormat {
+public final class Lucene41PostingsFormat extends PostingsFormat {
/**
* Filename extension for document number, frequencies, and skip data.
* See chapter: Frequencies and Skip Data
@@ -380,12 +380,12 @@ public final class BlockPostingsFormat extends PostingsFormat {
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
public final static int BLOCK_SIZE = 128;
- public BlockPostingsFormat() {
+ public Lucene41PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
- public BlockPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
- super("Block");
+ public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+ super("Lucene41");
this.minTermBlockSize = minTermBlockSize;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@@ -399,7 +399,7 @@ public final class BlockPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase postingsWriter = new BlockPostingsWriter(state);
+ PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
boolean success = false;
try {
@@ -418,7 +418,7 @@ public final class BlockPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postingsReader = new BlockPostingsReader(state.dir,
+ PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.dir,
state.fieldInfos,
state.segmentInfo,
state.context,
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
similarity index 95%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
index 77b766743f0..7e14977bc25 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
-import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
+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;
@@ -49,10 +49,10 @@ import org.apache.lucene.util.IOUtils;
* Concrete class that reads docId(maybe frq,pos,offset,payloads) list
* with postings format.
*
- * @see BlockSkipReader for details
- *
+ * @see Lucene41SkipReader for details
+ * @lucene.experimental
*/
-final class BlockPostingsReader extends PostingsReaderBase {
+public final class Lucene41PostingsReader extends PostingsReaderBase {
private final IndexInput docIn;
private final IndexInput posIn;
@@ -62,35 +62,35 @@ final class BlockPostingsReader extends PostingsReaderBase {
// public static boolean DEBUG = false;
- public BlockPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
+ public Lucene41PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
boolean success = false;
IndexInput docIn = null;
IndexInput posIn = null;
IndexInput payIn = null;
try {
- docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
+ docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
ioContext);
CodecUtil.checkHeader(docIn,
- BlockPostingsWriter.DOC_CODEC,
- BlockPostingsWriter.VERSION_CURRENT,
- BlockPostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsWriter.DOC_CODEC,
+ Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.VERSION_CURRENT);
forUtil = new ForUtil(docIn);
if (fieldInfos.hasProx()) {
- posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.POS_EXTENSION),
+ posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
ioContext);
CodecUtil.checkHeader(posIn,
- BlockPostingsWriter.POS_CODEC,
- BlockPostingsWriter.VERSION_CURRENT,
- BlockPostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsWriter.POS_CODEC,
+ Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.VERSION_CURRENT);
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
- payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
+ payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
ioContext);
CodecUtil.checkHeader(payIn,
- BlockPostingsWriter.PAY_CODEC,
- BlockPostingsWriter.VERSION_CURRENT,
- BlockPostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsWriter.PAY_CODEC,
+ Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.VERSION_CURRENT);
}
}
@@ -109,9 +109,9 @@ final class BlockPostingsReader extends PostingsReaderBase {
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
- BlockPostingsWriter.TERMS_CODEC,
- BlockPostingsWriter.VERSION_CURRENT,
- BlockPostingsWriter.VERSION_CURRENT);
+ Lucene41PostingsWriter.TERMS_CODEC,
+ Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.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 + ")");
@@ -321,7 +321,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
- private BlockSkipReader skipper;
+ private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@@ -353,7 +353,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public BlockDocsEnum(FieldInfo fieldInfo) throws IOException {
- this.startDocIn = BlockPostingsReader.this.docIn;
+ this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@@ -486,8 +486,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
if (skipper == null) {
// Lazy init: first time this enum has ever been used for skipping
- skipper = new BlockSkipReader(docIn.clone(),
- BlockPostingsWriter.maxSkipLevels,
+ skipper = new Lucene41SkipReader(docIn.clone(),
+ Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
indexHasPos,
indexHasOffsets,
@@ -502,7 +502,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
skipped = true;
}
- // always plus one to fix the result, since skip position in BlockSkipReader
+ // always plus one to fix the result, since skip position in Lucene41SkipReader
// is a little different from MultiLevelSkipListReader
final int newDocUpto = skipper.skipTo(target) + 1;
@@ -577,7 +577,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
private int posBufferUpto;
- private BlockSkipReader skipper;
+ private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@@ -628,9 +628,9 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public BlockDocsAndPositionsEnum(FieldInfo fieldInfo) throws IOException {
- this.startDocIn = BlockPostingsReader.this.docIn;
+ this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
- this.posIn = BlockPostingsReader.this.posIn.clone();
+ this.posIn = Lucene41PostingsReader.this.posIn.clone();
encoded = new byte[MAX_ENCODED_SIZE];
indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
indexHasPayloads = fieldInfo.hasPayloads();
@@ -797,8 +797,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
// if (DEBUG) {
// System.out.println(" create skipper");
// }
- skipper = new BlockSkipReader(docIn.clone(),
- BlockPostingsWriter.maxSkipLevels,
+ skipper = new Lucene41SkipReader(docIn.clone(),
+ Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@@ -987,7 +987,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
private int posBufferUpto;
- private BlockSkipReader skipper;
+ private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@@ -1044,10 +1044,10 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public EverythingEnum(FieldInfo fieldInfo) throws IOException {
- this.startDocIn = BlockPostingsReader.this.docIn;
+ this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
- this.posIn = BlockPostingsReader.this.posIn.clone();
- this.payIn = BlockPostingsReader.this.payIn.clone();
+ this.posIn = Lucene41PostingsReader.this.posIn.clone();
+ this.payIn = Lucene41PostingsReader.this.payIn.clone();
encoded = new byte[MAX_ENCODED_SIZE];
indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (indexHasOffsets) {
@@ -1282,8 +1282,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
// if (DEBUG) {
// System.out.println(" create skipper");
// }
- skipper = new BlockSkipReader(docIn.clone(),
- BlockPostingsWriter.maxSkipLevels,
+ skipper = new Lucene41SkipReader(docIn.clone(),
+ Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
similarity index 93%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
index 305e1f38d51..51e2b02422f 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
-import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
+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.ArrayList;
@@ -47,10 +47,10 @@ import org.apache.lucene.util.packed.PackedInts;
*
* Postings list for each term will be stored separately.
*
- * @see BlockSkipWriter for details about skipping setting and postings layout.
- *
+ * @see Lucene41SkipWriter for details about skipping setting and postings layout.
+ * @lucene.experimental
*/
-final class BlockPostingsWriter extends PostingsWriterBase {
+public final class Lucene41PostingsWriter extends PostingsWriterBase {
/**
* Expert: The maximum number of skip levels. Smaller values result in
@@ -58,12 +58,12 @@ final class BlockPostingsWriter extends PostingsWriterBase {
*/
static final int maxSkipLevels = 10;
- final static String TERMS_CODEC = "BlockPostingsWriterTerms";
- final static String DOC_CODEC = "BlockPostingsWriterDoc";
- final static String POS_CODEC = "BlockPostingsWriterPos";
- final static String PAY_CODEC = "BlockPostingsWriterPay";
+ 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:
+ // Increment version to change it: nocommit: we can start at 0
final static int VERSION_START = 0;
final static int VERSION_NO_OFFSETS_IN_SKIPDATA = 1; // LUCENE-4443
final static int VERSION_CURRENT = VERSION_NO_OFFSETS_IN_SKIPDATA;
@@ -112,12 +112,12 @@ final class BlockPostingsWriter extends PostingsWriterBase {
final byte[] encoded;
private final ForUtil forUtil;
- private final BlockSkipWriter skipWriter;
+ private final Lucene41SkipWriter skipWriter;
- public BlockPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
+ public Lucene41PostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
super();
- docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
+ docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
state.context);
IndexOutput posOut = null;
IndexOutput payOut = null;
@@ -127,7 +127,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
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, BlockPostingsFormat.POS_EXTENSION),
+ posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
state.context);
CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
@@ -148,7 +148,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
}
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
- payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
+ payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
state.context);
CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
}
@@ -172,7 +172,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
freqBuffer = new int[MAX_DATA_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
- skipWriter = new BlockSkipWriter(maxSkipLevels,
+ skipWriter = new Lucene41SkipWriter(maxSkipLevels,
BLOCK_SIZE,
state.segmentInfo.getDocCount(),
docOut,
@@ -182,7 +182,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
encoded = new byte[MAX_ENCODED_SIZE];
}
- public BlockPostingsWriter(SegmentWriteState state) throws IOException {
+ public Lucene41PostingsWriter(SegmentWriteState state) throws IOException {
this(state, PackedInts.COMPACT);
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
similarity index 91%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
index e5803fd9696..483b0ec21df 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -35,12 +35,12 @@ import org.apache.lucene.store.IndexInput;
* 0 1 2 3 4 5
* d d d d d d (posting list)
* ^ ^ (skip point in MultiLeveSkipWriter)
- * ^ (skip point in BlockSkipWriter)
+ * ^ (skip point in Lucene41SkipWriter)
*
* In this case, MultiLevelSkipListReader will use the last document as a skip point,
- * while BlockSkipReader should assume no skip point will comes.
+ * while Lucene41SkipReader should assume no skip point will comes.
*
- * If we use the interface directly in BlockSkipReader, it may silly try to read
+ * 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,
@@ -50,8 +50,8 @@ import org.apache.lucene.store.IndexInput;
* Therefore, we'll trim df before passing it to the interface. see trim(int)
*
*/
-final class BlockSkipReader extends MultiLevelSkipListReader {
- // private boolean DEBUG = BlockPostingsReader.DEBUG;
+final class Lucene41SkipReader extends MultiLevelSkipListReader {
+ // private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private final int blockSize;
private long docPointer[];
@@ -66,7 +66,7 @@ final class BlockSkipReader extends MultiLevelSkipListReader {
private long lastDocPointer;
private int lastPosBufferUpto;
- public BlockSkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
+ public Lucene41SkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
super(skipStream, maxSkipLevels, blockSize, 8);
this.blockSize = blockSize;
docPointer = new long[maxSkipLevels];
@@ -91,7 +91,7 @@ final class BlockSkipReader extends MultiLevelSkipListReader {
/**
* Trim original docFreq to tell skipReader read proper number of skip points.
*
- * Since our definition in BlockSkip* is a little different from MultiLevelSkip*
+ * Since our definition in Lucene41Skip* is a little different from MultiLevelSkip*
* This trimmed docFreq will prevent skipReader from:
* 1. silly reading a non-existed skip point after the last block boundary
* 2. moving into the vInt block
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
similarity index 94%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
index 409930c6fed..1bd082859d9 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -43,8 +43,8 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
* 4. start offset.
*
*/
-final class BlockSkipWriter extends MultiLevelSkipListWriter {
- // private boolean DEBUG = BlockPostingsReader.DEBUG;
+final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
+ // private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private int[] lastSkipDoc;
private long[] lastSkipDocPointer;
@@ -66,7 +66,7 @@ final class BlockSkipWriter extends MultiLevelSkipListWriter {
private boolean fieldHasOffsets;
private boolean fieldHasPayloads;
- public BlockSkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
+ public Lucene41SkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
super(blockSize, 8, maxSkipLevels, docCount);
this.docOut = docOut;
this.posOut = posOut;
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
new file mode 100644
index 00000000000..aff3d7a572f
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
@@ -0,0 +1,396 @@
+
+
+
+
+
+
+
+Lucene 4.1 file format.
+
+Apache Lucene - Index File Formats
+
+
+Introduction
+
+
This document defines the index file formats used in this version of Lucene.
+If you are using a different version of Lucene, please consult the copy of
+docs/
that was distributed with
+the version you are using.
+
Apache Lucene is written in Java, but several efforts are underway to write
+versions of
+Lucene in other programming languages. If these versions are to remain
+compatible with Apache Lucene, then a language-independent definition of the
+Lucene index format is required. This document thus attempts to provide a
+complete and independent definition of the Apache Lucene file formats.
+
As Lucene evolves, this document should evolve. Versions of Lucene in
+different programming languages should endeavor to agree on file formats, and
+generate new versions of this document.
+
+
+Definitions
+
+
The fundamental concepts in Lucene are index, document, field and term.
+
An index contains a sequence of documents.
+
+- A document is a sequence of fields.
+- A field is a named sequence of terms.
+- A term is a sequence of bytes.
+
+
The same sequence of bytes in two different fields is considered a different
+term. Thus terms are represented as a pair: the string naming the field, and the
+bytes within the field.
+
+
Inverted Indexing
+
The index stores statistics about terms in order to make term-based search
+more efficient. Lucene's index falls into the family of indexes known as an
+inverted index. This is because it can list, for a term, the documents
+that contain it. This is the inverse of the natural relationship, in which
+documents list terms.
+
+
Types of Fields
+
In Lucene, fields may be stored, in which case their text is stored
+in the index literally, in a non-inverted manner. Fields that are inverted are
+called indexed. A field may be both stored and indexed.
+
The text of a field may be tokenized into terms to be indexed, or the
+text of a field may be used literally as a term to be indexed. Most fields are
+tokenized, but sometimes it is useful for certain identifier fields to be
+indexed literally.
+
See the {@link org.apache.lucene.document.Field Field}
+java docs for more information on Fields.
+
+
Segments
+
Lucene indexes may be composed of multiple sub-indexes, or segments.
+Each segment is a fully independent index, which could be searched separately.
+Indexes evolve by:
+
+- Creating new segments for newly added documents.
+- Merging existing segments.
+
+
Searches may involve multiple segments and/or multiple indexes, each index
+potentially composed of a set of segments.
+
+
Document Numbers
+
Internally, Lucene refers to documents by an integer document number.
+The first document added to an index is numbered zero, and each subsequent
+document added gets a number one greater than the previous.
+
Note that a document's number may change, so caution should be taken when
+storing these numbers outside of Lucene. In particular, numbers may change in
+the following situations:
+
+-
+
The numbers stored in each segment are unique only within the segment, and
+must be converted before they can be used in a larger context. The standard
+technique is to allocate each segment a range of values, based on the range of
+numbers used in that segment. To convert a document number from a segment to an
+external value, the segment's base document number is added. To convert
+an external value back to a segment-specific value, the segment is identified
+by the range that the external value is in, and the segment's base value is
+subtracted. For example two five document segments might be combined, so that
+the first segment has a base value of zero, and the second of five. Document
+three from the second segment would have an external value of eight.
+
+-
+
When documents are deleted, gaps are created in the numbering. These are
+eventually removed as the index evolves through merging. Deleted documents are
+dropped when segments are merged. A freshly-merged segment thus has no gaps in
+its numbering.
+
+
+
+
+Index Structure Overview
+
+
Each segment index maintains the following:
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment info}.
+ This contains metadata about a segment, such as the number of documents,
+ what files it uses,
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
+ This contains the set of field names used in the index.
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Stored Field values}.
+This contains, for each document, a list of attribute-value pairs, where the attributes
+are field names. These are used to store auxiliary information about the document, such as
+its title, url, or an identifier to access a database. The set of stored fields are what is
+returned for each hit when searching. This is keyed by document number.
+
+-
+{@link org.apache.lucene.codecs.lucene41Lucene41PostingsFormat Term dictionary}.
+A dictionary containing all of the terms used in all of the
+indexed fields of all of the documents. The dictionary also contains the number
+of documents which contain the term, and pointers to the term's frequency and
+proximity data.
+
+-
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
+For each term in the dictionary, the numbers of all the
+documents that contain that term, and the frequency of the term in that
+document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
+
+-
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
+For each term in the dictionary, the positions that the
+term occurs in each document. Note that this will not exist if all fields in
+all documents omit position data.
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Normalization factors}.
+For each field in each document, a value is stored
+that is multiplied into the score for hits on that field.
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vectors}.
+For each field in each document, the term vector (sometimes
+called document vector) may be stored. A term vector consists of term text and
+term frequency. To add Term Vectors to your index see the
+{@link org.apache.lucene.document.Field Field} constructors
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-document values}.
+Like stored values, these are also keyed by document
+number, but are generally intended to be loaded into main memory for fast
+access. Whereas stored values are generally intended for summary results from
+searches, per-document values are useful for things like scoring factors.
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
+An optional file indicating which documents are deleted.
+
+
+
Details on each of these are provided in their linked pages.
+
+
+File Naming
+
+
All files belonging to a segment have the same name with varying extensions.
+The extensions correspond to the different file formats described below. When
+using the Compound File format (default in 1.4 and greater) these files (except
+for the Segment info file, the Lock file, and Deleted documents file) are collapsed
+into a single .cfs file (see below for details)
+
Typically, all segments in an index are stored in a single directory,
+although this is not required.
+
As of version 2.1 (lock-less commits), file names are never re-used (there
+is one exception, "segments.gen", see below). That is, when any file is saved
+to the Directory it is given a never before used filename. This is achieved
+using a simple generations approach. For example, the first segments file is
+segments_1, then segments_2, etc. The generation is a sequential long integer
+represented in alpha-numeric (base 36) form.
+
+
+Summary of File Extensions
+
+
The following table summarizes the names and extensions of the files in
+Lucene:
+
+
+Name |
+Extension |
+Brief Description |
+
+
+{@link org.apache.lucene.index.SegmentInfos Segments File} |
+segments.gen, segments_N |
+Stores information about a commit point |
+
+
+Lock File |
+write.lock |
+The Write lock prevents multiple IndexWriters from writing to the same
+file. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info} |
+.si |
+Stores metadata about a segment |
+
+
+{@link org.apache.lucene.store.CompoundFileDirectory Compound File} |
+.cfs, .cfe |
+An optional "virtual" file consisting of all the other index files for
+systems that frequently run out of file handles. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Fields} |
+.fnm |
+Stores information about the fields |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Index} |
+.fdx |
+Contains pointers to field data |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Data} |
+.fdt |
+The stored fields for documents |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary} |
+.tim |
+The term dictionary, stores term info |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index} |
+.tip |
+The index into the Term Dictionary |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies} |
+.doc |
+Contains the list of docs which contain each term along with frequency |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions} |
+.pos |
+Stores position information about where a term occurs in the index |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads} |
+.pay |
+Stores additional per-position metadata information such as character offsets and user payloads |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Norms} |
+.nrm.cfs, .nrm.cfe |
+Encodes length and boost factors for docs and fields |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-Document Values} |
+.dv.cfs, .dv.cfe |
+Encodes additional scoring factors or other per-document information. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index} |
+.tvx |
+Stores offset into the document data file |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents} |
+.tvd |
+Contains information about each document that has term vectors |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields} |
+.tvf |
+The field level info about term vectors |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents} |
+.del |
+Info about what files are deleted |
+
+
+
+
+Lock File
+The write lock, which is stored in the index directory by default, is named
+"write.lock". If the lock directory is different from the index directory then
+the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
+derived from the full path to the index directory. When this file is present, a
+writer is currently modifying the index (adding or removing documents). This
+lock file ensures that only one writer is modifying the index at a time.
+
+History
+Compatibility notes are provided in this document, describing how file
+formats have changed from prior versions:
+
+- In version 2.1, the file format was changed to allow lock-less commits (ie,
+no more commit lock). The change is fully backwards compatible: you can open a
+pre-2.1 index for searching or adding/deleting of docs. When the new segments
+file is saved (committed), it will be written in the new file format (meaning
+no specific "upgrade" process is needed). But note that once a commit has
+occurred, pre-2.1 Lucene will not be able to read the index.
+- In version 2.3, the file format was changed to allow segments to share a
+single set of doc store (vectors & stored fields) files. This allows for
+faster indexing in certain cases. The change is fully backwards compatible (in
+the same way as the lock-less commits change in 2.1).
+- In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+Java's modified UTF-8. See
+LUCENE-510 for details.
+- In version 2.9, an optional opaque Map<String,String> CommitUserData
+may be passed to IndexWriter's commit methods (and later retrieved), which is
+recorded in the segments_N file. See
+LUCENE-1382 for details. Also,
+diagnostics were added to each segment written recording details about why it
+was written (due to flush, merge; which OS/JRE was used; etc.). See issue
+LUCENE-1654 for details.
+- In version 3.0, compressed fields are no longer written to the index (they
+can still be read, but on merge the new segment will write them, uncompressed).
+See issue LUCENE-1960
+for details.
+- In version 3.1, segments records the code version that created them. See
+LUCENE-2720 for details.
+Additionally segments track explicitly whether or not they have term vectors.
+See LUCENE-2811
+for details.
+- In version 3.2, numeric fields are written as natively to stored fields
+file, previously they were stored in text format only.
+- In version 3.4, fields can omit position data while still indexing term
+frequencies.
+- In version 4.0, the format of the inverted index became extensible via
+the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
+({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
+factors need no longer be a single byte, they can be any DocValues
+{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
+strings, they can be any byte sequence. Term offsets can optionally be indexed
+into the postings lists. Payloads can be stored in the term vectors.
+- In version 4.1, the format of the postings list changed to use either
+of FOR compression or variable-byte encoding, depending upon the frequency
+of the term.
+
+
+Limitations
+
+
When referring to term numbers, Lucene's current implementation uses a Java
+int
to hold the term index, which means the
+maximum number of unique terms in any single index segment is ~2.1 billion
+times the term index interval (default 128) = ~274 billion. This is technically
+not a limitation of the index file format, just of Lucene's current
+implementation.
+
Similarly, Lucene uses a Java int
to refer to
+document numbers, and the index file format uses an Int32
+on-disk to store document numbers. This is a limitation
+of both the index file format and the current implementation. Eventually these
+should be replaced with either UInt64
values, or
+better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.
+
+
+
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 82c3e5c58f8..de1cc736ea9 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40Codec
+org.apache.lucene.codecs.lucene41.Lucene41Codec
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index 112a1698302..023d9c9e1a6 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/core/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/core/src/test/org/apache/lucene/TestExternalCodecs.java b/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
index aa5bf5bd2d8..4dcb5353fc1 100644
--- a/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
@@ -19,7 +19,7 @@ package org.apache.lucene;
import org.apache.lucene.analysis.*;
import org.apache.lucene.codecs.*;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.document.*;
import org.apache.lucene.index.*;
import org.apache.lucene.search.*;
@@ -31,11 +31,11 @@ import org.apache.lucene.util.*;
public class TestExternalCodecs extends LuceneTestCase {
- private static final class CustomPerFieldCodec extends Lucene40Codec {
+ private static final class CustomPerFieldCodec extends Lucene41Codec {
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
- private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
- private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing40");
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+ private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing41");
@Override
public PostingsFormat getPostingsFormatForField(String field) {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat.java
similarity index 71%
rename from lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat.java
index 235c85fe2e7..dd3231e36c3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -18,22 +18,13 @@ package org.apache.lucene.codecs.block;
*/
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.block.BlockPostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
* Tests BlockPostingsFormat
*/
public class TestBlockPostingsFormat extends BasePostingsFormatTestCase {
- private final PostingsFormat postings = new BlockPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
- @Override
- public PostingsFormat getPostingsFormatForField(String field) {
- return postings;
- }
- };
+ private final Codec codec = new Lucene41Codec();
@Override
protected Codec getCodec() {
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat2.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
similarity index 91%
rename from lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat2.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
index 8b462d2e587..0a49540f73d 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat2.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.block;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -47,10 +46,10 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
super.setUp();
dir = newFSDirectory(_TestUtil.getTempDir("testDFBlockSize"));
iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- iwc.setCodec(new Lucene40Codec() {
+ iwc.setCodec(new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
- return PostingsFormat.forName("Block");
+ return PostingsFormat.forName("Lucene41");
}
});
iw = new RandomIndexWriter(random(), dir, iwc);
@@ -88,7 +87,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df = blocksize */
public void testDFBlockSize() throws Exception {
Document doc = newDocument();
- for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE; i++) {
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@@ -99,7 +98,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df % blocksize = 0 */
public void testDFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
- for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE * 16; i++) {
+ for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@@ -110,7 +109,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf = blocksize */
public void testTTFBlockSize() throws Exception {
Document doc = newDocument();
- for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
+ 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");
}
@@ -121,7 +120,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf % blocksize = 0 */
public void testTTFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
- for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
+ 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");
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat3.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
similarity index 98%
rename from lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat3.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
index 9ef0aae6726..34bd00789d3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat3.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -30,7 +30,7 @@ import org.apache.lucene.analysis.MockVariableLengthPayloadFilter;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -64,7 +64,7 @@ import org.apache.lucene.util.automaton.RegExp;
* Tests partial enumeration (only pulling a subset of the prox data)
*/
public class TestBlockPostingsFormat3 extends LuceneTestCase {
- static final int MAXDOC = BlockPostingsFormat.BLOCK_SIZE * 20;
+ static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
// creates 6 fields with different options and does "duels" of fields against each other
public void test() throws Exception {
@@ -85,10 +85,10 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
}
};
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
- iwc.setCodec(new Lucene40Codec() {
+ iwc.setCodec(new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
- return PostingsFormat.forName("Block");
+ return PostingsFormat.forName("Lucene41");
// TODO: we could actually add more fields implemented with different PFs
}
});
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestForUtil.java
similarity index 92%
rename from lucene/codecs/src/test/org/apache/lucene/codecs/block/TestForUtil.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestForUtil.java
index 025a6348164..3831033a6ea 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestForUtil.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestForUtil.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.block;
+package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
-import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
+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;
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
index dac3b5af94a..582e774d126 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
@@ -21,10 +21,10 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
-import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
+import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -142,7 +142,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
assertQuery(new Term("content", "ccc"), dir, 10);
assertQuery(new Term("content", "aaa"), dir, 10);
- Lucene40Codec codec = (Lucene40Codec)iwconf.getCodec();
+ Lucene41Codec codec = (Lucene41Codec)iwconf.getCodec();
iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
.setOpenMode(OpenMode.APPEND).setCodec(codec);
@@ -158,7 +158,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
addDocs2(writer, 10);
writer.commit();
- codec = (Lucene40Codec)iwconf.getCodec();
+ codec = (Lucene41Codec)iwconf.getCodec();
assertEquals(30, writer.maxDoc());
assertQuery(new Term("content", "bbb"), dir, 10);
assertQuery(new Term("content", "ccc"), dir, 10); ////
@@ -200,8 +200,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
- public static class MockCodec extends Lucene40Codec {
- final PostingsFormat lucene40 = new Lucene40PostingsFormat();
+ public static class MockCodec extends Lucene41Codec {
+ final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
final PostingsFormat mockSep = new MockSepPostingsFormat();
@@ -217,8 +217,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
}
- public static class MockCodec2 extends Lucene40Codec {
- final PostingsFormat lucene40 = new Lucene40PostingsFormat();
+ public static class MockCodec2 extends Lucene41Codec {
+ final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
@Override
@@ -268,13 +268,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentInstance() throws Exception {
- Codec codec = new Lucene40Codec() {
+ Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
- return new Pulsing40PostingsFormat(1);
+ return new Pulsing41PostingsFormat(1);
} else if ("date".equals(field)) {
- return new Pulsing40PostingsFormat(1);
+ return new Pulsing41PostingsFormat(1);
} else {
return super.getPostingsFormatForField(field);
}
@@ -284,13 +284,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentParams() throws Exception {
- Codec codec = new Lucene40Codec() {
+ Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
- return new Pulsing40PostingsFormat(1);
+ return new Pulsing41PostingsFormat(1);
} else if ("date".equals(field)) {
- return new Pulsing40PostingsFormat(2);
+ return new Pulsing41PostingsFormat(2);
} else {
return super.getPostingsFormatForField(field);
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
index 31b4d190f80..42db793e0db 100755
--- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
@@ -27,8 +27,8 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
-import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
+import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -1058,9 +1058,9 @@ public class TestAddIndexes extends LuceneTestCase {
aux2.close();
}
- private static final class CustomPerFieldCodec extends Lucene40Codec {
+ private static final class CustomPerFieldCodec extends Lucene41Codec {
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
- private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
@Override
@@ -1109,7 +1109,7 @@ public class TestAddIndexes extends LuceneTestCase {
private static final class UnRegisteredCodec extends FilterCodec {
public UnRegisteredCodec() {
- super("NotRegistered", new Lucene40Codec());
+ super("NotRegistered", new Lucene41Codec());
}
}
@@ -1138,7 +1138,7 @@ public class TestAddIndexes extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random()));
- conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1 + random().nextInt(20))));
+ conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1 + random().nextInt(20))));
IndexWriter w = new IndexWriter(dir, conf);
try {
w.addIndexes(toAdd);
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
similarity index 95%
rename from lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java
rename to lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
index d6535df143d..8466b90a521 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40;
+package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -35,13 +35,13 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
/**
- * Test that a plain Lucene40Codec puts codec headers in all files.
+ * Test that a plain default puts codec headers in all files.
*/
public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
public void test() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- conf.setCodec(Codec.forName("Lucene40"));
+ conf.setCodec(Codec.forName("Lucene41"));
// riw should sometimes create docvalues fields, etc
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
Document doc = new Document();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
index 3bc247dcc1f..08819143a1c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
@@ -28,7 +28,7 @@ import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -828,7 +828,7 @@ public void testFilesOpenClose() throws IOException {
// LUCENE-1609: don't load terms index
public void testNoTermsIndex() throws Throwable {
Directory dir = newDirectory();
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
+ IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
Document doc = new Document();
doc.add(newTextField("field", "a b c d e f g h i j k l m n o p q r s t u v w x y z", Field.Store.NO));
doc.add(newTextField("number", "0 1 2 3 4 5 6 7 8 9", Field.Store.NO));
@@ -848,7 +848,7 @@ public void testFilesOpenClose() throws IOException {
writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
- setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).
+ setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).
setMergePolicy(newLogMergePolicy(10))
);
writer.addDocument(doc);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java b/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
index 22b2360fcd8..5c5adce7650 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
@@ -110,7 +110,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
// Make sure terms index has ords:
- Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
+ Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
conf.setCodec(codec);
}
@@ -207,7 +207,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
- Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
+ Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
conf.setCodec(codec);
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
index 0c9bd4b5a79..bb304c4a666 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
@@ -61,7 +61,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
- // for now its SimpleText vs Lucene40(random postings format)
+ // for now its SimpleText vs Lucene41(random postings format)
// as this gives the best overall coverage. when we have more
// codecs we should probably pick 2 from Codec.availableCodecs()
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFlex.java b/lucene/core/src/test/org/apache/lucene/index/TestFlex.java
index 76a1ee53481..2bd65a9ab85 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFlex.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFlex.java
@@ -19,7 +19,7 @@ package org.apache.lucene.index;
import org.apache.lucene.store.*;
import org.apache.lucene.analysis.*;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.*;
import org.apache.lucene.util.*;
@@ -65,7 +65,7 @@ public class TestFlex extends LuceneTestCase {
public void testTermOrd() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT,
- new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
+ new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
Document doc = new Document();
doc.add(newTextField("f", "a b c", Field.Store.NO));
w.addDocument(doc);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java b/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
index 42de0b6e414..2ee0449b5d0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.*;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.Directory;
@@ -69,7 +69,7 @@ public class TestMultiLevelSkipList extends LuceneTestCase {
public void testSimpleSkip() throws IOException {
Directory dir = new CountingRAMDirectory(new RAMDirectory());
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).setMergePolicy(newLogMergePolicy()));
+ IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).setMergePolicy(newLogMergePolicy()));
Term term = new Term("test", "a");
for (int i = 0; i < 5000; i++) {
Document d1 = new Document();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java
index 4554243ca1a..1379a3c223e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermEnum.java
@@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.store.Directory;
@@ -75,7 +75,7 @@ public class TestSegmentTermEnum extends LuceneTestCase {
public void testPrevTermAtEnd() throws IOException
{
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
+ IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
addDoc(writer, "aaa bbb");
writer.close();
SegmentReader reader = getOnlySegmentReader(DirectoryReader.open(dir));
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java b/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
index ef7d047b06d..bff508ff8ea 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
@@ -25,8 +25,8 @@ import org.apache.lucene.codecs.Codec;
// enough to test the basics via Codec
public class TestNamedSPILoader extends LuceneTestCase {
public void testLookup() {
- Codec codec = Codec.forName("Lucene40");
- assertEquals("Lucene40", codec.getName());
+ Codec codec = Codec.forName("Lucene41");
+ assertEquals("Lucene41", codec.getName());
}
// we want an exception if its not found.
@@ -39,6 +39,6 @@ public class TestNamedSPILoader extends LuceneTestCase {
public void testAvailableServices() {
Set codecs = Codec.availableCodecs();
- assertTrue(codecs.contains("Lucene40"));
+ assertTrue(codecs.contains("Lucene41"));
}
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene41Postings.java
similarity index 87%
rename from lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java
rename to lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene41Postings.java
index 50c5a98bdfa..d5229f6f496 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene41Postings.java
@@ -22,19 +22,19 @@ 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.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
- * delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
+ * delegate (Lucene41). Creates a Bloom filter on ALL fields and with tiny
* amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
* APPLICATION This is not a realistic application of Bloom Filters as they
* ordinarily are larger and operate on only primary key type fields.
*/
-public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
+public final class TestBloomFilteredLucene41Postings extends PostingsFormat {
private BloomFilteringPostingsFormat delegate;
@@ -54,9 +54,9 @@ public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
}
}
- public TestBloomFilteredLucene40Postings() {
- super("TestBloomFilteredLucene40Postings");
- delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
+ public TestBloomFilteredLucene41Postings() {
+ super("TestBloomFilteredLucene41Postings");
+ delegate = new BloomFilteringPostingsFormat(new Lucene41PostingsFormat(),
new LowMemoryBloomFactory());
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java
similarity index 89%
rename from lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java
rename to lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java
index 42f0d853688..8865136be1b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/Lucene41WithOrds.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40ords;
+package org.apache.lucene.codecs.lucene41ords;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -30,9 +30,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec; // javadocs
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec; // javadocs
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.BytesRef;
@@ -41,18 +41,18 @@ import org.apache.lucene.util.BytesRef;
// any PostingsBaseFormat and make it ord-able...
/**
- * Customized version of {@link Lucene40Codec} that uses
+ * Customized version of {@link Lucene41Codec} that uses
* {@link FixedGapTermsIndexWriter}.
*/
-public final class Lucene40WithOrds extends PostingsFormat {
+public final class Lucene41WithOrds extends PostingsFormat {
- public Lucene40WithOrds() {
- super("Lucene40WithOrds");
+ public Lucene41WithOrds() {
+ super("Lucene41WithOrds");
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase docs = new Lucene40PostingsWriter(state);
+ PostingsWriterBase docs = new Lucene41PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@@ -91,7 +91,7 @@ public final class Lucene40WithOrds extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ PostingsReaderBase postings = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
TermsIndexReaderBase indexReader;
boolean success = false;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/package.html b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/package.html
similarity index 100%
rename from lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/package.html
rename to lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41ords/package.html
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index b8676d7dafc..03737a41983 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -29,11 +29,10 @@ import java.util.Set;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
-import org.apache.lucene.codecs.block.BlockPostingsFormat;
-import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
-import org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
+import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
+import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@@ -41,7 +40,7 @@ import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
-import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
+import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@@ -55,7 +54,7 @@ import org.apache.lucene.util._TestUtil;
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
-public class RandomCodec extends Lucene40Codec {
+public class RandomCodec extends Lucene41Codec {
/** Shuffled list of postings formats to use for new mappings */
private List formats = new ArrayList();
@@ -94,23 +93,22 @@ public class RandomCodec extends Lucene40Codec {
int lowFreqCutoff = _TestUtil.nextInt(random, 2, 100);
add(avoidCodecs,
- new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
- new BlockPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
+ new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
- new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
+ new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
// add pulsing again with (usually) different parameters
- new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
+ new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene40Postings to be constructed
//with a choice of concrete PostingsFormats. Maybe useful to have a generic means of marking and dealing
//with such "wrapper" classes?
- new TestBloomFilteredLucene40Postings(),
+ new TestBloomFilteredLucene41Postings(),
new MockSepPostingsFormat(),
new MockFixedIntBlockPostingsFormat(_TestUtil.nextInt(random, 1, 2000)),
new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)),
new MockRandomPostingsFormat(random),
new NestedPulsingPostingsFormat(),
- new Lucene40WithOrds(),
+ new Lucene41WithOrds(),
new SimpleTextPostingsFormat(),
new AssertingPostingsFormat(),
new MemoryPostingsFormat(true, random.nextFloat()),
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
index b432416c15b..afa1ccdb30e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
@@ -32,6 +32,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@@ -129,26 +130,24 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
avoidCodecs.addAll(Arrays.asList(a.value()));
}
- PREFLEX_IMPERSONATION_IS_ACTIVE = false;
savedCodec = Codec.getDefault();
int randomVal = random.nextInt(10);
- /* note: re-enable this if we make a 4.x impersonator
- if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
+
+ if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
"random".equals(TEST_POSTINGSFORMAT) &&
randomVal < 2 &&
- !shouldAvoidCodec("Lucene3x"))) { // preflex-only setup
- codec = Codec.forName("Lucene3x");
- assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
- PREFLEX_IMPERSONATION_IS_ACTIVE = true;
- } else */ if (!"random".equals(TEST_POSTINGSFORMAT)) {
+ !shouldAvoidCodec("Lucene40"))) {
+ codec = Codec.forName("Lucene40");
+ // nocommit: assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
+ } else if (!"random".equals(TEST_POSTINGSFORMAT)) {
final PostingsFormat format;
if ("MockRandom".equals(TEST_POSTINGSFORMAT)) {
format = new MockRandomPostingsFormat(random);
} else {
format = PostingsFormat.forName(TEST_POSTINGSFORMAT);
}
- codec = new Lucene40Codec() {
+ codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
diff --git a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index 4c82a0146bb..98619f2243b 100644
--- a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
@@ -19,7 +19,7 @@ org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat
org.apache.lucene.codecs.mocksep.MockSepPostingsFormat
org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
-org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds
-org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings
+org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
+org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
org.apache.lucene.codecs.asserting.AssertingPostingsFormat
From b1a58d9a9c9435bf6a9552f160b301fb1af87b65 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Fri, 12 Oct 2012 02:17:48 +0000
Subject: [PATCH 03/20] LUCENE-4446: cut over remaining tests etc
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397418 13f79535-47bb-0310-9956-ffa450edef68
---
.../codecs/blockterms/TestFixedGapPostingsFormat.java | 4 ++--
.../lucene/codecs/bloom/TestBloomPostingsFormat.java | 4 ++--
.../compressing/TestCompressingStoredFieldsFormat.java | 6 +++---
.../intblock/TestFixedIntBlockPostingsFormat.java | 4 ++--
.../intblock/TestVariableIntBlockPostingsFormat.java | 4 ++--
.../lucene/codecs/memory/TestDirectPostingsFormat.java | 4 ++--
.../lucene/codecs/memory/TestMemoryPostingsFormat.java | 4 ++--
.../lucene/codecs/sep/TestSepPostingsFormat.java | 4 ++--
.../src/java/org/apache/lucene/codecs/FilterCodec.java | 2 +-
.../apache/lucene/codecs/lucene40/Lucene40Codec.java | 3 ++-
.../org/apache/lucene/index/LiveIndexWriterConfig.java | 10 +++++-----
.../lucene/codecs/lucene40/TestReuseDocsEnum.java | 1 +
.../apache/lucene/index/memory/MemoryIndexTest.java | 4 ++--
.../apache/lucene/codecs/asserting/AssertingCodec.java | 6 +++---
.../codecs/asserting/AssertingPostingsFormat.java | 6 +++---
.../lucene/codecs/compressing/CompressingCodec.java | 6 +++---
.../codecs/mockrandom/MockRandomPostingsFormat.java | 3 +++
.../nestedpulsing/NestedPulsingPostingsFormat.java | 10 +++++-----
.../src/java/org/apache/lucene/util/_TestUtil.java | 4 ++--
.../java/org/apache/solr/core/SchemaCodecFactory.java | 4 ++--
20 files changed, 49 insertions(+), 44 deletions(-)
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 055bc21e2fb..141ff99f5fe 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
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.blockterms;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -30,7 +30,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
// TODO: ensure both of these are also in rotation in RandomCodec
public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
private final PostingsFormat postings = new Lucene41WithOrds();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
index 143163feead..6c3034c5fd3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.bloom;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -27,7 +27,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
*/
public class TestBloomPostingsFormat extends BasePostingsFormatTestCase {
private final PostingsFormat postings = new TestBloomFilteredLucene41Postings();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
index 9b25a4c1090..bb3a4824589 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
@@ -23,7 +23,7 @@ import java.util.Collections;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleField;
import org.apache.lucene.document.Field;
@@ -90,10 +90,10 @@ public class TestCompressingStoredFieldsFormat extends LuceneTestCase {
if (random().nextBoolean() && (i % (data.length / 10) == 0)) {
iw.w.close();
// switch codecs
- if (iwConf.getCodec() instanceof Lucene40Codec) {
+ if (iwConf.getCodec() instanceof Lucene41Codec) {
iwConf.setCodec(CompressingCodec.randomInstance(random()));
} else {
- iwConf.setCodec(new Lucene40Codec());
+ iwConf.setCodec(new Lucene41Codec());
}
iw = new RandomIndexWriter(random(), dir, iwConf);
}
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
index e6338a0a674..93a1b548a62 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.intblock;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestFixedIntBlockPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize blocksize
private final PostingsFormat postings = new MockFixedIntBlockPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
index c7955acc9f5..156f91840b0 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.intblock;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestVariableIntBlockPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize blocksize
private final PostingsFormat postings = new MockVariableIntBlockPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
index caf55a83084..bab45bcc366 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -29,7 +29,7 @@ public class TestDirectPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize parameters
private final PostingsFormat postings = new DirectPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
index ca07382b3a9..93892c7b640 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -28,7 +28,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestMemoryPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize doPack
private final PostingsFormat postings = new MemoryPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
index 318822ce19d..8c6df1d93fd 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.sep;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestSepPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize cutoff
private final PostingsFormat postings = new MockSepPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
index 4dfae68232a..ca8e439b6d8 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
@@ -27,7 +27,7 @@ package org.apache.lucene.codecs;
* public final class CustomCodec extends FilterCodec {
*
* public CustomCodec() {
- * super("CustomCodec", new Lucene40Codec());
+ * super("CustomCodec", new Lucene41Codec());
* }
*
* public LiveDocsFormat liveDocsFormat() {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
index b98205ec8cd..076eeeaeadd 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
@@ -41,7 +41,8 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
// NOTE: if we make largish changes in a minor release, easier to just make Lucene42Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
-public class Lucene40Codec extends Codec {
+// nocommit: make readonly and add impersonator
+public final class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
index 7652fa24211..4d76f595494 100755
--- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
@@ -19,7 +19,7 @@ package org.apache.lucene.index;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -186,14 +186,14 @@ public class LiveIndexWriterConfig {
* NOTE: This parameter does not apply to all PostingsFormat implementations,
* including the default one in this release. It only makes sense for term indexes
* that are implemented as a fixed gap between terms. For example,
- * {@link Lucene40PostingsFormat} implements the term index instead based upon how
+ * {@link Lucene41PostingsFormat} implements the term index instead based upon how
* terms share prefixes. To configure its parameters (the minimum and maximum size
- * for a block), you would instead use {@link Lucene40PostingsFormat#Lucene40PostingsFormat(int, int)}.
+ * for a block), you would instead use {@link Lucene41PostingsFormat#Lucene41PostingsFormat(int, int)}.
* which can also be configured on a per-field basis:
*
* //customize Lucene40PostingsFormat, passing minBlockSize=50, maxBlockSize=100
- * final PostingsFormat tweakedPostings = new Lucene40PostingsFormat(50, 100);
- * iwc.setCodec(new Lucene40Codec() {
+ * final PostingsFormat tweakedPostings = new Lucene41PostingsFormat(50, 100);
+ * iwc.setCodec(new Lucene41Codec() {
* @Override
* public PostingsFormat getPostingsFormatForField(String field) {
* if (field.equals("fieldWithTonsOfTerms"))
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
index e5a0ae51ff0..175f7a08aa7 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
@@ -38,6 +38,7 @@ import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
+// nocommit: really this should be in BaseTestPF or somewhere else? useful test!
public class TestReuseDocsEnum extends LuceneTestCase {
public void testReuseDocsEnumNoReuse() throws IOException {
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java b/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
index 68ac8106fd9..77dcedf5036 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
@@ -30,7 +30,7 @@ import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenFilter;
import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.AtomicReader;
@@ -123,7 +123,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
Directory ramdir = new RAMDirectory();
Analyzer analyzer = randomAnalyzer();
IndexWriter writer = new IndexWriter(ramdir,
- new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
+ new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
Document doc = new Document();
Field field1 = newTextField("foo", fooField.toString(), Field.Store.NO);
Field field2 = newTextField("term", termField.toString(), Field.Store.NO);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
index 49a20b09fff..7d775e3613f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
@@ -20,10 +20,10 @@ package org.apache.lucene.codecs.asserting;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
/**
- * Acts like {@link Lucene40Codec} but with additional asserts.
+ * Acts like {@link Lucene41Codec} but with additional asserts.
*/
public final class AssertingCodec extends FilterCodec {
@@ -31,7 +31,7 @@ public final class AssertingCodec extends FilterCodec {
private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
public AssertingCodec() {
- super("Asserting", new Lucene40Codec());
+ super("Asserting", new Lucene41Codec());
}
@Override
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
index 775f972a1eb..94b88117e23 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
@@ -27,7 +27,7 @@ import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -38,10 +38,10 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.OpenBitSet;
/**
- * Just like {@link Lucene40PostingsFormat} but with additional asserts.
+ * Just like {@link Lucene41PostingsFormat} but with additional asserts.
*/
public final class AssertingPostingsFormat extends PostingsFormat {
- private final PostingsFormat in = new Lucene40PostingsFormat();
+ private final PostingsFormat in = new Lucene41PostingsFormat();
public AssertingPostingsFormat() {
super("Asserting");
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
index 91f6055b79f..904fedf0f9b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
@@ -21,14 +21,14 @@ import java.util.Random;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
/**
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
- * fields and delegates to {@link Lucene40Codec} for everything else.
+ * fields and delegates to {@link Lucene41Codec} for everything else.
*/
public class CompressingCodec extends FilterCodec {
@@ -49,7 +49,7 @@ public class CompressingCodec extends FilterCodec {
*/
public CompressingCodec(CompressionMode compressionMode, int chunkSize,
CompressingStoredFieldsIndex storedFieldsIndexFormat) {
- super("Compressing", new Lucene40Codec());
+ super("Compressing", new Lucene41Codec());
this.storedFieldsFormat = new CompressingStoredFieldsFormat(compressionMode, chunkSize, storedFieldsIndexFormat);
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
index c44f05bcc38..9aa48f52091 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
@@ -40,6 +40,7 @@ import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSingleIntFactory;
@@ -174,6 +175,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing Standard postings");
}
+ // nocommit: way to randomize skipInterval and acceptibleOverHead?!
postingsWriter = new Lucene40PostingsWriter(state, skipInterval);
}
@@ -313,6 +315,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Standard postings");
}
+ // nocommit
postingsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
index c44f3ef5db7..31f897e39f0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
@@ -26,8 +26,8 @@ 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.lucene40.Lucene40PostingsReader;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.pulsing.PulsingPostingsReader;
import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
import org.apache.lucene.index.SegmentReadState;
@@ -35,7 +35,7 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
- * Pulsing(1, Pulsing(2, Lucene40))
+ * Pulsing(1, Pulsing(2, Lucene41))
* @lucene.experimental
*/
// TODO: if we create PulsingPostingsBaseFormat then we
@@ -55,7 +55,7 @@ public final class NestedPulsingPostingsFormat extends PostingsFormat {
// Terms dict
boolean success = false;
try {
- docsWriter = new Lucene40PostingsWriter(state);
+ docsWriter = new Lucene41PostingsWriter(state);
pulsingWriterInner = new PulsingPostingsWriter(2, docsWriter);
pulsingWriter = new PulsingPostingsWriter(1, pulsingWriterInner);
@@ -77,7 +77,7 @@ public final class NestedPulsingPostingsFormat extends PostingsFormat {
PostingsReaderBase pulsingReader = null;
boolean success = false;
try {
- docsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ docsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
pulsingReaderInner = new PulsingPostingsReader(docsReader);
pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
FieldsProducer ret = new BlockTreeTermsReader(
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 8e46b10f385..d2760ae6151 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
@@ -44,7 +44,7 @@ import java.util.zip.ZipFile;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.ByteDocValuesField;
import org.apache.lucene.document.DerefBytesDocValuesField;
@@ -651,7 +651,7 @@ public class _TestUtil {
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing postings format to:" + format);
}
- return new Lucene40Codec() {
+ return new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
diff --git a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
index c5dbe80726a..fa32081b973 100644
--- a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
@@ -2,7 +2,7 @@ package org.apache.solr.core;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaAware;
import org.apache.solr.schema.SchemaField;
@@ -42,7 +42,7 @@ public class SchemaCodecFactory extends CodecFactory implements SchemaAware {
@Override
public void inform(final IndexSchema schema) {
- codec = new Lucene40Codec() {
+ codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField fieldOrNull = schema.getFieldOrNull(field);
From fcb7a55d3b1579f9fcfea9bfa36a593420295cb4 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Fri, 12 Oct 2012 15:56:38 +0000
Subject: [PATCH 04/20] svn:eol-style native
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1397628 13f79535-47bb-0310-9956-ffa450edef68
From 28fe93b39b47f7002344168bfeb997f0c4521691 Mon Sep 17 00:00:00 2001
From: Mark Robert Miller
Date: Fri, 12 Oct 2012 18:25:44 +0000
Subject: [PATCH 05/20] SOLR-3939: Consider a sync attempt from leader to
replica that fails due to 404 a success. SOLR-3940: Rejoining the leader
election incorrectly triggers the code path for a fresh cluster start rather
than fail over.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1397665 13f79535-47bb-0310-9956-ffa450edef68
---
solr/CHANGES.txt | 10 ++++--
.../apache/solr/cloud/ElectionContext.java | 2 +-
.../org/apache/solr/cloud/LeaderElector.java | 7 ++--
.../org/apache/solr/cloud/ZkController.java | 6 ++--
.../java/org/apache/solr/update/PeerSync.java | 5 +++
.../solr/cloud/BasicDistributedZkTest.java | 36 +++++++++++++++----
.../apache/solr/cloud/LeaderElectionTest.java | 9 +++--
.../org/apache/solr/cloud/OverseerTest.java | 4 +--
8 files changed, 54 insertions(+), 25 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index cb4852e907f..62a0fadde7b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -42,9 +42,7 @@ New Features
values of a multiValued field in their original order when highlighting.
(Joel Bernstein via yonik)
-* SOLR-3929
-support configuring IndexWriter max thread count in solrconfig
-
+* SOLR-3929: Support configuring IndexWriter max thread count in solrconfig.
(phunt via Mark Miller)
Optimizations
@@ -73,6 +71,12 @@ Bug Fixes
* SOLR-3917: Partial State on Schema-Browser UI is not defined for Dynamic
Fields & Types (steffkes)
+
+* SOLR-3939: Consider a sync attempt from leader to replica that fails due
+ to 404 a success. (Mark Miller, Joel Bernstein)
+
+* SOLR-3940: Rejoining the leader election incorrectly triggers the code path
+ for a fresh cluster start rather than fail over. (Mark Miller)
Other Changes
----------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 7baa465ea68..77417e9ee06 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -324,7 +324,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
SolrException.log(log, "Error trying to start recovery", t);
}
- leaderElector.joinElection(this);
+ leaderElector.joinElection(this, true);
}
private boolean shouldIBeLeader(ZkNodeProps leaderProps, SolrCore core) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 4d3a016091b..07caa55734b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -18,7 +18,6 @@ package org.apache.solr.cloud;
*/
import java.io.IOException;
-import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@@ -43,7 +42,7 @@ import org.slf4j.LoggerFactory;
* Leader Election process. This class contains the logic by which a
* leader is chosen. First call * {@link #setup(ElectionContext)} to ensure
* the election process is init'd. Next call
- * {@link #joinElection(ElectionContext)} to start the leader election.
+ * {@link #joinElection(ElectionContext, boolean)} to start the leader election.
*
* The implementation follows the classic ZooKeeper recipe of creating an
* ephemeral, sequential node for each candidate and then looking at the set
@@ -203,7 +202,7 @@ public class LeaderElector {
*
* @return sequential node number
*/
- public int joinElection(ElectionContext context) throws KeeperException, InterruptedException, IOException {
+ public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
long sessionId = zkClient.getSolrZooKeeper().getSessionId();
@@ -259,7 +258,7 @@ public class LeaderElector {
}
}
int seq = getSeq(leaderSeqPath);
- checkIfIamLeader(seq, context, false);
+ checkIfIamLeader(seq, context, replacement);
return seq;
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 0ccab0e6b72..db869ff4651 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -191,7 +191,7 @@ public final class ZkController {
ZkController.this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
- overseerElector.joinElection(context);
+ overseerElector.joinElection(context, true);
zkStateReader.createClusterStateWatchersAndUpdate();
// cc.newCmdDistribExecutor();
@@ -422,7 +422,7 @@ public final class ZkController {
this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
overseerElector.setup(context);
- overseerElector.joinElection(context);
+ overseerElector.joinElection(context, false);
zkStateReader.createClusterStateWatchersAndUpdate();
} catch (IOException e) {
@@ -730,7 +730,7 @@ public final class ZkController {
leaderElector.setup(context);
electionContexts.put(coreZkNodeName, context);
- leaderElector.joinElection(context);
+ leaderElector.joinElection(context, false);
}
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index a98917e53ad..0466864595d 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -312,6 +312,11 @@ public class PeerSync {
log.warn(msg() + " got a 503 from " + srsp.getShardAddress() + ", counting as success");
return true;
}
+
+ if (cantReachIsSuccess && sreq.purpose == 1 && srsp.getException() instanceof SolrException && ((SolrException) srsp.getException()).code() == 404) {
+ log.warn(msg() + " got a 404 from " + srsp.getShardAddress() + ", counting as success");
+ return true;
+ }
// TODO: at least log???
// srsp.getException().printStackTrace(System.out);
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
index dcf3963f442..ab55358fe61 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
@@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
@@ -742,10 +743,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
0,
((HttpSolrServer) client).getBaseURL().length()
- DEFAULT_COLLECTION.length() - 1);
- createCollection(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
- createCollection(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
- createCollection(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
- createCollection(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
+ createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
+ createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
+ createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
+ createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
while (pending != null && pending.size() > 0) {
@@ -764,7 +765,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
assertAllActive(oneInstanceCollection2, solrj.getZkStateReader());
- printLayout();
+ //printLayout();
// TODO: enable when we don't falsely get slice1...
// solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
@@ -803,6 +804,27 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
assertNotNull(slices);
String roles = slices.get("slice1").getReplicasMap().values().iterator().next().getStr(ZkStateReader.ROLES_PROP);
assertEquals("none", roles);
+
+
+ ZkCoreNodeProps props = new ZkCoreNodeProps(solrj.getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "slice1"));
+
+ // now test that unloading a core gets us a new leader
+ HttpSolrServer server = new HttpSolrServer(baseUrl);
+ Unload unloadCmd = new Unload(true);
+ unloadCmd.setCoreName(props.getCoreName());
+
+ String leader = props.getCoreUrl();
+
+ server.request(unloadCmd);
+
+ int tries = 50;
+ while (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "slice1", 10000))) {
+ Thread.sleep(100);
+ if (tries-- == 0) {
+ fail("Leader never changed");
+ }
+ }
+
}
private void testSearchByCollectionName() throws SolrServerException {
@@ -875,10 +897,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
private void createCollection(String collection,
List collectionClients, String baseUrl, int num) {
- createCollection(collection, collectionClients, baseUrl, num, null);
+ createSolrCore(collection, collectionClients, baseUrl, num, null);
}
- private void createCollection(final String collection,
+ private void createSolrCore(final String collection,
List collectionClients, final String baseUrl, final int num,
final String shardId) {
Callable call = new Callable() {
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
index 1b12c659586..a4f69fcbfce 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
@@ -40,7 +40,6 @@ import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-import org.junit.Ignore;
import org.junit.Test;
@Slow
@@ -114,7 +113,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
elector, "shard1", "collection1", Integer.toString(nodeNumber),
props, zkStateReader);
elector.setup(context);
- seq = elector.joinElection(context);
+ seq = elector.joinElection(context, false);
electionDone = true;
seqToThread.put(seq, this);
}
@@ -175,7 +174,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext context = new ShardLeaderElectionContextBase(elector,
"shard2", "collection1", "dummynode1", props, zkStateReader);
elector.setup(context);
- elector.joinElection(context);
+ elector.joinElection(context, false);
assertEquals("http://127.0.0.1/solr/",
getLeaderUrl("collection1", "shard2"));
}
@@ -188,7 +187,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
"slice1", "collection2", "dummynode1", props, zkStateReader);
first.setup(firstContext);
- first.joinElection(firstContext);
+ first.joinElection(firstContext, false);
Thread.sleep(1000);
assertEquals("original leader was not registered", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
@@ -199,7 +198,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext context = new ShardLeaderElectionContextBase(second,
"slice1", "collection2", "dummynode1", props, zkStateReader);
second.setup(context);
- second.joinElection(context);
+ second.joinElection(context, false);
Thread.sleep(1000);
assertEquals("original leader should have stayed leader", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
firstContext.cancelElection();
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 6520c6bd42b..59071c7d951 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -139,7 +139,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
elector, shardId, collection, nodeName + "_" + coreName, props,
zkStateReader);
- elector.joinElection(ctx);
+ elector.joinElection(ctx, false);
return shardId;
}
Thread.sleep(500);
@@ -876,7 +876,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
new HttpShardHandlerFactory().getShardHandler(), "/admin/cores", reader);
ElectionContext ec = new OverseerElectionContext(zkClient, overseer, address.replaceAll("/", "_"));
overseerElector.setup(ec);
- overseerElector.joinElection(ec);
+ overseerElector.joinElection(ec, false);
return zkClient;
}
From 25afdaf65b81fec19daa255c0a8bbe94466a9598 Mon Sep 17 00:00:00 2001
From: Mark Robert Miller
Date: Fri, 12 Oct 2012 19:31:13 +0000
Subject: [PATCH 06/20] SOLR-3941: The "commitOnLeader" part of distributed
recovery can use openSearcher=false.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1397698 13f79535-47bb-0310-9956-ffa450edef68
---
solr/CHANGES.txt | 3 +++
solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java | 2 ++
2 files changed, 5 insertions(+)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 62a0fadde7b..779d9d71562 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -60,6 +60,9 @@ Optimizations
* SOLR-3734: Improve Schema-Browser Handling for CopyField using
dynamicField's (steffkes)
+
+* SOLR-3941: The "commitOnLeader" part of distributed recovery can use
+ openSearcher=false. (Tomas Fernandez Lobbe via Mark Miller)
Bug Fixes
----------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index fab5c2a4a60..35fb620746c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.RequestHandlers.LazyRequestHandlerWrapper;
@@ -177,6 +178,7 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+ ureq.getParams().set(UpdateParams.OPEN_SEARCHER, false);
ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true).process(
server);
server.shutdown();
From 2ea2adcf6b3633aa895b841eedc09ab540dc9b4d Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Sat, 13 Oct 2012 15:27:24 +0000
Subject: [PATCH 07/20] LUCENE-4446: wrap up cutover to blockpostingsformat
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397858 13f79535-47bb-0310-9956-ffa450edef68
---
.../codecs/memory/DirectPostingsFormat.java | 8 +--
.../pulsing/Pulsing41PostingsFormat.java | 6 +--
.../java/org/apache/lucene/codecs/Codec.java | 2 +-
.../org/apache/lucene/codecs/FilterCodec.java | 2 +-
.../lucene/codecs/lucene40/Lucene40Codec.java | 4 +-
.../lucene40/Lucene40PostingsBaseFormat.java | 5 +-
.../lucene40/Lucene40PostingsFormat.java | 30 ++++-------
.../lucene40/Lucene40PostingsReader.java | 22 +++++---
.../lucene40/Lucene40SkipListReader.java | 3 +-
.../lucene41/Lucene41PostingsFormat.java | 6 +++
.../lucene41/Lucene41PostingsWriter.java | 5 +-
.../lucene/codecs/lucene41/package.html | 2 +-
.../org/apache/lucene/codecs/package.html | 4 +-
.../lucene/index/LiveIndexWriterConfig.java | 2 +-
.../codecs/lucene40/TestReuseDocsEnum.java | 8 +--
.../lucene40/Lucene40PostingsWriter.java | 16 ++----
.../lucene40/Lucene40RWPostingsFormat.java | 50 +++++++++++++++++++
.../lucene40/Lucene40SkipListWriter.java | 3 +-
.../lucene/codecs/lucene40/package.html | 25 ++++++++++
.../mockrandom/MockRandomPostingsFormat.java | 10 ++--
.../org/apache/lucene/index/RandomCodec.java | 2 +-
.../util/TestRuleSetupAndRestoreClassEnv.java | 4 +-
.../org.apache.lucene.codecs.PostingsFormat | 2 +-
.../solr/collection1/conf/schema_codec.xml | 4 +-
.../apache/solr/core/TestCodecSupport.java | 14 +++---
25 files changed, 154 insertions(+), 85 deletions(-)
rename lucene/{core => test-framework}/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (95%)
create mode 100644 lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java
rename lucene/{core => test-framework}/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListWriter.java (99%)
create mode 100644 lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/package.html
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 97e22f4a03d..936d4ed5d61 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
@@ -27,7 +27,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.lucene40.Lucene40PostingsFormat; // javadocs
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -52,7 +52,7 @@ import org.apache.lucene.util.automaton.Transition;
// - build depth-N prefix hash?
// - or: longer dense skip lists than just next byte?
-/** Wraps {@link Lucene40PostingsFormat} format for on-disk
+/** Wraps {@link Lucene41PostingsFormat} format for on-disk
* storage, but then at read time loads and stores all
* terms & postings directly in RAM as byte[], int[].
*
@@ -100,12 +100,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- return PostingsFormat.forName("Lucene40").fieldsConsumer(state);
+ return PostingsFormat.forName("Lucene41").fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- FieldsProducer postings = PostingsFormat.forName("Lucene40").fieldsProducer(state);
+ FieldsProducer postings = PostingsFormat.forName("Lucene41").fieldsProducer(state);
if (state.context.context != IOContext.Context.MERGE) {
FieldsProducer loadedPostings;
try {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
index 7fd7fb0504a..9946062e09b 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
@@ -28,17 +28,17 @@ import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
*/
public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
- /** Inlines docFreq=1 terms, otherwise uses the normal "Lucene40" format. */
+ /** Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format. */
public Pulsing41PostingsFormat() {
this(1);
}
- /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
+ /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene41" format. */
public Pulsing41PostingsFormat(int freqCutoff) {
this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
- /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
+ /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene41" format. */
public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 1892df6d300..7a473a3ed38 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
loader.reload(classloader);
}
- private static Codec defaultCodec = Codec.forName("Lucene40");
+ private static Codec defaultCodec = Codec.forName("Lucene41");
/** expert: returns the default codec used for newly created
* {@link IndexWriterConfig}s.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
index ca8e439b6d8..12f17197d2b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
@@ -21,7 +21,7 @@ package org.apache.lucene.codecs;
* A codec that forwards all its method calls to another codec.
*
* Extend this class when you need to reuse the functionality of an existing
- * codec. For example, if you want to build a codec that redefines Lucene40's
+ * codec. For example, if you want to build a codec that redefines Lucene41's
* {@link LiveDocsFormat}:
*
* public final class CustomCodec extends FilterCodec {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
index 076eeeaeadd..a0d66af61d0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
@@ -36,12 +36,12 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene40 package documentation for file format details.
- * @lucene.experimental
+ * @deprecated Only for reading old 4.0 segments
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene42Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
-// nocommit: make readonly and add impersonator
+@Deprecated
public final class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
index df6611922e2..eaf452d6252 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
@@ -29,9 +29,10 @@ import org.apache.lucene.index.SegmentWriteState;
* Provides a {@link PostingsReaderBase} and {@link
* PostingsWriterBase}.
*
- * @lucene.experimental */
+ * @deprecated Only for reading old 4.0 segments */
// TODO: should these also be named / looked up via SPI?
+@Deprecated
public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
/** Sole constructor. */
@@ -46,6 +47,6 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
@Override
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
- return new Lucene40PostingsWriter(state);
+ throw new UnsupportedOperationException("this codec can only be used for reading");
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
index 16d9c47ed97..1f9c28efdb4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
@@ -211,15 +211,18 @@ import org.apache.lucene.util.fst.FST; // javadocs
* previous occurrence and an OffsetLength follows. Offset data is only written for
* {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.
*
- * @lucene.experimental */
+ * @deprecated Only for reading old 4.0 segments */
// TODO: this class could be created by wrapping
// BlockTreeTermsDict around Lucene40PostingsBaseFormat; ie
// we should not duplicate the code from that class here:
-public final class Lucene40PostingsFormat extends PostingsFormat {
+@Deprecated
+public class Lucene40PostingsFormat extends PostingsFormat {
- private final int minBlockSize;
- private final int maxBlockSize;
+ /** minimum items (terms or sub-blocks) per block for BlockTree */
+ protected final int minBlockSize;
+ /** maximum items (terms or sub-blocks) per block for BlockTree */
+ protected final int maxBlockSize;
/** Creates {@code Lucene40PostingsFormat} with default
* settings. */
@@ -231,7 +234,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
* values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
- public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
+ private Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
super("Lucene40");
this.minBlockSize = minBlockSize;
assert minBlockSize > 1;
@@ -240,22 +243,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase docs = new Lucene40PostingsWriter(state);
-
- // TODO: should we make the terms index more easily
- // pluggable? Ie so that this codec would record which
- // index impl was used, and switch on loading?
- // Or... you must make a new Codec for this?
- boolean success = false;
- try {
- FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
- success = true;
- return ret;
- } finally {
- if (!success) {
- docs.close();
- }
- }
+ throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
index 64d2e49b1ff..a3729e2f1da 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
@@ -45,10 +45,21 @@ import org.apache.lucene.util.IOUtils;
* postings format.
*
* @see Lucene40PostingsFormat
- * @lucene.experimental */
-
+ * @deprecated Only for reading old 4.0 segments */
+@Deprecated
public class Lucene40PostingsReader extends PostingsReaderBase {
+ final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
+ final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
+ final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
+
+ //private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+ // Increment version to change it:
+ final static int VERSION_START = 0;
+ final static int VERSION_LONG_SKIP = 1;
+ final static int VERSION_CURRENT = VERSION_LONG_SKIP;
+
private final IndexInput freqIn;
private final IndexInput proxIn;
// public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
@@ -67,7 +78,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
try {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
ioContext);
- CodecUtil.checkHeader(freqIn, Lucene40PostingsWriter.FRQ_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
+ CodecUtil.checkHeader(freqIn, FRQ_CODEC, VERSION_START, VERSION_CURRENT);
// TODO: hasProx should (somehow!) become codec private,
// but it's tricky because 1) FIS.hasProx is global (it
// could be all fields that have prox are written by a
@@ -79,7 +90,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
if (fieldInfos.hasProx()) {
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
ioContext);
- CodecUtil.checkHeader(proxIn, Lucene40PostingsWriter.PRX_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
+ CodecUtil.checkHeader(proxIn, PRX_CODEC, VERSION_START, VERSION_CURRENT);
} else {
proxIn = null;
}
@@ -97,8 +108,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching past writer
- CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.TERMS_CODEC,
- Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_CURRENT);
+ CodecUtil.checkHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT);
skipInterval = termsIn.readInt();
maxSkipLevels = termsIn.readInt();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
index 4cef37a5977..1580a390ba9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
@@ -28,8 +28,9 @@ import org.apache.lucene.store.IndexInput;
* that stores positions and payloads.
*
* @see Lucene40PostingsFormat
- * @lucene.experimental
+ * @deprecated Only for reading old 4.0 segments
*/
+@Deprecated
public class Lucene40SkipListReader extends MultiLevelSkipListReader {
private boolean currentFieldStoresPayloads;
private boolean currentFieldStoresOffsets;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
index d1c21ed2846..3cbc9653bed 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
@@ -380,10 +380,16 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
// 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() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
+ /** Creates {@code Lucene41PostingsFormat} with custom
+ * values for {@code minBlockSize} and {@code
+ * maxBlockSize} passed to block terms dictionary.
+ * @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene41");
this.minTermBlockSize = minTermBlockSize;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
index 51e2b02422f..4298ea6c13b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
@@ -63,10 +63,9 @@ public final class Lucene41PostingsWriter extends PostingsWriterBase {
final static String POS_CODEC = "Lucene41PostingsWriterPos";
final static String PAY_CODEC = "Lucene41PostingsWriterPay";
- // Increment version to change it: nocommit: we can start at 0
+ // Increment version to change it
final static int VERSION_START = 0;
- final static int VERSION_NO_OFFSETS_IN_SKIPDATA = 1; // LUCENE-4443
- final static int VERSION_CURRENT = VERSION_NO_OFFSETS_IN_SKIPDATA;
+ final static int VERSION_CURRENT = VERSION_START;
final IndexOutput docOut;
final IndexOutput posOut;
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
index aff3d7a572f..14782803a1c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
@@ -153,7 +153,7 @@ its title, url, or an identifier to access a database. The set of stored fields
returned for each hit when searching. This is keyed by document number.
-{@link org.apache.lucene.codecs.lucene41Lucene41PostingsFormat Term dictionary}.
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term dictionary}.
A dictionary containing all of the terms used in all of the
indexed fields of all of the documents. The dictionary also contains the number
of documents which contain the term, and pointers to the term's frequency and
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/package.html b/lucene/core/src/java/org/apache/lucene/codecs/package.html
index e6de64d057b..91a65458ac1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/package.html
+++ b/lucene/core/src/java/org/apache/lucene/codecs/package.html
@@ -61,8 +61,8 @@ name of your codec.
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
formats for different fields, then you can register your custom postings format in the same way (in
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
- {@link org.apache.lucene.codecs.lucene40.Lucene40Codec} and override
- {@link org.apache.lucene.codecs.lucene40.Lucene40Codec#getPostingsFormatForField(String)} to return your custom
+ {@link org.apache.lucene.codecs.lucene41.Lucene41Codec} and override
+ {@link org.apache.lucene.codecs.lucene41.Lucene41Codec#getPostingsFormatForField(String)} to return your custom
postings format.