You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/26 20:33:51 UTC
svn commit: r1189367 [2/3] - in /lucene/dev/branches/lucene2621:
lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/
lucene/src/java/org/apache/lucene/index/
lucene/src/java/org/apache/lucene/index/codecs/
lucene/src/java/org/apache/...
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -51,7 +51,7 @@ import org.apache.lucene.util.UnicodeUti
* @deprecated (4.0)
*/
@Deprecated
-public class PreFlexFields extends FieldsProducer {
+public class Lucene3xFields extends FieldsProducer {
private static final boolean DEBUG_SURROGATES = false;
@@ -68,7 +68,7 @@ public class PreFlexFields extends Field
private final IOContext context;
private Directory cfsReader;
- public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
+ public Lucene3xFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
throws IOException {
si = info;
@@ -95,7 +95,7 @@ public class PreFlexFields extends Field
// make sure that all index files have been read or are kept open
// so that if an index update removes them we'll still have them
- freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION), context);
+ freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION), context);
boolean anyProx = false;
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed) {
@@ -108,7 +108,7 @@ public class PreFlexFields extends Field
}
if (anyProx) {
- proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION), context);
+ proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION), context);
} else {
proxStream = null;
}
@@ -136,16 +136,16 @@ public class PreFlexFields extends Field
}
static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
- files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_EXTENSION));
- files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION));
- files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION));
if (info.getHasProx()) {
// LUCENE-1739: for certain versions of 2.9-dev,
// hasProx would be incorrectly computed during
// indexing as true, and then stored into the segments
// file, when it should have been false. So we do the
// extra check, here:
- final String prx = IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION);
+ final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
if (dir.fileExists(prx)) {
files.add(prx);
}
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -40,7 +40,7 @@ import org.apache.lucene.index.codecs.Pe
* @lucene.experimental
*/
@Deprecated
-public class PreFlexPostingsFormat extends PostingsFormat {
+public class Lucene3xPostingsFormat extends PostingsFormat {
/** Extension of terms file */
public static final String TERMS_EXTENSION = "tis";
@@ -54,7 +54,7 @@ public class PreFlexPostingsFormat exten
/** Extension of prox postings file */
public static final String PROX_EXTENSION = "prx";
- public PreFlexPostingsFormat() {
+ public Lucene3xPostingsFormat() {
super("PreFlex");
}
@@ -65,13 +65,13 @@ public class PreFlexPostingsFormat exten
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
+ return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
}
@Override
public void files(Directory dir, SegmentInfo info, int id, Set<String> files) throws IOException {
// preflex fields have no codec ID - we ignore it here
- PreFlexFields.files(dir, info, files);
+ Lucene3xFields.files(dir, info, files);
}
@Override
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -23,7 +23,7 @@ import org.apache.lucene.index.FieldInfo
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Term;
-import org.apache.lucene.index.codecs.standard.DefaultSkipListReader;
+import org.apache.lucene.index.codecs.lucene40.DefaultSkipListReader;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -110,7 +110,7 @@ public final class TermInfosReader {
segment = seg;
fieldInfos = fis;
- origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_EXTENSION),
+ origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_EXTENSION),
context), fieldInfos, false);
size = origEnum.size;
@@ -118,7 +118,7 @@ public final class TermInfosReader {
if (indexDivisor != -1) {
// Load terms index
totalIndexInterval = origEnum.indexInterval * indexDivisor;
- final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION),
+ final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION),
context), fieldInfos, true);
try {
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java?rev=1189367&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Wed Oct 26 18:33:48 2011
@@ -0,0 +1,76 @@
+package org.apache.lucene.index.codecs.lucene40;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.codecs.DefaultFieldsFormat;
+import org.apache.lucene.index.codecs.FieldsFormat;
+import org.apache.lucene.index.codecs.PostingsFormat;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.perfield.PerFieldCodec;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+
+/**
+ * Implements the Lucene 4.0 index format, with configurable per-field postings formats
+ * and using {@link DefaultFieldsFormat}
+ * @lucene.experimental
+ */
+// TODO: which postings formats will we actually support for backwards compatibility?
+public class Lucene40Codec extends PerFieldCodec {
+ private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
+
+ public Lucene40Codec() {
+ this(Collections.<String,String>emptyMap());
+ }
+
+ public Lucene40Codec(Map<String,String> perFieldMap) {
+ this("Lucene40", perFieldMap);
+ }
+
+ public Lucene40Codec(String defaultFormat, Map<String,String> perFieldMap) {
+ super("Lucene40", defaultFormat, perFieldMap);
+ }
+
+ @Override
+ public FieldsFormat fieldsFormat() {
+ return fieldsFormat;
+ }
+
+ @Override
+ public PostingsFormat lookup(String name) {
+ final PostingsFormat codec = CORE_FORMATS.get(name);
+ if (codec == null) {
+ throw new IllegalArgumentException("required format '" + name + "' not found; known formats: " + CORE_FORMATS.keySet());
+ }
+ return codec;
+ }
+
+ // postings formats
+ private static final Map<String,PostingsFormat> CORE_FORMATS = new HashMap<String,PostingsFormat>();
+ static {
+ CORE_FORMATS.put("Lucene40", new Lucene40PostingsFormat());
+ CORE_FORMATS.put("Pulsing", new PulsingPostingsFormat());
+ CORE_FORMATS.put("SimpleText", new SimpleTextPostingsFormat());
+ CORE_FORMATS.put("Memory", new MemoryPostingsFormat());
+ }
+}
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -39,17 +39,17 @@ import org.apache.lucene.store.Directory
/** Default codec.
* @lucene.experimental */
-public class StandardPostingsFormat extends PostingsFormat {
+public class Lucene40PostingsFormat extends PostingsFormat {
private final int minBlockSize;
private final int maxBlockSize;
- public StandardPostingsFormat() {
+ public Lucene40PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
- public StandardPostingsFormat(int minBlockSize, int maxBlockSize) {
- super("Standard");
+ public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
+ super("Lucene40");
this.minBlockSize = minBlockSize;
assert minBlockSize > 1;
this.maxBlockSize = maxBlockSize;
@@ -57,7 +57,7 @@ public class StandardPostingsFormat exte
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase docs = new StandardPostingsWriter(state);
+ PostingsWriterBase docs = new Lucene40PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@@ -79,7 +79,7 @@ public class StandardPostingsFormat exte
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+ PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
boolean success = false;
try {
@@ -89,7 +89,7 @@ public class StandardPostingsFormat exte
state.segmentInfo.name,
postings,
state.context,
- state.codecId,
+ state.formatId,
state.termsIndexDivisor);
success = true;
return ret;
@@ -108,7 +108,7 @@ public class StandardPostingsFormat exte
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
- StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+ Lucene40PostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
}
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -42,7 +42,7 @@ import org.apache.lucene.util.CodecUtil;
* postings format.
* @lucene.experimental */
-public class StandardPostingsReader extends PostingsReaderBase {
+public class Lucene40PostingsReader extends PostingsReaderBase {
private final IndexInput freqIn;
private final IndexInput proxIn;
@@ -54,14 +54,14 @@ public class StandardPostingsReader exte
// private String segment;
- public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
- freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.FREQ_EXTENSION),
+ public Lucene40PostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
+ freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, Lucene40PostingsFormat.FREQ_EXTENSION),
ioContext);
// this.segment = segmentInfo.name;
if (segmentInfo.getHasProx()) {
boolean success = false;
try {
- proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.PROX_EXTENSION),
+ proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, Lucene40PostingsFormat.PROX_EXTENSION),
ioContext);
success = true;
} finally {
@@ -75,9 +75,9 @@ public class StandardPostingsReader exte
}
public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
- files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.FREQ_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, Lucene40PostingsFormat.FREQ_EXTENSION));
if (segmentInfo.getHasProx()) {
- files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.PROX_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, Lucene40PostingsFormat.PROX_EXTENSION));
}
}
@@ -85,8 +85,8 @@ public class StandardPostingsReader exte
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching past writer
- CodecUtil.checkHeader(termsIn, StandardPostingsWriter.CODEC,
- StandardPostingsWriter.VERSION_START, StandardPostingsWriter.VERSION_START);
+ CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.CODEC,
+ Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_START);
skipInterval = termsIn.readInt();
maxSkipLevels = termsIn.readInt();
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -38,8 +38,8 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
/** @lucene.experimental */
-public final class StandardPostingsWriter extends PostingsWriterBase {
- final static String CODEC = "StandardPostingsWriter";
+public final class Lucene40PostingsWriter extends PostingsWriterBase {
+ final static String CODEC = "Lucene40PostingsWriter";
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
@@ -81,21 +81,21 @@ public final class StandardPostingsWrite
// private String segment;
- public StandardPostingsWriter(SegmentWriteState state) throws IOException {
+ public Lucene40PostingsWriter(SegmentWriteState state) throws IOException {
this(state, DEFAULT_SKIP_INTERVAL);
}
- public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
+ public Lucene40PostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
super();
this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */
// this.segment = state.segmentName;
- String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.FREQ_EXTENSION);
+ String fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, Lucene40PostingsFormat.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName, state.context);
if (state.fieldInfos.hasProx()) {
// At least one field does not omit TF, so create the
// prox file
- fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.PROX_EXTENSION);
+ fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, Lucene40PostingsFormat.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName, state.context);
} else {
// Every field omits TF so we will write no prox file
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -248,7 +248,7 @@ public class MemoryPostingsFormat extend
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
+ final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, EXTENSION);
final IndexOutput out = state.directory.createOutput(fileName, state.context);
return new FieldsConsumer() {
@@ -740,7 +740,7 @@ public class MemoryPostingsFormat extend
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
+ final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.formatId, EXTENSION);
final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java?rev=1189367&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java Wed Oct 26 18:33:48 2011
@@ -0,0 +1,87 @@
+package org.apache.lucene.index.codecs.perfield;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultFieldsFormat;
+import org.apache.lucene.index.codecs.FieldsFormat;
+import org.apache.lucene.index.codecs.PostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+
+public abstract class PerFieldCodec extends Codec {
+ private final PostingsFormat postingsFormat;
+
+ private final String defaultPostingsFormat;
+ private final Map<String,String> perFieldMap;
+
+ public PerFieldCodec(String name, String defaultPostingsFormat, Map<String,String> perFieldMap) {
+ super(name);
+ this.defaultPostingsFormat = defaultPostingsFormat;
+ this.perFieldMap = perFieldMap;
+ this.postingsFormat = new PerFieldPostingsFormat(this);
+ }
+
+ @Override
+ public PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ /**
+ * Returns the {@link PostingsFormat} name for the given field or the default format if
+ * not set.
+ *
+ * @param name
+ * the fields name
+ * @return the {@link PostingsFormat} name for the given field or the default format if
+ * not set.
+ */
+ public String getPostingsFormat(String name) {
+ final String format;
+ if ((format = perFieldMap.get(name)) == null) {
+ return defaultPostingsFormat;
+ }
+ return format;
+ }
+
+ /**
+ * Returns <code>true</code> if this Codec has a Format registered for this
+ * field.
+ */
+ public boolean hasPostingsFormatFormat(String name) {
+ return perFieldMap.containsKey(name);
+ }
+
+ /**
+ * Returns the default {@link PostingsFormat} for this {@link Codec}
+ *
+ * @return the default {@link PostingsFormat} for this {@link Codec}
+ */
+ public String getDefaultPostingsFormat() {
+ return defaultPostingsFormat;
+ }
+
+ public abstract PostingsFormat lookup(String name);
+}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -49,20 +49,20 @@ import org.apache.lucene.store.IOContext
import org.apache.lucene.util.IOUtils;
/**
- * Enables native per field codec support. This class selects the codec used to
- * write a field depending on the provided {@link SegmentCodecs}. For each field
- * seen it resolves the codec based on the {@link FieldInfo#codecId} which is
- * only valid during a segment merge. See {@link SegmentCodecs} javadoc for
+ * Enables per field format support. This class selects the format used to
+ * write a field depending on the provided {@link SegmentFormats}. For each field
+ * seen it resolves the format based on the {@link FieldInfo#formatId} which is
+ * only valid during a segment merge. See {@link SegmentFormats} javadoc for
* details.
*
* @lucene.internal
*/
final class PerFieldPostingsFormat extends PostingsFormat {
- private final SegmentCodecs segmentCodecs;
+ private final SegmentFormats segmentFormats;
- PerFieldPostingsFormat(SegmentCodecs segmentCodecs) {
+ PerFieldPostingsFormat(SegmentFormats segmentFormats) {
super("PerField");
- this.segmentCodecs = segmentCodecs;
+ this.segmentFormats = segmentFormats;
}
@Override
@@ -75,12 +75,12 @@ final class PerFieldPostingsFormat exten
private final ArrayList<FieldsConsumer> consumers = new ArrayList<FieldsConsumer>();
public FieldsWriter(SegmentWriteState state) throws IOException {
- assert segmentCodecs == state.segmentCodecs;
- final PostingsFormat[] codecs = segmentCodecs.codecs;
- for (int i = 0; i < codecs.length; i++) {
+ assert segmentFormats == state.segmentFormats;
+ final PostingsFormat[] formats = segmentFormats.formats;
+ for (int i = 0; i < formats.length; i++) {
boolean success = false;
try {
- consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, i)));
+ consumers.add(formats[i].fieldsConsumer(new SegmentWriteState(state, i)));
success = true;
} finally {
if (!success) {
@@ -92,8 +92,8 @@ final class PerFieldPostingsFormat exten
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
- assert field.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
- final FieldsConsumer fields = consumers.get(field.getCodecId());
+ assert field.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+ final FieldsConsumer fields = consumers.get(field.getFormatId());
return fields.addField(field);
}
@@ -117,13 +117,13 @@ final class PerFieldPostingsFormat exten
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed) {
fields.add(fi.name);
- assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
- PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
- if (!producers.containsKey(codec)) {
- producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
- si, fieldInfos, context, indexDivisor, fi.getCodecId())));
+ assert fi.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+ PostingsFormat format = segmentFormats.formats[fi.getFormatId()];
+ if (!producers.containsKey(format)) {
+ producers.put(format, format.fieldsProducer(new SegmentReadState(dir,
+ si, fieldInfos, context, indexDivisor, fi.getFormatId())));
}
- codecs.put(fi.name, producers.get(codec));
+ codecs.put(fi.name, producers.get(format));
}
}
success = true;
@@ -193,16 +193,16 @@ final class PerFieldPostingsFormat exten
}
@Override
- public void files(Directory dir, SegmentInfo info, int codecId, Set<String> files)
+ public void files(Directory dir, SegmentInfo info, int formatId, Set<String> files)
throws IOException {
- // ignore codecid since segmentCodec will assign it per codec
- segmentCodecs.files(dir, info, files);
+ // ignore formatid since segmentFormat will assign it per codec
+ segmentFormats.files(dir, info, files);
}
@Override
public void getExtensions(Set<String> extensions) {
- for (PostingsFormat codec : segmentCodecs.codecs) {
- codec.getExtensions(extensions);
+ for (PostingsFormat format : segmentFormats.formats) {
+ format.getExtensions(extensions);
}
}
@@ -227,11 +227,11 @@ final class PerFieldPostingsFormat exten
try {
for (FieldInfo fi : fieldInfos) {
if (fi.hasDocValues()) {
- assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
- PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
+ assert fi.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+ PostingsFormat codec = segmentFormats.formats[fi.getFormatId()];
if (!producers.containsKey(codec)) {
producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
- si, fieldInfos, context, indexDivisor, fi.getCodecId())));
+ si, fieldInfos, context, indexDivisor, fi.getFormatId())));
}
codecs.put(fi.name, producers.get(codec));
}
@@ -264,14 +264,14 @@ final class PerFieldPostingsFormat exten
private final class PerDocConsumers extends PerDocConsumer {
private final PerDocConsumer[] consumers;
- private final PostingsFormat[] codecs;
+ private final PostingsFormat[] formats;
private final PerDocWriteState state;
public PerDocConsumers(PerDocWriteState state) throws IOException {
- assert segmentCodecs == state.segmentCodecs;
+ assert segmentFormats == state.segmentFormats;
this.state = state;
- codecs = segmentCodecs.codecs;
- consumers = new PerDocConsumer[codecs.length];
+ formats = segmentFormats.formats;
+ consumers = new PerDocConsumer[formats.length];
}
public void close() throws IOException {
@@ -280,13 +280,13 @@ final class PerFieldPostingsFormat exten
@Override
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
- final int codecId = field.getCodecId();
- assert codecId != FieldInfo.UNASSIGNED_CODEC_ID;
- PerDocConsumer perDoc = consumers[codecId];
+ final int formatId = field.getFormatId();
+ assert formatId != FieldInfo.UNASSIGNED_FORMAT_ID;
+ PerDocConsumer perDoc = consumers[formatId];
if (perDoc == null) {
- perDoc = codecs[codecId].docsConsumer(new PerDocWriteState(state, codecId));
+ perDoc = formats[formatId].docsConsumer(new PerDocWriteState(state, formatId));
assert perDoc != null;
- consumers[codecId] = perDoc;
+ consumers[formatId] = perDoc;
}
return perDoc.addValuesField(field);
}
Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java Wed Oct 26 18:33:48 2011
@@ -28,76 +28,75 @@ import org.apache.lucene.index.FieldInfo
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
/**
- * SegmentCodecs maintains an ordered list of distinct codecs used within a
- * segment. Within a segment on codec is used to write multiple fields while
- * each field could be written by a different codec. To enable codecs per field
- * within a single segment we need to record the distinct codecs and map them to
+ * SegmentFormats maintains an ordered list of distinct formats used within a
+ * segment. Within a segment a format is used to write multiple fields while
+ * each field could be written by a different format. To enable formats per field
+ * within a single segment we need to record the distinct formats and map them to
* each field present in the segment. SegmentCodecs is created together with
* {@link SegmentWriteState} for each flush and is maintained in the
* corresponding {@link SegmentInfo} until it is committed.
* <p>
- * During indexing {@link FieldInfos} uses {@link SegmentCodecsBuilder} to incrementally
- * build the {@link SegmentCodecs} mapping. Once a segment is flushed
- * DocumentsWriter creates a {@link SegmentCodecs} instance from
- * {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#codecId}
- * assigned by {@link SegmentCodecsBuilder} refers to the codecs ordinal
- * maintained inside {@link SegmentCodecs}. This ord is later used to get the
- * right codec when the segment is opened in a reader.The {@link PostingsFormat} returned
- * from {@link SegmentCodecs#codec()} in turn uses {@link SegmentCodecs}
- * internal structure to select and initialize the right codec for a fields when
+ * During indexing {@link FieldInfos} uses {@link SegmentFormatsBuilder} to incrementally
+ * build the {@link SegmentFormats} mapping. Once a segment is flushed
+ * DocumentsWriter creates a {@link SegmentFormats} instance from
+ * {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#formatId}
+ * assigned by {@link SegmentFormatsBuilder} refers to the formats ordinal
+ * maintained inside {@link SegmentFormats}. This ord is later used to get the
+ * right format when the segment is opened in a reader.The {@link PostingsFormat} returned
+ * from {@link SegmentFormats#format()} in turn uses {@link SegmentFormats}
+ * internal structure to select and initialize the right format for a fields when
* it is written.
* <p>
- * Once a flush succeeded the {@link SegmentCodecs} is maintained inside the
+ * Once a flush succeeded the {@link SegmentFormats} is maintained inside the
* {@link SegmentInfo} for the flushed segment it was created for.
- * {@link SegmentInfo} writes the name of each codec in {@link SegmentCodecs}
+ * {@link SegmentInfo} writes the name of each format in {@link SegmentFormats}
* for each segment and maintains the order. Later if a segment is opened by a
- * reader this mapping is deserialized and used to create the codec per field.
+ * reader this mapping is deserialized and used to create the format per field.
*
*
* @lucene.internal
*/
-public final class SegmentCodecs implements Cloneable {
+public final class SegmentFormats implements Cloneable {
/**
- * internal structure to map codecs to fields - don't modify this from outside
+ * internal structure to map formats to fields - don't modify this from outside
* of this class!
*/
- public final PostingsFormat[] codecs;
- public final CodecProvider provider;
- private final PostingsFormat codec;
+ public final PostingsFormat[] formats;
+ public final PerFieldCodec provider;
+ private final PostingsFormat format;
- public SegmentCodecs(CodecProvider provider, IndexInput input) throws IOException {
+ public SegmentFormats(PerFieldCodec provider, IndexInput input) throws IOException {
this(provider, read(input, provider));
}
- public SegmentCodecs(CodecProvider provider, PostingsFormat... codecs) {
+ public SegmentFormats(PerFieldCodec provider, PostingsFormat... formats) {
this.provider = provider;
- this.codecs = codecs;
- if (codecs.length == 1 && codecs[0] instanceof PreFlexPostingsFormat) {
- this.codec = codecs[0]; // hack for backwards break... don't wrap the codec in preflex
+ this.formats = formats;
+ if (formats.length == 1 && formats[0] instanceof Lucene3xPostingsFormat) {
+ this.format = formats[0]; // hack for backwards break... don't wrap the codec in preflex
} else {
- this.codec = new PerFieldPostingsFormat(this);
+ this.format = new PerFieldPostingsFormat(this);
}
}
- public PostingsFormat codec() {
- return codec;
+ public PostingsFormat format() {
+ return format;
}
public void write(IndexOutput out) throws IOException {
- out.writeVInt(codecs.length);
- for (PostingsFormat codec : codecs) {
- out.writeString(codec.name);
+ out.writeVInt(formats.length);
+ for (PostingsFormat format : formats) {
+ out.writeString(format.name);
}
}
- private static PostingsFormat[] read(IndexInput in, CodecProvider provider) throws IOException {
+ private static PostingsFormat[] read(IndexInput in, PerFieldCodec provider) throws IOException {
final int size = in.readVInt();
final ArrayList<PostingsFormat> list = new ArrayList<PostingsFormat>();
for (int i = 0; i < size; i++) {
@@ -110,7 +109,7 @@ public final class SegmentCodecs impleme
public void files(Directory dir, SegmentInfo info, Set<String> files)
throws IOException {
- final PostingsFormat[] codecArray = codecs;
+ final PostingsFormat[] codecArray = formats;
for (int i = 0; i < codecArray.length; i++) {
codecArray[i].files(dir, info, i, files);
}
@@ -119,52 +118,52 @@ public final class SegmentCodecs impleme
@Override
public String toString() {
- return "SegmentCodecs [codecs=" + Arrays.toString(codecs) + ", provider=" + provider + "]";
+ return "SegmentFormats [formats=" + Arrays.toString(formats) + ", provider=" + provider + "]";
}
/**
- * Used in {@link FieldInfos} to incrementally build the codec ID mapping for
+ * Used in {@link FieldInfos} to incrementally build the format ID mapping for
* {@link FieldInfo} instances.
* <p>
* Note: this class is not thread-safe
* </p>
- * @see FieldInfo#getCodecId()
+ * @see FieldInfo#getFormatId()
*/
- public final static class SegmentCodecsBuilder {
- private final Map<PostingsFormat, Integer> codecRegistry = new IdentityHashMap<PostingsFormat, Integer>();
- private final ArrayList<PostingsFormat> codecs = new ArrayList<PostingsFormat>();
- private final CodecProvider provider;
+ public final static class SegmentFormatsBuilder {
+ private final Map<PostingsFormat, Integer> formatRegistry = new IdentityHashMap<PostingsFormat, Integer>();
+ private final ArrayList<PostingsFormat> formats = new ArrayList<PostingsFormat>();
+ private final PerFieldCodec provider;
- private SegmentCodecsBuilder(CodecProvider provider) {
+ private SegmentFormatsBuilder(PerFieldCodec provider) {
this.provider = provider;
}
- public static SegmentCodecsBuilder create(CodecProvider provider) {
- return new SegmentCodecsBuilder(provider);
+ public static SegmentFormatsBuilder create(PerFieldCodec provider) {
+ return new SegmentFormatsBuilder(provider);
}
- public SegmentCodecsBuilder tryAddAndSet(FieldInfo fi) {
- if (fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
+ public SegmentFormatsBuilder tryAddAndSet(FieldInfo fi) {
+ if (fi.getFormatId() == FieldInfo.UNASSIGNED_FORMAT_ID) {
final PostingsFormat fieldCodec = provider.lookup(provider
- .getFieldCodec(fi.name));
- Integer ord = codecRegistry.get(fieldCodec);
+ .getPostingsFormat(fi.name));
+ Integer ord = formatRegistry.get(fieldCodec);
if (ord == null) {
- ord = Integer.valueOf(codecs.size());
- codecRegistry.put(fieldCodec, ord);
- codecs.add(fieldCodec);
+ ord = Integer.valueOf(formats.size());
+ formatRegistry.put(fieldCodec, ord);
+ formats.add(fieldCodec);
}
- fi.setCodecId(ord.intValue());
+ fi.setFormatId(ord.intValue());
}
return this;
}
- public SegmentCodecs build() {
- return new SegmentCodecs(provider, codecs.toArray(PostingsFormat.EMPTY));
+ public SegmentFormats build() {
+ return new SegmentFormats(provider, formats.toArray(PostingsFormat.EMPTY));
}
- public SegmentCodecsBuilder clear() {
- codecRegistry.clear();
- codecs.clear();
+ public SegmentFormatsBuilder clear() {
+ formatRegistry.clear();
+ formats.clear();
return this;
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -35,9 +35,9 @@ import org.apache.lucene.index.codecs.Fi
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsReader;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsWriter;
import org.apache.lucene.store.Directory;
/** This codec "inlines" the postings for terms that have
@@ -82,7 +82,7 @@ public class PulsingPostingsFormat exten
// We wrap StandardPostingsWriter, but any PostingsWriterBase
// will work:
- PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
+ PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
// Terms that have <= freqCutoff number of docs are
// "pulsed" (inlined):
@@ -106,7 +106,7 @@ public class PulsingPostingsFormat exten
// We wrap StandardPostingsReader, but any StandardPostingsReader
// will work:
- PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+ PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
boolean success = false;
@@ -115,7 +115,7 @@ public class PulsingPostingsFormat exten
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.context,
- state.codecId,
+ state.formatId,
state.termsIndexDivisor);
success = true;
return ret;
@@ -132,14 +132,14 @@ public class PulsingPostingsFormat exten
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
- StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+ Lucene40PostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
}
@Override
public void getExtensions(Set<String> extensions) {
- StandardPostingsFormat.getStandardExtensions(extensions);
+ Lucene40PostingsFormat.getStandardExtensions(extensions);
DefaultDocValuesConsumer.getExtensions(extensions);
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java Wed Oct 26 18:33:48 2011
@@ -47,12 +47,12 @@ public class SepDocValuesConsumer extend
}
@SuppressWarnings("fallthrough")
- public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
+ public static void files(Directory dir, SegmentInfo segmentInfo, int formatId,
Set<String> files) throws IOException {
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
for (FieldInfo fieldInfo : fieldInfos) {
- if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
- String filename = docValuesId(segmentInfo.name, codecId, fieldInfo.number);
+ if (fieldInfo.getFormatId() == formatId && fieldInfo.hasDocValues()) {
+ String filename = docValuesId(segmentInfo.name, formatId, fieldInfo.number);
switch (fieldInfo.getDocValues()) {
case BYTES_FIXED_DEREF:
case BYTES_VAR_DEREF:
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java Wed Oct 26 18:33:48 2011
@@ -39,7 +39,7 @@ public class SepDocValuesProducer extend
* {@link IndexDocValues} instances for this segment and codec.
*/
public SepDocValuesProducer(SegmentReadState state) throws IOException {
- docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.codecId, state.context);
+ docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.formatId, state.context);
}
@Override
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Wed Oct 26 18:33:48 2011
@@ -116,27 +116,27 @@ public final class SepPostingsWriter ext
try {
this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */
- final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
+ final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, DOC_EXTENSION);
docOut = factory.createOutput(state.directory, docFileName, state.context);
docIndex = docOut.index();
if (state.fieldInfos.hasFreq()) {
- final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
+ final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, FREQ_EXTENSION);
freqOut = factory.createOutput(state.directory, frqFileName, state.context);
freqIndex = freqOut.index();
}
if (state.fieldInfos.hasProx()) {
- final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
+ final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, POS_EXTENSION);
posOut = factory.createOutput(state.directory, posFileName, state.context);
posIndex = posOut.index();
// TODO: -- only if at least one field stores payloads?
- final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
+ final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, PAYLOAD_EXTENSION);
payloadOut = state.directory.createOutput(payloadFileName, state.context);
}
- final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
+ final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, SKIP_EXTENSION);
skipOut = state.directory.createOutput(skipFileName, state.context);
totalNumDocs = state.numDocs;
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Wed Oct 26 18:33:48 2011
@@ -60,7 +60,7 @@ class SimpleTextFieldsReader extends Fie
final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
- in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
+ in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.formatId), state.context);
fieldInfos = state.fieldInfos;
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Wed Oct 26 18:33:48 2011
@@ -47,7 +47,7 @@ class SimpleTextFieldsWriter extends Fie
final static BytesRef PAYLOAD = new BytesRef(" payload ");
public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
- final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.codecId);
+ final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.formatId);
out = state.directory.createOutput(fileName, state.context);
}
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java Wed Oct 26 18:33:48 2011
@@ -27,11 +27,11 @@ import java.util.Random;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
import org.apache.lucene.util._TestUtil;
/**
@@ -54,8 +54,8 @@ public class RandomCodecProvider extends
// block via CL:
int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
- register(new StandardPostingsFormat(minItemsPerBlock, maxItemsPerBlock));
- register(new PreFlexPostingsFormat());
+ register(new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock));
+ register(new Lucene3xPostingsFormat());
// TODO: make it possible to specify min/max iterms per
// block via CL:
minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -34,9 +34,9 @@ import org.apache.lucene.index.codecs.se
import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -187,7 +187,7 @@ public class MockFixedIntBlockPostingsFo
state.segmentInfo.name,
postingsReader,
state.context,
- StandardPostingsFormat.TERMS_CACHE_SIZE,
+ Lucene40PostingsFormat.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -34,9 +34,9 @@ import org.apache.lucene.index.codecs.se
import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -210,7 +210,7 @@ public class MockVariableIntBlockPosting
state.segmentInfo.name,
postingsReader,
state.context,
- StandardPostingsFormat.TERMS_CACHE_SIZE,
+ Lucene40PostingsFormat.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -50,6 +50,8 @@ import org.apache.lucene.index.codecs.Te
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsReader;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsWriter;
import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
@@ -62,8 +64,6 @@ import org.apache.lucene.index.codecs.se
import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@@ -164,7 +164,7 @@ public class MockRandomPostingsFormat ex
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing Standard postings");
}
- postingsWriter = new StandardPostingsWriter(state, skipInterval);
+ postingsWriter = new Lucene40PostingsWriter(state, skipInterval);
}
if (random.nextBoolean()) {
@@ -303,7 +303,7 @@ public class MockRandomPostingsFormat ex
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Standard postings");
}
- postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+ postingsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
}
if (random.nextBoolean()) {
@@ -421,7 +421,7 @@ public class MockRandomPostingsFormat ex
final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);
files.add(seedFileName);
SepPostingsReader.files(segmentInfo, codecId, files);
- StandardPostingsReader.files(dir, segmentInfo, codecId, files);
+ Lucene40PostingsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -37,7 +37,7 @@ import org.apache.lucene.index.codecs.Bl
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
@@ -119,7 +119,7 @@ public class MockSepPostingsFormat exten
state.segmentInfo.name,
postingsReader,
state.context,
- StandardPostingsFormat.TERMS_CACHE_SIZE,
+ Lucene40PostingsFormat.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Wed Oct 26 18:33:48 2011
@@ -29,9 +29,9 @@ import org.apache.lucene.index.codecs.Fi
import org.apache.lucene.index.codecs.PostingsConsumer;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.TermsConsumer;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.TermInfo;
-import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.TermInfo;
+import org.apache.lucene.index.codecs.lucene40.DefaultSkipListWriter;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@@ -50,12 +50,12 @@ class PreFlexFieldsWriter extends Fields
state.fieldInfos,
state.termIndexInterval);
- final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.FREQ_EXTENSION);
+ final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", Lucene3xPostingsFormat.FREQ_EXTENSION);
freqOut = state.directory.createOutput(freqFile, state.context);
totalNumDocs = state.numDocs;
if (state.fieldInfos.hasProx()) {
- final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.PROX_EXTENSION);
+ final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", Lucene3xPostingsFormat.PROX_EXTENSION);
proxOut = state.directory.createOutput(proxFile, state.context);
} else {
proxOut = null;
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -21,8 +21,8 @@ import java.io.IOException;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexFields;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xFields;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.util.LuceneTestCase;
@@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTest
*
* @lucene.experimental
*/
-public class PreFlexRWPostingsFormat extends PreFlexPostingsFormat {
+public class PreFlexRWPostingsFormat extends Lucene3xPostingsFormat {
public PreFlexRWPostingsFormat() {
// NOTE: we impersonate the PreFlex codec so that it can
@@ -50,7 +50,7 @@ public class PreFlexRWPostingsFormat ext
// Whenever IW opens readers, eg for merging, we have to
// keep terms order in UTF16:
- return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
+ return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
@Override
protected boolean sortTermsByUnicode() {
// We carefully peek into stack track above us: if
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java Wed Oct 26 18:33:48 2011
@@ -23,8 +23,8 @@ import java.io.IOException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.TermInfo;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.TermInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
@@ -106,8 +106,8 @@ final class TermInfosWriter implements C
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
- (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
- : PreFlexPostingsFormat.TERMS_EXTENSION)));
+ (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+ : Lucene3xPostingsFormat.TERMS_EXTENSION)));
} catch (IOException ignored) {
}
}
@@ -125,8 +125,8 @@ final class TermInfosWriter implements C
fieldInfos = fis;
isIndex = isi;
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "",
- (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
- : PreFlexPostingsFormat.TERMS_EXTENSION)), IOContext.DEFAULT);
+ (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+ : Lucene3xPostingsFormat.TERMS_EXTENSION)), IOContext.DEFAULT);
boolean success = false;
try {
output.writeInt(FORMAT_CURRENT); // write format
@@ -147,8 +147,8 @@ final class TermInfosWriter implements C
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
- (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
- : PreFlexPostingsFormat.TERMS_EXTENSION)));
+ (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+ : Lucene3xPostingsFormat.TERMS_EXTENSION)));
} catch (IOException ignored) {
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Wed Oct 26 18:33:48 2011
@@ -41,11 +41,11 @@ import org.apache.lucene.document.FieldT
import org.apache.lucene.index.*;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.index.codecs.mockrandom.MockRandomPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
import org.apache.lucene.index.codecs.preflexrw.PreFlexRWPostingsFormat;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
import org.apache.lucene.search.BooleanQuery;
@@ -293,7 +293,7 @@ public abstract class LuceneTestCase ext
if (preFlex != null) {
cp.unregister(preFlex);
}
- cp.register(new PreFlexPostingsFormat());
+ cp.register(new Lucene3xPostingsFormat());
}
cp.unregister(cp.lookup("MockSep"));
cp.unregister(cp.lookup("MockFixedIntBlock"));
Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Oct 26 18:33:48 2011
@@ -30,10 +30,10 @@ import org.apache.lucene.document.String
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.store.AlreadyClosedException;
@@ -1039,7 +1039,7 @@ public class TestAddIndexes extends Luce
public static class MockCodecProvider extends CodecProvider {
public MockCodecProvider() {
- StandardPostingsFormat standardCodec = new StandardPostingsFormat();
+ Lucene40PostingsFormat standardCodec = new Lucene40PostingsFormat();
SimpleTextPostingsFormat simpleTextCodec = new SimpleTextPostingsFormat();
MockSepPostingsFormat mockSepCodec = new MockSepPostingsFormat();
register(standardCodec);
@@ -1146,7 +1146,7 @@ public class TestAddIndexes extends Luce
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random));
CodecProvider provider = new CodecProvider();
- provider.register(new StandardPostingsFormat());
+ provider.register(new Lucene40PostingsFormat());
conf.setCodecProvider(provider);
IndexWriter w = new IndexWriter(toAdd, conf);
Document doc = new Document();