();
+ while (randomTerms.size() < numTerms) {
+ randomTerms.add(_TestUtil.randomSimpleString(random()));
+ }
+
+ // create first index
+ for (int i = 0; i < numDocs; i++) {
+ Document doc = new Document();
+ doc.add(new StringField("id", RandomPicks.randomFrom(random(), randomTerms), Store.NO));
+ doc.add(new NumericDocValuesField("ndv", 4L));
+ doc.add(new NumericDocValuesField("control", 8L));
+ writer.addDocument(doc);
+ }
+
+ if (random().nextBoolean()) {
+ writer.commit();
+ }
+
+ // update some docs to a random value
+ long value = random().nextInt();
+ Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
+ writer.updateNumericDocValue(term, "ndv", value);
+ writer.updateNumericDocValue(term, "control", value * 2);
+ writer.close();
+
+ Directory dir2 = newDirectory();
+ conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ writer = new IndexWriter(dir2, conf);
+ if (random().nextBoolean()) {
+ writer.addIndexes(dir1);
+ } else {
+ DirectoryReader reader = DirectoryReader.open(dir1);
+ writer.addIndexes(reader);
+ reader.close();
+ }
+ writer.close();
+
+ DirectoryReader reader = DirectoryReader.open(dir2);
+ for (AtomicReaderContext context : reader.leaves()) {
+ AtomicReader r = context.reader();
+ NumericDocValues ndv = r.getNumericDocValues("ndv");
+ NumericDocValues control = r.getNumericDocValues("control");
+ for (int i = 0; i < r.maxDoc(); i++) {
+ assertEquals(ndv.get(i)*2, control.get(i));
+ }
+ }
+ reader.close();
+
+ IOUtils.close(dir1, dir2);
+ }
+
}
Index: lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (revision 1525565)
+++ lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (working copy)
@@ -127,7 +127,7 @@
seg = writer.newestSegment();
writer.close();
- fieldInfos = _TestUtil.getFieldInfos(seg.info);
+ fieldInfos = _TestUtil.getFieldInfos(seg);
}
@Override
Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java
===================================================================
--- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java (revision 1525565)
+++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java (working copy)
@@ -1,79 +0,0 @@
-package org.apache.lucene.facet.codecs.facet45;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
-import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
-import org.apache.lucene.facet.params.CategoryListParams;
-import org.apache.lucene.facet.params.FacetIndexingParams;
-
-/**
- * Same as {@link Lucene45Codec} except it uses {@link Facet42DocValuesFormat}
- * for facet fields (faster-but-more-RAM-consuming doc values).
- *
- *
- * NOTE: this codec does not support facet partitions (see
- * {@link FacetIndexingParams#getPartitionSize()}).
- *
- *
- * NOTE: this format cannot handle more than 2 GB
- * of facet data in a single segment. If your usage may hit
- * this limit, you can either use Lucene's default
- * DocValuesFormat, limit the maximum segment size in your
- * MergePolicy, or send us a patch fixing the limitation.
- *
- * @lucene.experimental
- */
-public class Facet45Codec extends Lucene45Codec {
-
- private final Set facetFields;
- private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42");
-
- /** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */
- public Facet45Codec() {
- this(FacetIndexingParams.DEFAULT);
- }
-
- /**
- * Initializes with the given {@link FacetIndexingParams}. Returns the proper
- * {@link DocValuesFormat} for the fields that are returned by
- * {@link FacetIndexingParams#getAllCategoryListParams()}.
- */
- public Facet45Codec(FacetIndexingParams fip) {
- if (fip.getPartitionSize() != Integer.MAX_VALUE) {
- throw new IllegalArgumentException("this Codec does not support partitions");
- }
- this.facetFields = new HashSet();
- for (CategoryListParams clp : fip.getAllCategoryListParams()) {
- facetFields.add(clp.field);
- }
- }
-
- @Override
- public DocValuesFormat getDocValuesFormatForField(String field) {
- if (facetFields.contains(field)) {
- return facetsDVFormat;
- } else {
- return super.getDocValuesFormatForField(field);
- }
- }
-}
Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html
===================================================================
--- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html (revision 1525565)
+++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html (working copy)
@@ -1,22 +0,0 @@
-
-
-
-
-Codec + DocValuesFormat that are optimized for facets.
-
-
Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java
===================================================================
--- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java (revision 0)
+++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java (working copy)
@@ -0,0 +1,79 @@
+package org.apache.lucene.facet.codecs.facet46;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
+import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
+import org.apache.lucene.facet.params.CategoryListParams;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+
+/**
+ * Same as {@link Lucene46Codec} except it uses {@link Facet42DocValuesFormat}
+ * for facet fields (faster-but-more-RAM-consuming doc values).
+ *
+ *
+ * NOTE: this codec does not support facet partitions (see
+ * {@link FacetIndexingParams#getPartitionSize()}).
+ *
+ *
+ * NOTE: this format cannot handle more than 2 GB
+ * of facet data in a single segment. If your usage may hit
+ * this limit, you can either use Lucene's default
+ * DocValuesFormat, limit the maximum segment size in your
+ * MergePolicy, or send us a patch fixing the limitation.
+ *
+ * @lucene.experimental
+ */
+public class Facet46Codec extends Lucene46Codec {
+
+ private final Set facetFields;
+ private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42");
+
+ /** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */
+ public Facet46Codec() {
+ this(FacetIndexingParams.DEFAULT);
+ }
+
+ /**
+ * Initializes with the given {@link FacetIndexingParams}. Returns the proper
+ * {@link DocValuesFormat} for the fields that are returned by
+ * {@link FacetIndexingParams#getAllCategoryListParams()}.
+ */
+ public Facet46Codec(FacetIndexingParams fip) {
+ if (fip.getPartitionSize() != Integer.MAX_VALUE) {
+ throw new IllegalArgumentException("this Codec does not support partitions");
+ }
+ this.facetFields = new HashSet();
+ for (CategoryListParams clp : fip.getAllCategoryListParams()) {
+ facetFields.add(clp.field);
+ }
+ }
+
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ if (facetFields.contains(field)) {
+ return facetsDVFormat;
+ } else {
+ return super.getDocValuesFormatForField(field);
+ }
+ }
+}
Property changes on: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/Facet46Codec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Added: svn:executable
## -0,0 +1 ##
+*
\ No newline at end of property
Index: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html
===================================================================
--- lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html (revision 0)
+++ lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html (working copy)
@@ -0,0 +1,22 @@
+
+
+
+
+Codec + DocValuesFormat that are optimized for facets.
+
+
Property changes on: lucene/facet/src/java/org/apache/lucene/facet/codecs/facet46/package.html
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Added: svn:executable
## -0,0 +1 ##
+*
\ No newline at end of property
Index: lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
===================================================================
--- lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (revision 1525565)
+++ lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (working copy)
@@ -3,7 +3,7 @@
import java.util.Random;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
+import org.apache.lucene.facet.codecs.facet46.Facet46Codec;
import org.apache.lucene.facet.encoding.DGapIntEncoder;
import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
import org.apache.lucene.facet.encoding.EightFlagsIntEncoder;
@@ -53,7 +53,7 @@
public static void beforeClassFacetTestCase() throws Exception {
if (random().nextDouble() < 0.3) {
savedDefault = Codec.getDefault(); // save to restore later
- Codec.setDefault(new Facet45Codec());
+ Codec.setDefault(new Facet46Codec());
}
}
Index: lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
===================================================================
--- lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (revision 1525565)
+++ lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (working copy)
@@ -31,7 +31,7 @@
import org.apache.lucene.document.Field;
import org.apache.lucene.facet.FacetTestCase;
import org.apache.lucene.facet.FacetTestUtils;
-import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
+import org.apache.lucene.facet.codecs.facet46.Facet46Codec;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.params.CategoryListParams;
import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -260,7 +260,7 @@
Directory dir = newDirectory();
Directory taxoDir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- iwc.setCodec(new Facet45Codec());
+ iwc.setCodec(new Facet46Codec());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
Index: lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
===================================================================
--- lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (revision 1525565)
+++ lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (working copy)
@@ -141,7 +141,7 @@
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
info.getUseCompoundFile(),
info.getCodec(), info.getDiagnostics(), info.attributes());
- destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getDocValuesGen()));
+ destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen()));
// now copy files over
Collection files = infoPerCommit.files();
for (final String srcName : files) {
Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
===================================================================
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (revision 1525565)
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (working copy)
@@ -34,7 +34,7 @@
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -164,7 +164,7 @@
* codec to use. */
protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer) {
IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
- iwc.setCodec(new Lucene45Codec());
+ iwc.setCodec(new Lucene46Codec());
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
return iwc;
}
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (working copy)
@@ -23,10 +23,10 @@
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
/**
- * Acts like {@link Lucene45Codec} but with additional asserts.
+ * Acts like {@link Lucene46Codec} but with additional asserts.
*/
public final class AssertingCodec extends FilterCodec {
@@ -37,7 +37,7 @@
private final NormsFormat norms = new AssertingNormsFormat();
public AssertingCodec() {
- super("Asserting", new Lucene45Codec());
+ super("Asserting", new Lucene46Codec());
}
@Override
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java (working copy)
@@ -28,7 +28,7 @@
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
// TODO: better name :)
@@ -45,9 +45,10 @@
private final NormsFormat norms = new DiskNormsFormat();
public CheapBastardCodec() {
- super("CheapBastard", new Lucene45Codec());
+ super("CheapBastard", new Lucene46Codec());
}
+ @Override
public PostingsFormat postingsFormat() {
return postings;
}
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (working copy)
@@ -23,13 +23,13 @@
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
/**
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
- * fields and delegates to {@link Lucene45Codec} for everything else.
+ * fields and delegates to {@link Lucene46Codec} for everything else.
*/
public abstract class CompressingCodec extends FilterCodec {
@@ -73,7 +73,7 @@
* Creates a compressing codec with a given segment suffix
*/
public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize) {
- super(name, new Lucene45Codec());
+ super(name, new Lucene46Codec());
this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize);
this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize);
}
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java (working copy)
@@ -37,6 +37,7 @@
* @see Lucene40FieldInfosFormat
* @lucene.experimental
*/
+@Deprecated
public class Lucene40FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
@@ -44,7 +45,7 @@
}
@Override
- public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
+ public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
boolean success = false;
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (revision 0)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (working copy)
@@ -0,0 +1,109 @@
+package org.apache.lucene.codecs.lucene42;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Lucene 4.2 FieldInfos writer.
+ *
+ * @see Lucene42FieldInfosFormat
+ * @lucene.experimental
+ */
+@Deprecated
+public final class Lucene42FieldInfosWriter extends FieldInfosWriter {
+
+ /** Sole constructor. */
+ public Lucene42FieldInfosWriter() {
+ }
+
+ @Override
+ public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+ final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
+ IndexOutput output = directory.createOutput(fileName, context);
+ boolean success = false;
+ try {
+ CodecUtil.writeHeader(output, Lucene42FieldInfosFormat.CODEC_NAME, Lucene42FieldInfosFormat.FORMAT_CURRENT);
+ output.writeVInt(infos.size());
+ for (FieldInfo fi : infos) {
+ IndexOptions indexOptions = fi.getIndexOptions();
+ byte bits = 0x0;
+ if (fi.hasVectors()) bits |= Lucene42FieldInfosFormat.STORE_TERMVECTOR;
+ if (fi.omitsNorms()) bits |= Lucene42FieldInfosFormat.OMIT_NORMS;
+ if (fi.hasPayloads()) bits |= Lucene42FieldInfosFormat.STORE_PAYLOADS;
+ if (fi.isIndexed()) {
+ bits |= Lucene42FieldInfosFormat.IS_INDEXED;
+ assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
+ if (indexOptions == IndexOptions.DOCS_ONLY) {
+ bits |= Lucene42FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
+ } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
+ bits |= Lucene42FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
+ } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
+ bits |= Lucene42FieldInfosFormat.OMIT_POSITIONS;
+ }
+ }
+ output.writeString(fi.name);
+ output.writeVInt(fi.number);
+ output.writeByte(bits);
+
+ // pack the DV types in one byte
+ final byte dv = docValuesByte(fi.getDocValuesType());
+ final byte nrm = docValuesByte(fi.getNormType());
+ assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
+ byte val = (byte) (0xff & ((nrm << 4) | dv));
+ output.writeByte(val);
+ output.writeStringStringMap(fi.attributes());
+ }
+ success = true;
+ } finally {
+ if (success) {
+ output.close();
+ } else {
+ IOUtils.closeWhileHandlingException(output);
+ }
+ }
+ }
+
+ private static byte docValuesByte(DocValuesType type) {
+ if (type == null) {
+ return 0;
+ } else if (type == DocValuesType.NUMERIC) {
+ return 1;
+ } else if (type == DocValuesType.BINARY) {
+ return 2;
+ } else if (type == DocValuesType.SORTED) {
+ return 3;
+ } else if (type == DocValuesType.SORTED_SET) {
+ return 4;
+ } else {
+ throw new AssertionError();
+ }
+ }
+}
Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (working copy)
@@ -17,12 +17,18 @@
* limitations under the License.
*/
+import java.io.IOException;
+
import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.util.LuceneTestCase;
/**
* Read-write version of {@link Lucene42Codec} for testing.
*/
+@SuppressWarnings("deprecation")
public class Lucene42RWCodec extends Lucene42Codec {
private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
private static final NormsFormat norms = new Lucene42NormsFormat();
@@ -36,4 +42,19 @@
public NormsFormat normsFormat() {
return norms;
}
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return new Lucene42FieldInfosFormat() {
+ @Override
+ public FieldInfosWriter getFieldInfosWriter() throws IOException {
+ if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
+ return super.getFieldInfosWriter();
+ } else {
+ return new Lucene42FieldInfosWriter();
+ }
+ }
+ };
+ }
+
}
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (revision 0)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (working copy)
@@ -0,0 +1,48 @@
+package org.apache.lucene.codecs.lucene45;
+
+/*
+ * 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.FieldInfosFormat;
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Read-write version of {@link Lucene45Codec} for testing.
+ */
+@SuppressWarnings("deprecation")
+public class Lucene45RWCodec extends Lucene45Codec {
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return new Lucene42FieldInfosFormat() {
+ @Override
+ public FieldInfosWriter getFieldInfosWriter() throws IOException {
+ if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
+ return super.getFieldInfosWriter();
+ } else {
+ return new Lucene42FieldInfosWriter();
+ }
+ }
+ };
+ }
+
+}
Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html (revision 0)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html (working copy)
@@ -0,0 +1,25 @@
+
+
+
+
+
+
+
+Support for testing {@link org.apache.lucene.codecs.lucene45.Lucene45Codec}.
+
+
\ No newline at end of file
Property changes on: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene45/package.html
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (working copy)
@@ -40,7 +40,7 @@
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -1386,7 +1386,7 @@
// TODO: would be better to use / delegate to the current
// Codec returned by getCodec()
- iwc.setCodec(new Lucene45Codec() {
+ iwc.setCodec(new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
Index: lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (working copy)
@@ -31,13 +31,14 @@
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleField;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.document.FloatField;
import org.apache.lucene.document.IntField;
import org.apache.lucene.document.LongField;
@@ -44,8 +45,6 @@
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.NumericRangeQuery;
@@ -59,7 +58,6 @@
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@@ -78,6 +76,7 @@
*/
protected abstract Codec getCodec();
+ @Override
public void setUp() throws Exception {
super.setUp();
// set the default codec, so adding test cases to this isn't fragile
@@ -85,6 +84,7 @@
Codec.setDefault(getCodec());
}
+ @Override
public void tearDown() throws Exception {
Codec.setDefault(savedCodec); // restore
super.tearDown();
@@ -502,7 +502,7 @@
// get another codec, other than the default: so we are merging segments across different codecs
final Codec otherCodec;
if ("SimpleText".equals(Codec.getDefault().getName())) {
- otherCodec = new Lucene45Codec();
+ otherCodec = new Lucene46Codec();
} else {
otherCodec = new SimpleTextCodec();
}
Index: lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (working copy)
@@ -27,19 +27,23 @@
import java.util.Random;
import java.util.Set;
+import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
+import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
-import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
-import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@@ -48,12 +52,8 @@
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
+import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
-import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
-import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
-import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
-import org.apache.lucene.codecs.memory.FSTPostingsFormat;
-import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@@ -66,7 +66,7 @@
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
-public class RandomCodec extends Lucene45Codec {
+public class RandomCodec extends Lucene46Codec {
/** Shuffled list of postings formats to use for new mappings */
private List formats = new ArrayList();
Index: lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (working copy)
@@ -1386,7 +1386,7 @@
/** Returns true if the codec "supports" field updates. */
public static boolean defaultCodecSupportsFieldUpdates() {
String name = Codec.getDefault().getName();
- if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42")) {
+ if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42") || name.equals("Lucene45")) {
return false;
}
return true;
Index: lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (working copy)
@@ -17,14 +17,22 @@
* limitations under the License.
*/
+import static org.apache.lucene.util.LuceneTestCase.INFOSTREAM;
+import static org.apache.lucene.util.LuceneTestCase.TEST_CODEC;
+import static org.apache.lucene.util.LuceneTestCase.TEST_DOCVALUESFORMAT;
+import static org.apache.lucene.util.LuceneTestCase.TEST_POSTINGSFORMAT;
+import static org.apache.lucene.util.LuceneTestCase.VERBOSE;
+import static org.apache.lucene.util.LuceneTestCase.assumeFalse;
+import static org.apache.lucene.util.LuceneTestCase.localeForName;
+import static org.apache.lucene.util.LuceneTestCase.random;
+import static org.apache.lucene.util.LuceneTestCase.randomLocale;
+import static org.apache.lucene.util.LuceneTestCase.randomTimeZone;
+
import java.io.PrintStream;
-import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
@@ -40,23 +48,21 @@
import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
import org.apache.lucene.search.RandomSimilarityProvider;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; // javadocs
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.junit.internal.AssumptionViolatedException;
+
import com.carrotsearch.randomizedtesting.RandomizedContext;
-import static org.apache.lucene.util.LuceneTestCase.*;
-
/**
* Setup and restore suite-level environment (fine grained junk that
* doesn't fit anywhere else).
@@ -192,7 +198,7 @@
dvFormat = DocValuesFormat.forName(TEST_DOCVALUESFORMAT);
}
- codec = new Lucene45Codec() {
+ codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
Index: lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (revision 1525565)
+++ lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (working copy)
@@ -44,7 +44,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@@ -59,18 +59,17 @@
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
import org.apache.lucene.index.CheckIndex.Status.FieldNormStatus;
import org.apache.lucene.index.CheckIndex.Status.StoredFieldStatus;
import org.apache.lucene.index.CheckIndex.Status.TermIndexStatus;
import org.apache.lucene.index.CheckIndex.Status.TermVectorStatus;
-import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexableField;
@@ -78,7 +77,8 @@
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfoPerCommit;
+import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
@@ -87,9 +87,7 @@
import org.apache.lucene.search.FilteredQuery.FilterStrategy;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
import org.junit.Assert;
import com.carrotsearch.randomizedtesting.RandomizedContext;
@@ -703,7 +701,7 @@
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing postings format to:" + format);
}
- return new Lucene45Codec() {
+ return new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;
@@ -721,7 +719,7 @@
if (LuceneTestCase.VERBOSE) {
System.out.println("forcing docvalues format to:" + format);
}
- return new Lucene45Codec() {
+ return new Lucene46Codec() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return format;
@@ -999,25 +997,8 @@
}
}
- public static FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
- Directory cfsDir = null;
- try {
- if (info.getUseCompoundFile()) {
- cfsDir = new CompoundFileDirectory(info.dir,
- IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
- IOContext.READONCE,
- false);
- } else {
- cfsDir = info.dir;
- }
- return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir,
- info.name,
- IOContext.READONCE);
- } finally {
- if (info.getUseCompoundFile() && cfsDir != null) {
- cfsDir.close();
- }
- }
+ public static FieldInfos getFieldInfos(SegmentInfoPerCommit info) throws IOException {
+ return SegmentReader.readFieldInfos(info);
}
/**
Index: lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
===================================================================
--- lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (revision 1525565)
+++ lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (working copy)
@@ -22,3 +22,4 @@
org.apache.lucene.codecs.lucene40.Lucene40RWCodec
org.apache.lucene.codecs.lucene41.Lucene41RWCodec
org.apache.lucene.codecs.lucene42.Lucene42RWCodec
+org.apache.lucene.codecs.lucene45.Lucene45RWCodec
Index: solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
===================================================================
--- solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (revision 1525565)
+++ solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (working copy)
@@ -3,7 +3,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.util.plugin.SolrCoreAware;
@@ -51,7 +51,7 @@
@Override
public void init(NamedList args) {
super.init(args);
- codec = new Lucene45Codec() {
+ codec = new Lucene46Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField fieldOrNull = core.getLatestSchema().getFieldOrNull(field);