LUCENE-4086: remove 3.x index support from trunk

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1344076 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-05-30 02:07:31 +00:00
parent f9c744422a
commit 54be02eef2
110 changed files with 189 additions and 7849 deletions

View File

@ -21,12 +21,10 @@ package org.apache.lucene.collation;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CollationTestBase;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import java.text.Collator;
import java.util.Locale;
@SuppressCodecs("Lucene3x")
public class TestCollationKeyAnalyzer extends CollationTestBase {
// the sort order of Ø versus U depends on the version of the rules being used
// for the inherited root locale: Ø's order isnt specified in Locale.US since

View File

@ -22,13 +22,10 @@ import com.ibm.icu.text.Collator;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CollationTestBase;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import java.util.Locale;
@SuppressCodecs("Lucene3x")
public class TestICUCollationKeyAnalyzer extends CollationTestBase {
private Collator collator = Collator.getInstance(new Locale("fa"));

View File

@ -1,151 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.HashSet;
import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.MutableBits;
/**
* Supports the Lucene 3.x index format (readonly)
* @deprecated
*/
@Deprecated
public class Lucene3xCodec extends Codec {
public Lucene3xCodec() {
super("Lucene3x");
}
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
private final StoredFieldsFormat fieldsFormat = new Lucene3xStoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene3xTermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene3xFieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene3xSegmentInfoFormat();
private final Lucene3xNormsFormat normsFormat = new Lucene3xNormsFormat();
/** Extension of compound file for doc store files*/
static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
// TODO: this should really be a different impl
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() {
@Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};
// 3.x doesn't support docvalues
private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return null;
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return null;
}
};
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
/** Returns file names for shared doc stores, if any, else
* null. */
public static Set<String> getDocStoreFiles(SegmentInfo info) {
if (Lucene3xSegmentInfoFormat.getDocStoreOffset(info) != -1) {
final String dsName = Lucene3xSegmentInfoFormat.getDocStoreSegment(info);
Set<String> files = new HashSet<String>();
if (Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(info)) {
files.add(IndexFileNames.segmentFileName(dsName, "", COMPOUND_FILE_STORE_EXTENSION));
} else {
files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
files.add(IndexFileNames.segmentFileName(dsName, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
}
return files;
} else {
return null;
}
}
}

View File

@ -1,45 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
/**
* Lucene3x ReadOnly FieldInfosFromat implementation
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
class Lucene3xFieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene3xFieldInfosReader();
@Override
public FieldInfosReader getFieldInfosReader() throws IOException {
return reader;
}
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -1,117 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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 org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
/**
* @lucene.experimental
* @deprecated
*/
@Deprecated
class Lucene3xFieldInfosReader extends FieldInfosReader {
/** Extension of field infos */
static final String FIELD_INFOS_EXTENSION = "fnm";
// First used in 2.9; prior to 2.9 there was no format header
static final int FORMAT_START = -2;
// First used in 3.4: omit only positional information
static final int FORMAT_OMIT_POSITIONS = -3;
static final int FORMAT_MINIMUM = FORMAT_START;
static final int FORMAT_CURRENT = FORMAT_OMIT_POSITIONS;
static final byte IS_INDEXED = 0x1;
static final byte STORE_TERMVECTOR = 0x2;
static final byte OMIT_NORMS = 0x10;
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
static final byte OMIT_POSITIONS = -128;
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
try {
final int format = input.readVInt();
if (format > FORMAT_MINIMUM) {
throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
}
if (format < FORMAT_CURRENT) {
throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
}
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
for (int i = 0; i < size; i++) {
String name = input.readString();
final int fieldNumber = i;
byte bits = input.readByte();
boolean isIndexed = (bits & IS_INDEXED) != 0;
boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & OMIT_NORMS) != 0;
boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & OMIT_POSITIONS) != 0) {
if (format <= FORMAT_OMIT_POSITIONS) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else {
throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
}
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
}
// LUCENE-3027: past indices were able to write
// storePayloads=true when omitTFAP is also true,
// which is invalid. We correct that, here:
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
storePayloads = false;
}
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, null, isIndexed && !omitNorms? Type.FIXED_INTS_8 : null, Collections.<String,String>emptyMap());
}
if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
return new FieldInfos(infos);
} finally {
input.close();
}
}
}

View File

@ -1,46 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
/**
* Lucene3x ReadOnly NormsFormat implementation
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
class Lucene3xNormsFormat extends NormsFormat {
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene3xNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
}
}

View File

@ -1,240 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.IdentityHashMap;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
/**
* Reads Lucene 3.x norms format and exposes it via DocValues API
* @lucene.experimental
* @deprecated
*/
@Deprecated
class Lucene3xNormsProducer extends PerDocProducer {
/** norms header placeholder */
static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
/** Extension of norms file */
static final String NORMS_EXTENSION = "nrm";
/** Extension of separate norms file */
static final String SEPARATE_NORMS_EXTENSION = "s";
final Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
// any .nrm or .sNN files we have open at any time.
// TODO: just a list, and double-close() separate norms files?
final Set<IndexInput> openFiles = Collections.newSetFromMap(new IdentityHashMap<IndexInput,Boolean>());
// points to a singleNormFile
IndexInput singleNormStream;
final int maxdoc;
// note: just like segmentreader in 3.x, we open up all the files here (including separate norms) up front.
// but we just don't do any seeks or reading yet.
public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context) throws IOException {
Directory separateNormsDir = info.dir; // separate norms are never inside CFS
maxdoc = info.getDocCount();
String segmentName = info.name;
boolean success = false;
try {
long nextNormSeek = NORMS_HEADER.length; //skip header (header unused for now)
for (FieldInfo fi : fields) {
if (fi.hasNorms()) {
String fileName = getNormFilename(info, fi.number);
Directory d = hasSeparateNorms(info, fi.number) ? separateNormsDir : dir;
// singleNormFile means multiple norms share this file
boolean singleNormFile = IndexFileNames.matchesExtension(fileName, NORMS_EXTENSION);
IndexInput normInput = null;
long normSeek;
if (singleNormFile) {
normSeek = nextNormSeek;
if (singleNormStream == null) {
singleNormStream = d.openInput(fileName, context);
openFiles.add(singleNormStream);
}
// All norms in the .nrm file can share a single IndexInput since
// they are only used in a synchronized context.
// If this were to change in the future, a clone could be done here.
normInput = singleNormStream;
} else {
normInput = d.openInput(fileName, context);
openFiles.add(normInput);
// if the segment was created in 3.2 or after, we wrote the header for sure,
// and don't need to do the sketchy file size check. otherwise, we check
// if the size is exactly equal to maxDoc to detect a headerless file.
// NOTE: remove this check in Lucene 5.0!
String version = info.getVersion();
final boolean isUnversioned =
(version == null || StringHelper.getVersionComparator().compare(version, "3.2") < 0)
&& normInput.length() == maxdoc;
if (isUnversioned) {
normSeek = 0;
} else {
normSeek = NORMS_HEADER.length;
}
}
NormsDocValues norm = new NormsDocValues(normInput, normSeek);
norms.put(fi.name, norm);
nextNormSeek += maxdoc; // increment also if some norms are separate
}
}
// TODO: change to a real check? see LUCENE-3619
assert singleNormStream == null || nextNormSeek == singleNormStream.length() : singleNormStream != null ? "len: " + singleNormStream.length() + " expected: " + nextNormSeek : "null";
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(openFiles);
}
}
}
@Override
public DocValues docValues(String field) throws IOException {
return norms.get(field);
}
@Override
public void close() throws IOException {
try {
IOUtils.close(openFiles);
} finally {
norms.clear();
openFiles.clear();
}
}
private static String getNormFilename(SegmentInfo info, int number) {
if (hasSeparateNorms(info, number)) {
long gen = Long.parseLong(info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number));
return IndexFileNames.fileNameFromGeneration(info.name, SEPARATE_NORMS_EXTENSION + number, gen);
} else {
// single file for all norms
return IndexFileNames.segmentFileName(info.name, "", NORMS_EXTENSION);
}
}
private static boolean hasSeparateNorms(SegmentInfo info, int number) {
String v = info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number);
if (v == null) {
return false;
} else {
assert Long.parseLong(v) != SegmentInfo.NO;
return true;
}
}
static final class NormSource extends Source {
protected NormSource(byte[] bytes) {
super(Type.FIXED_INTS_8);
this.bytes = bytes;
}
final byte bytes[];
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.bytes = bytes;
ref.offset = docID;
ref.length = 1;
return ref;
}
@Override
public long getInt(int docID) {
return bytes[docID];
}
@Override
public boolean hasArray() {
return true;
}
@Override
public Object getArray() {
return bytes;
}
}
private class NormsDocValues extends DocValues {
private final IndexInput file;
private final long offset;
public NormsDocValues(IndexInput normInput, long normSeek) {
this.file = normInput;
this.offset = normSeek;
}
@Override
public Source load() throws IOException {
return new NormSource(bytes());
}
@Override
public Source getDirectSource() throws IOException {
return getSource();
}
@Override
public Type getType() {
return Type.FIXED_INTS_8;
}
byte[] bytes() throws IOException {
byte[] bytes = new byte[maxdoc];
// some norms share fds
synchronized(file) {
file.seek(offset);
file.readBytes(bytes, 0, bytes.length, false);
}
// we are done with this file
if (file != singleNormStream) {
openFiles.remove(file);
file.close();
}
return bytes;
}
@Override
public int getValueSize() {
return 1;
}
}
}

View File

@ -1,64 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
/** Codec that reads the pre-flex-indexing postings
* format. It does not provide a writer because newly
* written segments should use the Codec configured on IndexWriter.
*
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
class Lucene3xPostingsFormat extends PostingsFormat {
/** Extension of terms file */
public static final String TERMS_EXTENSION = "tis";
/** Extension of terms index file */
public static final String TERMS_INDEX_EXTENSION = "tii";
/** Extension of freq postings file */
public static final String FREQ_EXTENSION = "frq";
/** Extension of prox postings file */
public static final String PROX_EXTENSION = "prx";
public Lucene3xPostingsFormat() {
super("Lucene3x");
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
}
}

View File

@ -1,89 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.codecs.SegmentInfoWriter;
import org.apache.lucene.index.SegmentInfo;
/**
* Lucene3x ReadOnly SegmentInfoFormat implementation
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
public class Lucene3xSegmentInfoFormat extends SegmentInfoFormat {
private final SegmentInfoReader reader = new Lucene3xSegmentInfoReader();
/** This format adds optional per-segment String
* diagnostics storage, and switches userData to Map */
public static final int FORMAT_DIAGNOSTICS = -9;
/** Each segment records whether it has term vectors */
public static final int FORMAT_HAS_VECTORS = -10;
/** Each segment records the Lucene version that created it. */
public static final int FORMAT_3_1 = -11;
/** Extension used for saving each SegmentInfo, once a 3.x
* index is first committed to with 4.0. */
public static final String UPGRADED_SI_EXTENSION = "si";
public static final String UPGRADED_SI_CODEC_NAME = "Lucene3xSegmentInfo";
public static final int UPGRADED_SI_VERSION_START = 0;
public static final int UPGRADED_SI_VERSION_CURRENT = UPGRADED_SI_VERSION_START;
@Override
public SegmentInfoReader getSegmentInfosReader() {
return reader;
}
@Override
public SegmentInfoWriter getSegmentInfosWriter() {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
// only for backwards compat
public static final String DS_OFFSET_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsoffset";
public static final String DS_NAME_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsname";
public static final String DS_COMPOUND_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dscompound";
public static final String NORMGEN_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normgen";
public static final String NORMGEN_PREFIX = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normfield";
/**
* @return if this segment shares stored fields & vectors, this
* offset is where in that file this segment's docs begin
*/
public static int getDocStoreOffset(SegmentInfo si) {
String v = si.getAttribute(DS_OFFSET_KEY);
return v == null ? -1 : Integer.parseInt(v);
}
/** @return name used to derive fields/vectors file we share with other segments */
public static String getDocStoreSegment(SegmentInfo si) {
String v = si.getAttribute(DS_NAME_KEY);
return v == null ? si.name : v;
}
/** @return whether doc store files are stored in compound file (*.cfx) */
public static boolean getDocStoreIsCompoundFile(SegmentInfo si) {
String v = si.getAttribute(DS_COMPOUND_KEY);
return v == null ? false : Boolean.parseBoolean(v);
}
}

View File

@ -1,272 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.SegmentInfoReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfoPerCommit;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 3x implementation of {@link SegmentInfoReader}.
* @lucene.experimental
* @deprecated
*/
@Deprecated
public class Lucene3xSegmentInfoReader extends SegmentInfoReader {
public static void readLegacyInfos(SegmentInfos infos, Directory directory, IndexInput input, int format) throws IOException {
infos.version = input.readLong(); // read version
infos.counter = input.readInt(); // read counter
Lucene3xSegmentInfoReader reader = new Lucene3xSegmentInfoReader();
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
SegmentInfoPerCommit siPerCommit = reader.readLegacySegmentInfo(directory, format, input);
SegmentInfo si = siPerCommit.info;
if (si.getVersion() == null) {
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
// 2.x segment, and an IndexFormatTooOldException will be thrown,
// which is what we want.
Directory dir = directory;
if (Lucene3xSegmentInfoFormat.getDocStoreOffset(si) != -1) {
if (Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
Lucene3xSegmentInfoFormat.getDocStoreSegment(si), "",
Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), IOContext.READONCE, false);
}
} else if (si.getUseCompoundFile()) {
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
}
try {
Lucene3xStoredFieldsReader.checkCodeVersion(dir, Lucene3xSegmentInfoFormat.getDocStoreSegment(si));
} finally {
// If we opened the directory, close it
if (dir != directory) dir.close();
}
// Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
// time the segment is read, its version won't be null and we won't
// need to open FieldsReader every time for each such segment.
si.setVersion("3.0");
} else if (si.getVersion().equals("2.x")) {
// If it's a 3x index touched by 3.1+ code, then segments record their
// version, whether they are 2.x ones or not. We detect that and throw
// appropriate exception.
throw new IndexFormatTooOldException("segment " + si.name + " in resource " + input, si.getVersion());
}
infos.add(siPerCommit);
}
infos.userData = input.readStringStringMap();
}
@Override
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
// NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene3xSegmentInfoFormat.UPGRADED_SI_EXTENSION);
boolean success = false;
IndexInput input = directory.openInput(fileName, context);
try {
SegmentInfo si = readUpgradedSegmentInfo(segmentName, directory, input);
success = true;
return si;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input);
} else {
input.close();
}
}
}
private static void addIfExists(Directory dir, Set<String> files, String fileName) throws IOException {
if (dir.fileExists(fileName)) {
files.add(fileName);
}
}
/** reads from legacy 3.x segments_N */
private SegmentInfoPerCommit readLegacySegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
// check that it is a format we can understand
if (format > Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS) {
throw new IndexFormatTooOldException(input, format,
Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfoFormat.FORMAT_3_1);
}
if (format < Lucene3xSegmentInfoFormat.FORMAT_3_1) {
throw new IndexFormatTooNewException(input, format,
Lucene3xSegmentInfoFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfoFormat.FORMAT_3_1);
}
final String version;
if (format <= Lucene3xSegmentInfoFormat.FORMAT_3_1) {
version = input.readString();
} else {
version = null;
}
final String name = input.readString();
final int docCount = input.readInt();
final long delGen = input.readLong();
final int docStoreOffset = input.readInt();
final Map<String,String> attributes = new HashMap<String,String>();
// parse the docstore stuff and shove it into attributes
final String docStoreSegment;
final boolean docStoreIsCompoundFile;
if (docStoreOffset != -1) {
docStoreSegment = input.readString();
docStoreIsCompoundFile = input.readByte() == SegmentInfo.YES;
attributes.put(Lucene3xSegmentInfoFormat.DS_OFFSET_KEY, Integer.toString(docStoreOffset));
attributes.put(Lucene3xSegmentInfoFormat.DS_NAME_KEY, docStoreSegment);
attributes.put(Lucene3xSegmentInfoFormat.DS_COMPOUND_KEY, Boolean.toString(docStoreIsCompoundFile));
} else {
docStoreSegment = name;
docStoreIsCompoundFile = false;
}
// pre-4.0 indexes write a byte if there is a single norms file
byte b = input.readByte();
//System.out.println("version=" + version + " name=" + name + " docCount=" + docCount + " delGen=" + delGen + " dso=" + docStoreOffset + " dss=" + docStoreSegment + " dssCFs=" + docStoreIsCompoundFile + " b=" + b + " format=" + format);
assert 1 == b : "expected 1 but was: "+ b + " format: " + format;
final int numNormGen = input.readInt();
final Map<Integer,Long> normGen;
if (numNormGen == SegmentInfo.NO) {
normGen = null;
} else {
normGen = new HashMap<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
normGen.put(j, input.readLong());
}
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final int delCount = input.readInt();
assert delCount <= docCount;
final boolean hasProx = input.readByte() == 1;
final Map<String,String> diagnostics = input.readStringStringMap();
if (format <= Lucene3xSegmentInfoFormat.FORMAT_HAS_VECTORS) {
// NOTE: unused
final int hasVectors = input.readByte();
}
// Replicate logic from 3.x's SegmentInfo.files():
final Set<String> files = new HashSet<String>();
if (isCompoundFile) {
files.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
} else {
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xFieldInfosReader.FIELD_INFOS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.FREQ_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.PROX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.TERMS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xNormsProducer.NORMS_EXTENSION));
}
if (docStoreOffset != -1) {
if (docStoreIsCompoundFile) {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
} else {
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(docStoreSegment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
}
} else if (!isCompoundFile) {
files.add(IndexFileNames.segmentFileName(name, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
files.add(IndexFileNames.segmentFileName(name, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
}
// parse the normgen stuff and shove it into attributes
if (normGen != null) {
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_KEY, Integer.toString(numNormGen));
for(Map.Entry<Integer,Long> ent : normGen.entrySet()) {
long gen = ent.getValue();
if (gen >= SegmentInfo.YES) {
// Definitely a separate norm file, with generation:
files.add(IndexFileNames.fileNameFromGeneration(name, "s" + ent.getKey(), gen));
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + ent.getKey(), Long.toString(gen));
} else if (gen == SegmentInfo.NO) {
// No separate norm
} else {
// We should have already hit indexformat too old exception
assert false;
}
}
}
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
null, diagnostics, Collections.unmodifiableMap(attributes));
info.setFiles(files);
SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, delCount, delGen);
return infoPerCommit;
}
private SegmentInfo readUpgradedSegmentInfo(String name, Directory dir, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene3xSegmentInfoFormat.UPGRADED_SI_CODEC_NAME,
Lucene3xSegmentInfoFormat.UPGRADED_SI_VERSION_START,
Lucene3xSegmentInfoFormat.UPGRADED_SI_VERSION_CURRENT);
final String version = input.readString();
final int docCount = input.readInt();
final Map<String,String> attributes = input.readStringStringMap();
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readStringStringMap();
final Set<String> files = input.readStringSet();
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
null, diagnostics, Collections.unmodifiableMap(attributes));
info.setFiles(files);
return info;
}
}

View File

@ -1,117 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.MultiLevelSkipListReader;
import org.apache.lucene.store.IndexInput;
/**
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
*/
@Deprecated
final class Lucene3xSkipListReader extends MultiLevelSkipListReader {
private boolean currentFieldStoresPayloads;
private long freqPointer[];
private long proxPointer[];
private int payloadLength[];
private long lastFreqPointer;
private long lastProxPointer;
private int lastPayloadLength;
public Lucene3xSkipListReader(IndexInput skipStream, int maxSkipLevels, int skipInterval) {
super(skipStream, maxSkipLevels, skipInterval);
freqPointer = new long[maxSkipLevels];
proxPointer = new long[maxSkipLevels];
payloadLength = new int[maxSkipLevels];
}
public void init(long skipPointer, long freqBasePointer, long proxBasePointer, int df, boolean storesPayloads) {
super.init(skipPointer, df);
this.currentFieldStoresPayloads = storesPayloads;
lastFreqPointer = freqBasePointer;
lastProxPointer = proxBasePointer;
Arrays.fill(freqPointer, freqBasePointer);
Arrays.fill(proxPointer, proxBasePointer);
Arrays.fill(payloadLength, 0);
}
/** Returns the freq pointer of the doc to which the last call of
* {@link MultiLevelSkipListReader#skipTo(int)} has skipped. */
public long getFreqPointer() {
return lastFreqPointer;
}
/** Returns the prox pointer of the doc to which the last call of
* {@link MultiLevelSkipListReader#skipTo(int)} has skipped. */
public long getProxPointer() {
return lastProxPointer;
}
/** Returns the payload length of the payload stored just before
* the doc to which the last call of {@link MultiLevelSkipListReader#skipTo(int)}
* has skipped. */
public int getPayloadLength() {
return lastPayloadLength;
}
@Override
protected void seekChild(int level) throws IOException {
super.seekChild(level);
freqPointer[level] = lastFreqPointer;
proxPointer[level] = lastProxPointer;
payloadLength[level] = lastPayloadLength;
}
@Override
protected void setLastSkipData(int level) {
super.setLastSkipData(level);
lastFreqPointer = freqPointer[level];
lastProxPointer = proxPointer[level];
lastPayloadLength = payloadLength[level];
}
@Override
protected int readSkipData(int level, IndexInput skipStream) throws IOException {
int delta;
if (currentFieldStoresPayloads) {
// the current field stores payloads.
// if the doc delta is odd then we have
// to read the current payload length
// because it differs from the length of the
// previous payload
delta = skipStream.readVInt();
if ((delta & 1) != 0) {
payloadLength[level] = skipStream.readVInt();
}
delta >>>= 1;
} else {
delta = skipStream.readVInt();
}
freqPointer[level] += skipStream.readVInt();
proxPointer[level] += skipStream.readVInt();
return delta;
}
}

View File

@ -1,45 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** @deprecated */
@Deprecated
class Lucene3xStoredFieldsFormat extends StoredFieldsFormat {
@Override
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si,
FieldInfos fn, IOContext context) throws IOException {
return new Lucene3xStoredFieldsReader(directory, si, fn, context);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -1,298 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.StoredFieldsReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils;
import java.io.Closeable;
/**
* Class responsible for access to stored document fields.
* <p/>
* It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
*
* @deprecated
*/
@Deprecated
final class Lucene3xStoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
private final static int FORMAT_SIZE = 4;
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
/** Extension of stored fields index file */
public static final String FIELDS_INDEX_EXTENSION = "fdx";
// Lucene 3.0: Removal of compressed fields
static final int FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS = 2;
// Lucene 3.2: NumericFields are stored in binary format
static final int FORMAT_LUCENE_3_2_NUMERIC_FIELDS = 3;
// NOTE: if you introduce a new format, make it 1 higher
// than the current one, and always change this if you
// switch to a new format!
public static final int FORMAT_CURRENT = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
// when removing support for old versions, leave the last supported version here
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
// NOTE: bit 0 is free here! You can steal it!
public static final int FIELD_IS_BINARY = 1 << 1;
// the old bit 1 << 2 was compressed, is now left out
private static final int _NUMERIC_BIT_SHIFT = 3;
static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
public static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
public static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
public static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
public static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
private final FieldInfos fieldInfos;
private final IndexInput fieldsStream;
private final IndexInput indexStream;
private int numTotalDocs;
private int size;
private boolean closed;
private final int format;
// The docID offset where our docs begin in the index
// file. This will be 0 if we have our own private file.
private int docStoreOffset;
// when we are inside a compound share doc store (CFX),
// (lucene 3.0 indexes only), we privately open our own fd.
private final CompoundFileDirectory storeCFSReader;
/** Returns a cloned FieldsReader that shares open
* IndexInputs with the original one. It is the caller's
* job not to close the original FieldsReader until all
* clones are called (eg, currently SegmentReader manages
* this logic). */
@Override
public Lucene3xStoredFieldsReader clone() {
ensureOpen();
return new Lucene3xStoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
}
/** Verifies that the code version which wrote the segment is supported. */
public static void checkCodeVersion(Directory dir, String segment) throws IOException {
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION);
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
try {
int format = idxStream.readInt();
if (format < FORMAT_MINIMUM)
throw new IndexFormatTooOldException(idxStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
if (format > FORMAT_CURRENT)
throw new IndexFormatTooNewException(idxStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
} finally {
idxStream.close();
}
}
// Used only by clone
private Lucene3xStoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
IndexInput fieldsStream, IndexInput indexStream) {
this.fieldInfos = fieldInfos;
this.numTotalDocs = numTotalDocs;
this.size = size;
this.format = format;
this.docStoreOffset = docStoreOffset;
this.fieldsStream = fieldsStream;
this.indexStream = indexStream;
this.storeCFSReader = null;
}
public Lucene3xStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
final String segment = Lucene3xSegmentInfoFormat.getDocStoreSegment(si);
final int docStoreOffset = Lucene3xSegmentInfoFormat.getDocStoreOffset(si);
final int size = si.getDocCount();
boolean success = false;
fieldInfos = fn;
try {
if (docStoreOffset != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
d = storeCFSReader = new CompoundFileDirectory(si.dir,
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
} else {
storeCFSReader = null;
}
fieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION);
indexStream = d.openInput(indexStreamFN, context);
format = indexStream.readInt();
if (format < FORMAT_MINIMUM)
throw new IndexFormatTooOldException(indexStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
if (format > FORMAT_CURRENT)
throw new IndexFormatTooNewException(indexStream, format, FORMAT_MINIMUM, FORMAT_CURRENT);
final long indexSize = indexStream.length() - FORMAT_SIZE;
if (docStoreOffset != -1) {
// We read only a slice out of this shared fields file
this.docStoreOffset = docStoreOffset;
this.size = size;
// Verify the file is long enough to hold all of our
// docs
assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
} else {
this.docStoreOffset = 0;
this.size = (int) (indexSize >> 3);
// Verify two sources of "maxDoc" agree:
if (this.size != si.getDocCount()) {
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.getDocCount());
}
}
numTotalDocs = (int) (indexSize >> 3);
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
// fine) to hit a FileNotFound exception above. In
// this case, we want to explicitly close any subset
// of things that were opened so that we don't have to
// wait for a GC to do so.
if (!success) {
close();
}
}
}
/**
* @throws AlreadyClosedException if this FieldsReader is closed
*/
private void ensureOpen() throws AlreadyClosedException {
if (closed) {
throw new AlreadyClosedException("this FieldsReader is closed");
}
}
/**
* Closes the underlying {@link org.apache.lucene.store.IndexInput} streams.
* This means that the Fields values will not be accessible.
*
* @throws IOException
*/
public final void close() throws IOException {
if (!closed) {
IOUtils.close(fieldsStream, indexStream, storeCFSReader);
closed = true;
}
}
private void seekIndex(int docID) throws IOException {
indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
}
public final void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
seekIndex(n);
fieldsStream.seek(indexStream.readLong());
final int numFields = fieldsStream.readVInt();
for (int fieldIDX = 0; fieldIDX < numFields; fieldIDX++) {
int fieldNumber = fieldsStream.readVInt();
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
int bits = fieldsStream.readByte() & 0xFF;
assert bits <= (FIELD_IS_NUMERIC_MASK | FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
switch(visitor.needsField(fieldInfo)) {
case YES:
readField(visitor, fieldInfo, bits);
break;
case NO:
skipField(bits);
break;
case STOP:
return;
}
}
}
private void readField(StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
final int numeric = bits & FIELD_IS_NUMERIC_MASK;
if (numeric != 0) {
switch(numeric) {
case FIELD_IS_NUMERIC_INT:
visitor.intField(info, fieldsStream.readInt());
return;
case FIELD_IS_NUMERIC_LONG:
visitor.longField(info, fieldsStream.readLong());
return;
case FIELD_IS_NUMERIC_FLOAT:
visitor.floatField(info, Float.intBitsToFloat(fieldsStream.readInt()));
return;
case FIELD_IS_NUMERIC_DOUBLE:
visitor.doubleField(info, Double.longBitsToDouble(fieldsStream.readLong()));
return;
default:
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
}
} else {
final int length = fieldsStream.readVInt();
byte bytes[] = new byte[length];
fieldsStream.readBytes(bytes, 0, length);
if ((bits & FIELD_IS_BINARY) != 0) {
visitor.binaryField(info, bytes, 0, bytes.length);
} else {
visitor.stringField(info, new String(bytes, 0, bytes.length, IOUtils.CHARSET_UTF_8));
}
}
}
private void skipField(int bits) throws IOException {
final int numeric = bits & FIELD_IS_NUMERIC_MASK;
if (numeric != 0) {
switch(numeric) {
case FIELD_IS_NUMERIC_INT:
case FIELD_IS_NUMERIC_FLOAT:
fieldsStream.readInt();
return;
case FIELD_IS_NUMERIC_LONG:
case FIELD_IS_NUMERIC_DOUBLE:
fieldsStream.readLong();
return;
default:
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
}
} else {
final int length = fieldsStream.readVInt();
fieldsStream.seek(fieldsStream.getFilePointer() + length);
}
}
}

View File

@ -1,80 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene3x ReadOnly TermVectorsFormat implementation
* @deprecated (4.0) This is only used to read indexes created
* before 4.0.
* @lucene.experimental
*/
@Deprecated
class Lucene3xTermVectorsFormat extends TermVectorsFormat {
@Override
public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(Lucene3xSegmentInfoFormat.getDocStoreSegment(segmentInfo), "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION);
// Unfortunately, for 3.x indices, each segment's
// FieldInfos can lie about hasVectors (claim it's true
// when really it's false).... so we have to carefully
// check if the files really exist before trying to open
// them (4.x has fixed this):
final boolean exists;
if (Lucene3xSegmentInfoFormat.getDocStoreOffset(segmentInfo) != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(segmentInfo)) {
String cfxFileName = IndexFileNames.segmentFileName(Lucene3xSegmentInfoFormat.getDocStoreSegment(segmentInfo), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION);
if (segmentInfo.dir.fileExists(cfxFileName)) {
Directory cfsDir = new CompoundFileDirectory(segmentInfo.dir, cfxFileName, context, false);
try {
exists = cfsDir.fileExists(fileName);
} finally {
cfsDir.close();
}
} else {
exists = false;
}
} else {
exists = directory.fileExists(fileName);
}
if (!exists) {
// 3x's FieldInfos sometimes lies and claims a segment
// has vectors when it doesn't:
return null;
} else {
return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context);
}
}
@Override
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -1,701 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** @deprecated */
@Deprecated
class Lucene3xTermVectorsReader extends TermVectorsReader {
// NOTE: if you make a new format, it must be larger than
// the current format
// Changed strings to UTF8 with length-in-bytes not length-in-chars
static final int FORMAT_UTF8_LENGTH_IN_BYTES = 4;
// NOTE: always change this if you switch to a new format!
// whenever you add a new format, make it 1 larger (positive version logic)!
public static final int FORMAT_CURRENT = FORMAT_UTF8_LENGTH_IN_BYTES;
// when removing support for old versions, leave the last supported version here
public static final int FORMAT_MINIMUM = FORMAT_UTF8_LENGTH_IN_BYTES;
//The size in bytes that the FORMAT_VERSION will take up at the beginning of each file
static final int FORMAT_SIZE = 4;
public static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x1;
public static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
/** Extension of vectors fields file */
public static final String VECTORS_FIELDS_EXTENSION = "tvf";
/** Extension of vectors documents file */
public static final String VECTORS_DOCUMENTS_EXTENSION = "tvd";
/** Extension of vectors index file */
public static final String VECTORS_INDEX_EXTENSION = "tvx";
private FieldInfos fieldInfos;
private IndexInput tvx;
private IndexInput tvd;
private IndexInput tvf;
private int size;
private int numTotalDocs;
// The docID offset where our docs begin in the index
// file. This will be 0 if we have our own private file.
private int docStoreOffset;
// when we are inside a compound share doc store (CFX),
// (lucene 3.0 indexes only), we privately open our own fd.
// TODO: if we are worried, maybe we could eliminate the
// extra fd somehow when you also have vectors...
private final CompoundFileDirectory storeCFSReader;
private final int format;
// used by clone
Lucene3xTermVectorsReader(FieldInfos fieldInfos, IndexInput tvx, IndexInput tvd, IndexInput tvf, int size, int numTotalDocs, int docStoreOffset, int format) {
this.fieldInfos = fieldInfos;
this.tvx = tvx;
this.tvd = tvd;
this.tvf = tvf;
this.size = size;
this.numTotalDocs = numTotalDocs;
this.docStoreOffset = docStoreOffset;
this.format = format;
this.storeCFSReader = null;
}
public Lucene3xTermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
throws CorruptIndexException, IOException {
final String segment = Lucene3xSegmentInfoFormat.getDocStoreSegment(si);
final int docStoreOffset = Lucene3xSegmentInfoFormat.getDocStoreOffset(si);
final int size = si.getDocCount();
boolean success = false;
try {
if (docStoreOffset != -1 && Lucene3xSegmentInfoFormat.getDocStoreIsCompoundFile(si)) {
d = storeCFSReader = new CompoundFileDirectory(si.dir,
IndexFileNames.segmentFileName(segment, "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION), context, false);
} else {
storeCFSReader = null;
}
String idxName = IndexFileNames.segmentFileName(segment, "", VECTORS_INDEX_EXTENSION);
tvx = d.openInput(idxName, context);
format = checkValidFormat(tvx);
String fn = IndexFileNames.segmentFileName(segment, "", VECTORS_DOCUMENTS_EXTENSION);
tvd = d.openInput(fn, context);
final int tvdFormat = checkValidFormat(tvd);
fn = IndexFileNames.segmentFileName(segment, "", VECTORS_FIELDS_EXTENSION);
tvf = d.openInput(fn, context);
final int tvfFormat = checkValidFormat(tvf);
assert format == tvdFormat;
assert format == tvfFormat;
numTotalDocs = (int) (tvx.length() >> 4);
if (-1 == docStoreOffset) {
this.docStoreOffset = 0;
this.size = numTotalDocs;
assert size == 0 || numTotalDocs == size;
} else {
this.docStoreOffset = docStoreOffset;
this.size = size;
// Verify the file is long enough to hold all of our
// docs
assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
}
this.fieldInfos = fieldInfos;
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
// fine) to hit a FileNotFound exception above. In
// this case, we want to explicitly close any subset
// of things that were opened so that we don't have to
// wait for a GC to do so.
if (!success) {
close();
}
}
}
// Not private to avoid synthetic access$NNN methods
void seekTvx(final int docNum) throws IOException {
tvx.seek((docNum + docStoreOffset) * 16L + FORMAT_SIZE);
}
private int checkValidFormat(IndexInput in) throws CorruptIndexException, IOException
{
int format = in.readInt();
if (format < FORMAT_MINIMUM)
throw new IndexFormatTooOldException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
if (format > FORMAT_CURRENT)
throw new IndexFormatTooNewException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
return format;
}
public void close() throws IOException {
IOUtils.close(tvx, tvd, tvf, storeCFSReader);
}
/**
*
* @return The number of documents in the reader
*/
int size() {
return size;
}
private class TVFields extends Fields {
private final int[] fieldNumbers;
private final long[] fieldFPs;
private final Map<Integer,Integer> fieldNumberToIndex = new HashMap<Integer,Integer>();
public TVFields(int docID) throws IOException {
seekTvx(docID);
tvd.seek(tvx.readLong());
final int fieldCount = tvd.readVInt();
assert fieldCount >= 0;
if (fieldCount != 0) {
fieldNumbers = new int[fieldCount];
fieldFPs = new long[fieldCount];
for(int fieldUpto=0;fieldUpto<fieldCount;fieldUpto++) {
final int fieldNumber = tvd.readVInt();
fieldNumbers[fieldUpto] = fieldNumber;
fieldNumberToIndex.put(fieldNumber, fieldUpto);
}
long position = tvx.readLong();
fieldFPs[0] = position;
for(int fieldUpto=1;fieldUpto<fieldCount;fieldUpto++) {
position += tvd.readVLong();
fieldFPs[fieldUpto] = position;
}
} else {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that
// this doc has no TVs
fieldNumbers = null;
fieldFPs = null;
}
}
@Override
public FieldsEnum iterator() throws IOException {
return new FieldsEnum() {
private int fieldUpto;
@Override
public String next() throws IOException {
if (fieldNumbers != null && fieldUpto < fieldNumbers.length) {
return fieldInfos.fieldInfo(fieldNumbers[fieldUpto++]).name;
} else {
return null;
}
}
@Override
public Terms terms() throws IOException {
return TVFields.this.terms(fieldInfos.fieldInfo(fieldNumbers[fieldUpto-1]).name);
}
};
}
@Override
public Terms terms(String field) throws IOException {
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
// No such field
return null;
}
final Integer fieldIndex = fieldNumberToIndex.get(fieldInfo.number);
if (fieldIndex == null) {
// Term vectors were not indexed for this field
return null;
}
return new TVTerms(fieldFPs[fieldIndex]);
}
@Override
public int size() {
if (fieldNumbers == null) {
return 0;
} else {
return fieldNumbers.length;
}
}
}
private class TVTerms extends Terms {
private final int numTerms;
private final long tvfFPStart;
private final boolean unicodeSortOrder;
public TVTerms(long tvfFP) throws IOException {
tvf.seek(tvfFP);
numTerms = tvf.readVInt();
tvfFPStart = tvf.getFilePointer();
unicodeSortOrder = sortTermsByUnicode();
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
TVTermsEnum termsEnum;
if (reuse instanceof TVTermsEnum) {
termsEnum = (TVTermsEnum) reuse;
if (!termsEnum.canReuse(tvf)) {
termsEnum = new TVTermsEnum();
}
} else {
termsEnum = new TVTermsEnum();
}
termsEnum.reset(numTerms, tvfFPStart, unicodeSortOrder);
return termsEnum;
}
@Override
public long size() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return -1;
}
@Override
public long getSumDocFreq() {
// Every term occurs in just one doc:
return numTerms;
}
@Override
public int getDocCount() {
return 1;
}
@Override
public Comparator<BytesRef> getComparator() {
if (unicodeSortOrder) {
return BytesRef.getUTF8SortedAsUnicodeComparator();
} else {
return BytesRef.getUTF8SortedAsUTF16Comparator();
}
}
}
static class TermAndPostings {
BytesRef term;
int freq;
int[] positions;
int[] startOffsets;
int[] endOffsets;
}
private class TVTermsEnum extends TermsEnum {
private boolean unicodeSortOrder;
private final IndexInput origTVF;
private final IndexInput tvf;
private int numTerms;
private int currentTerm;
private boolean storePositions;
private boolean storeOffsets;
private TermAndPostings[] termAndPostings;
// NOTE: tvf is pre-positioned by caller
public TVTermsEnum() throws IOException {
this.origTVF = Lucene3xTermVectorsReader.this.tvf;
tvf = (IndexInput) origTVF.clone();
}
public boolean canReuse(IndexInput tvf) {
return tvf == origTVF;
}
public void reset(int numTerms, long tvfFPStart, boolean unicodeSortOrder) throws IOException {
this.numTerms = numTerms;
currentTerm = -1;
tvf.seek(tvfFPStart);
final byte bits = tvf.readByte();
storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
this.unicodeSortOrder = unicodeSortOrder;
readVectors();
if (unicodeSortOrder) {
Arrays.sort(termAndPostings, new Comparator<TermAndPostings>() {
public int compare(TermAndPostings left, TermAndPostings right) {
return left.term.compareTo(right.term);
}
});
}
}
private void readVectors() throws IOException {
termAndPostings = new TermAndPostings[numTerms];
BytesRef lastTerm = new BytesRef();
for (int i = 0; i < numTerms; i++) {
TermAndPostings t = new TermAndPostings();
BytesRef term = new BytesRef();
term.copyBytes(lastTerm);
final int start = tvf.readVInt();
final int deltaLen = tvf.readVInt();
term.length = start + deltaLen;
term.grow(term.length);
tvf.readBytes(term.bytes, start, deltaLen);
t.term = term;
int freq = tvf.readVInt();
t.freq = freq;
if (storePositions) {
int positions[] = new int[freq];
int pos = 0;
for(int posUpto=0;posUpto<freq;posUpto++) {
pos += tvf.readVInt();
positions[posUpto] = pos;
}
t.positions = positions;
}
if (storeOffsets) {
int startOffsets[] = new int[freq];
int endOffsets[] = new int[freq];
int offset = 0;
for(int posUpto=0;posUpto<freq;posUpto++) {
startOffsets[posUpto] = offset + tvf.readVInt();
offset = endOffsets[posUpto] = startOffsets[posUpto] + tvf.readVInt();
}
t.startOffsets = startOffsets;
t.endOffsets = endOffsets;
}
lastTerm.copyBytes(term);
termAndPostings[i] = t;
}
}
// NOTE: slow! (linear scan)
@Override
public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
Comparator<BytesRef> comparator = getComparator();
for (int i = 0; i < numTerms; i++) {
int cmp = comparator.compare(text, termAndPostings[i].term);
if (cmp < 0) {
currentTerm = i;
return SeekStatus.NOT_FOUND;
} else if (cmp == 0) {
currentTerm = i;
return SeekStatus.FOUND;
}
}
currentTerm = termAndPostings.length;
return SeekStatus.END;
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() throws IOException {
if (++currentTerm >= numTerms) {
return null;
}
return term();
}
@Override
public BytesRef term() {
return termAndPostings[currentTerm].term;
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
@Override
public int docFreq() {
return 1;
}
@Override
public long totalTermFreq() {
return termAndPostings[currentTerm].freq;
}
@Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs /* ignored */) throws IOException {
TVDocsEnum docsEnum;
if (reuse != null && reuse instanceof TVDocsEnum) {
docsEnum = (TVDocsEnum) reuse;
} else {
docsEnum = new TVDocsEnum();
}
docsEnum.reset(liveDocs, termAndPostings[currentTerm]);
return docsEnum;
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
if (needsOffsets && !storeOffsets) {
return null;
}
if (!storePositions && !storeOffsets) {
return null;
}
TVDocsAndPositionsEnum docsAndPositionsEnum;
if (reuse != null && reuse instanceof TVDocsAndPositionsEnum) {
docsAndPositionsEnum = (TVDocsAndPositionsEnum) reuse;
} else {
docsAndPositionsEnum = new TVDocsAndPositionsEnum();
}
docsAndPositionsEnum.reset(liveDocs, termAndPostings[currentTerm]);
return docsAndPositionsEnum;
}
@Override
public Comparator<BytesRef> getComparator() {
if (unicodeSortOrder) {
return BytesRef.getUTF8SortedAsUnicodeComparator();
} else {
return BytesRef.getUTF8SortedAsUTF16Comparator();
}
}
}
// NOTE: sort of a silly class, since you can get the
// freq() already by TermsEnum.totalTermFreq
private static class TVDocsEnum extends DocsEnum {
private boolean didNext;
private int doc = -1;
private int freq;
private Bits liveDocs;
@Override
public int freq() throws IOException {
return freq;
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() {
if (!didNext && (liveDocs == null || liveDocs.get(0))) {
didNext = true;
return (doc = 0);
} else {
return (doc = NO_MORE_DOCS);
}
}
@Override
public int advance(int target) {
if (!didNext && target == 0) {
return nextDoc();
} else {
return (doc = NO_MORE_DOCS);
}
}
public void reset(Bits liveDocs, TermAndPostings termAndPostings) {
this.liveDocs = liveDocs;
this.freq = termAndPostings.freq;
this.doc = -1;
didNext = false;
}
}
private static class TVDocsAndPositionsEnum extends DocsAndPositionsEnum {
private boolean didNext;
private int doc = -1;
private int nextPos;
private Bits liveDocs;
private int[] positions;
private int[] startOffsets;
private int[] endOffsets;
@Override
public int freq() throws IOException {
if (positions != null) {
return positions.length;
} else {
assert startOffsets != null;
return startOffsets.length;
}
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() {
if (!didNext && (liveDocs == null || liveDocs.get(0))) {
didNext = true;
return (doc = 0);
} else {
return (doc = NO_MORE_DOCS);
}
}
@Override
public int advance(int target) {
if (!didNext && target == 0) {
return nextDoc();
} else {
return (doc = NO_MORE_DOCS);
}
}
public void reset(Bits liveDocs, TermAndPostings termAndPostings) {
this.liveDocs = liveDocs;
this.positions = termAndPostings.positions;
this.startOffsets = termAndPostings.startOffsets;
this.endOffsets = termAndPostings.endOffsets;
this.doc = -1;
didNext = false;
nextPos = 0;
}
@Override
public BytesRef getPayload() {
return null;
}
@Override
public boolean hasPayload() {
return false;
}
@Override
public int nextPosition() {
assert (positions != null && nextPos < positions.length) ||
startOffsets != null && nextPos < startOffsets.length;
if (positions != null) {
return positions[nextPos++];
} else {
nextPos++;
return -1;
}
}
@Override
public int startOffset() {
assert startOffsets != null;
return startOffsets[nextPos-1];
}
@Override
public int endOffset() {
assert endOffsets != null;
return endOffsets[nextPos-1];
}
}
@Override
public Fields get(int docID) throws IOException {
if (docID < 0 || docID >= numTotalDocs) {
throw new IllegalArgumentException("doID=" + docID + " is out of bounds [0.." + (numTotalDocs-1) + "]");
}
if (tvx != null) {
Fields fields = new TVFields(docID);
if (fields.size() == 0) {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that
// this doc has no TVs
return null;
} else {
return fields;
}
} else {
return null;
}
}
@Override
public TermVectorsReader clone() {
IndexInput cloneTvx = null;
IndexInput cloneTvd = null;
IndexInput cloneTvf = null;
// These are null when a TermVectorsReader was created
// on a segment that did not have term vectors saved
if (tvx != null && tvd != null && tvf != null) {
cloneTvx = (IndexInput) tvx.clone();
cloneTvd = (IndexInput) tvd.clone();
cloneTvf = (IndexInput) tvf.clone();
}
return new Lucene3xTermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
}
// If this returns, we do the surrogates shuffle so that the
// terms are sorted by unicode sort order. This should be
// true when segments are used for "normal" searching;
// it's only false during testing, to create a pre-flex
// index, using the test-only PreFlexRW.
protected boolean sortTermsByUnicode() {
return true;
}
}

View File

@ -1,229 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
/** @deprecated (4.0)
* @lucene.experimental */
@Deprecated
class SegmentTermDocs {
//protected SegmentReader parent;
private final FieldInfos fieldInfos;
private final TermInfosReader tis;
protected Bits liveDocs;
protected IndexInput freqStream;
protected int count;
protected int df;
int doc = 0;
int freq;
private int skipInterval;
private int maxSkipLevels;
private Lucene3xSkipListReader skipListReader;
private long freqBasePointer;
private long proxBasePointer;
private long skipPointer;
private boolean haveSkipped;
protected boolean currentFieldStoresPayloads;
protected IndexOptions indexOptions;
public SegmentTermDocs(IndexInput freqStream, TermInfosReader tis, FieldInfos fieldInfos) {
this.freqStream = (IndexInput) freqStream.clone();
this.tis = tis;
this.fieldInfos = fieldInfos;
skipInterval = tis.getSkipInterval();
maxSkipLevels = tis.getMaxSkipLevels();
}
public void seek(Term term) throws IOException {
TermInfo ti = tis.get(term);
seek(ti, term);
}
public void setLiveDocs(Bits liveDocs) {
this.liveDocs = liveDocs;
}
public void seek(SegmentTermEnum segmentTermEnum) throws IOException {
TermInfo ti;
Term term;
// use comparison of fieldinfos to verify that termEnum belongs to the same segment as this SegmentTermDocs
if (segmentTermEnum.fieldInfos == fieldInfos) { // optimized case
term = segmentTermEnum.term();
ti = segmentTermEnum.termInfo();
} else { // punt case
term = segmentTermEnum.term();
ti = tis.get(term);
}
seek(ti, term);
}
void seek(TermInfo ti, Term term) throws IOException {
count = 0;
FieldInfo fi = fieldInfos.fieldInfo(term.field());
this.indexOptions = (fi != null) ? fi.getIndexOptions() : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
currentFieldStoresPayloads = (fi != null) ? fi.hasPayloads() : false;
if (ti == null) {
df = 0;
} else {
df = ti.docFreq;
doc = 0;
freqBasePointer = ti.freqPointer;
proxBasePointer = ti.proxPointer;
skipPointer = freqBasePointer + ti.skipOffset;
freqStream.seek(freqBasePointer);
haveSkipped = false;
}
}
public void close() throws IOException {
freqStream.close();
if (skipListReader != null)
skipListReader.close();
}
public final int doc() { return doc; }
public final int freq() {
assert indexOptions != IndexOptions.DOCS_ONLY;
return freq;
}
protected void skippingDoc() throws IOException {
}
public boolean next() throws IOException {
while (true) {
if (count == df)
return false;
final int docCode = freqStream.readVInt();
if (indexOptions == IndexOptions.DOCS_ONLY) {
doc += docCode;
} else {
doc += docCode >>> 1; // shift off low bit
if ((docCode & 1) != 0) // if low bit is set
freq = 1; // freq is one
else {
freq = freqStream.readVInt(); // else read freq
assert freq != 1;
}
}
count++;
if (liveDocs == null || liveDocs.get(doc)) {
break;
}
skippingDoc();
}
return true;
}
/** Optimized implementation. */
public int read(final int[] docs, final int[] freqs)
throws IOException {
final int length = docs.length;
if (indexOptions == IndexOptions.DOCS_ONLY) {
return readNoTf(docs, freqs, length);
} else {
int i = 0;
while (i < length && count < df) {
// manually inlined call to next() for speed
final int docCode = freqStream.readVInt();
doc += docCode >>> 1; // shift off low bit
if ((docCode & 1) != 0) // if low bit is set
freq = 1; // freq is one
else
freq = freqStream.readVInt(); // else read freq
count++;
if (liveDocs == null || liveDocs.get(doc)) {
docs[i] = doc;
freqs[i] = freq;
++i;
}
}
return i;
}
}
private final int readNoTf(final int[] docs, final int[] freqs, final int length) throws IOException {
int i = 0;
while (i < length && count < df) {
// manually inlined call to next() for speed
doc += freqStream.readVInt();
count++;
if (liveDocs == null || liveDocs.get(doc)) {
docs[i] = doc;
// Hardware freq to 1 when term freqs were not
// stored in the index
freqs[i] = 1;
++i;
}
}
return i;
}
/** Overridden by SegmentTermPositions to skip in prox stream. */
protected void skipProx(long proxPointer, int payloadLength) throws IOException {}
/** Optimized implementation. */
public boolean skipTo(int target) throws IOException {
// don't skip if the target is close (within skipInterval docs away)
if ((target - skipInterval) >= doc && df >= skipInterval) { // optimized case
if (skipListReader == null)
skipListReader = new Lucene3xSkipListReader((IndexInput) freqStream.clone(), maxSkipLevels, skipInterval); // lazily clone
if (!haveSkipped) { // lazily initialize skip stream
skipListReader.init(skipPointer, freqBasePointer, proxBasePointer, df, currentFieldStoresPayloads);
haveSkipped = true;
}
int newCount = skipListReader.skipTo(target);
if (newCount > count) {
freqStream.seek(skipListReader.getFreqPointer());
skipProx(skipListReader.getProxPointer(), skipListReader.getPayloadLength());
doc = skipListReader.getDoc();
count = newCount;
}
}
// done skipping, now just scan
do {
if (!next())
return false;
} while (target > doc);
return true;
}
}

View File

@ -1,225 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.IndexFormatTooNewException;
/**
* @deprecated (4.0) No longer used with flex indexing, except for
* reading old segments
* @lucene.experimental */
@Deprecated
final class SegmentTermEnum implements Cloneable {
private IndexInput input;
FieldInfos fieldInfos;
long size;
long position = -1;
// Changed strings to true utf8 with length-in-bytes not
// length-in-chars
public static final int FORMAT_VERSION_UTF8_LENGTH_IN_BYTES = -4;
// NOTE: always change this if you switch to a new format!
// whenever you add a new format, make it 1 smaller (negative version logic)!
public static final int FORMAT_CURRENT = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
// when removing support for old versions, leave the last supported version here
public static final int FORMAT_MINIMUM = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
private TermBuffer termBuffer = new TermBuffer();
private TermBuffer prevBuffer = new TermBuffer();
private TermBuffer scanBuffer = new TermBuffer(); // used for scanning
TermInfo termInfo = new TermInfo();
private int format;
private boolean isIndex = false;
long indexPointer = 0;
int indexInterval;
int skipInterval;
int newSuffixStart;
int maxSkipLevels;
private boolean first = true;
SegmentTermEnum(IndexInput i, FieldInfos fis, boolean isi)
throws CorruptIndexException, IOException {
input = i;
fieldInfos = fis;
isIndex = isi;
maxSkipLevels = 1; // use single-level skip lists for formats > -3
int firstInt = input.readInt();
if (firstInt >= 0) {
// original-format file, without explicit format version number
format = 0;
size = firstInt;
// back-compatible settings
indexInterval = 128;
skipInterval = Integer.MAX_VALUE; // switch off skipTo optimization
} else {
// we have a format version number
format = firstInt;
// check that it is a format we can understand
if (format > FORMAT_MINIMUM)
throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
if (format < FORMAT_CURRENT)
throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
size = input.readLong(); // read the size
indexInterval = input.readInt();
skipInterval = input.readInt();
maxSkipLevels = input.readInt();
assert indexInterval > 0: "indexInterval=" + indexInterval + " is negative; must be > 0";
assert skipInterval > 0: "skipInterval=" + skipInterval + " is negative; must be > 0";
}
}
@Override
protected SegmentTermEnum clone() {
SegmentTermEnum clone = null;
try {
clone = (SegmentTermEnum) super.clone();
} catch (CloneNotSupportedException e) {}
clone.input = (IndexInput) input.clone();
clone.termInfo = new TermInfo(termInfo);
clone.termBuffer = termBuffer.clone();
clone.prevBuffer = prevBuffer.clone();
clone.scanBuffer = new TermBuffer();
return clone;
}
final void seek(long pointer, long p, Term t, TermInfo ti)
throws IOException {
input.seek(pointer);
position = p;
termBuffer.set(t);
prevBuffer.reset();
//System.out.println(" ste doSeek prev=" + prevBuffer.toTerm() + " this=" + this);
termInfo.set(ti);
first = p == -1;
}
/** Increments the enumeration to the next element. True if one exists.*/
public final boolean next() throws IOException {
prevBuffer.set(termBuffer);
//System.out.println(" ste setPrev=" + prev() + " this=" + this);
if (position++ >= size - 1) {
termBuffer.reset();
//System.out.println(" EOF");
return false;
}
termBuffer.read(input, fieldInfos);
newSuffixStart = termBuffer.newSuffixStart;
termInfo.docFreq = input.readVInt(); // read doc freq
termInfo.freqPointer += input.readVLong(); // read freq pointer
termInfo.proxPointer += input.readVLong(); // read prox pointer
if (termInfo.docFreq >= skipInterval)
termInfo.skipOffset = input.readVInt();
if (isIndex)
indexPointer += input.readVLong(); // read index pointer
//System.out.println(" ste ret term=" + term());
return true;
}
/* Optimized scan, without allocating new terms.
* Return number of invocations to next().
*
* NOTE: LUCENE-3183: if you pass Term("", "") here then this
* will incorrectly return before positioning the enum,
* and position will be -1; caller must detect this. */
final int scanTo(Term term) throws IOException {
scanBuffer.set(term);
int count = 0;
if (first) {
// Always force initial next() in case term is
// Term("", "")
next();
first = false;
count++;
}
while (scanBuffer.compareTo(termBuffer) > 0 && next()) {
count++;
}
return count;
}
/** Returns the current Term in the enumeration.
Initially invalid, valid after next() called for the first time.*/
public final Term term() {
return termBuffer.toTerm();
}
/** Returns the previous Term enumerated. Initially null.*/
final Term prev() {
return prevBuffer.toTerm();
}
/** Returns the current TermInfo in the enumeration.
Initially invalid, valid after next() called for the first time.*/
final TermInfo termInfo() {
return new TermInfo(termInfo);
}
/** Sets the argument to the current TermInfo in the enumeration.
Initially invalid, valid after next() called for the first time.*/
final void termInfo(TermInfo ti) {
ti.set(termInfo);
}
/** Returns the docFreq from the current TermInfo in the enumeration.
Initially invalid, valid after next() called for the first time.*/
public final int docFreq() {
return termInfo.docFreq;
}
/* Returns the freqPointer from the current TermInfo in the enumeration.
Initially invalid, valid after next() called for the first time.*/
final long freqPointer() {
return termInfo.freqPointer;
}
/* Returns the proxPointer from the current TermInfo in the enumeration.
Initially invalid, valid after next() called for the first time.*/
final long proxPointer() {
return termInfo.proxPointer;
}
/** Closes the enumeration to further activity, freeing resources. */
public final void close() throws IOException {
input.close();
}
}

View File

@ -1,219 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.IndexInput;
/**
* @lucene.experimental
* @deprecated (4.0)
*/
@Deprecated
final class SegmentTermPositions
extends SegmentTermDocs {
private IndexInput proxStream;
private IndexInput proxStreamOrig;
private int proxCount;
private int position;
// the current payload length
private int payloadLength;
// indicates whether the payload of the current position has
// been read from the proxStream yet
private boolean needToLoadPayload;
// these variables are being used to remember information
// for a lazy skip
private long lazySkipPointer = -1;
private int lazySkipProxCount = 0;
/*
SegmentTermPositions(SegmentReader p) {
super(p);
this.proxStream = null; // the proxStream will be cloned lazily when nextPosition() is called for the first time
}
*/
public SegmentTermPositions(IndexInput freqStream, IndexInput proxStream, TermInfosReader tis, FieldInfos fieldInfos) {
super(freqStream, tis, fieldInfos);
this.proxStreamOrig = proxStream; // the proxStream will be cloned lazily when nextPosition() is called for the first time
}
@Override
final void seek(TermInfo ti, Term term) throws IOException {
super.seek(ti, term);
if (ti != null)
lazySkipPointer = ti.proxPointer;
lazySkipProxCount = 0;
proxCount = 0;
payloadLength = 0;
needToLoadPayload = false;
}
@Override
public final void close() throws IOException {
super.close();
if (proxStream != null) proxStream.close();
}
public final int nextPosition() throws IOException {
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
// This field does not store positions, payloads
return 0;
// perform lazy skips if necessary
lazySkip();
proxCount--;
return position += readDeltaPosition();
}
private final int readDeltaPosition() throws IOException {
int delta = proxStream.readVInt();
if (currentFieldStoresPayloads) {
// if the current field stores payloads then
// the position delta is shifted one bit to the left.
// if the LSB is set, then we have to read the current
// payload length
if ((delta & 1) != 0) {
payloadLength = proxStream.readVInt();
}
delta >>>= 1;
needToLoadPayload = true;
}
return delta;
}
@Override
protected final void skippingDoc() throws IOException {
// we remember to skip a document lazily
lazySkipProxCount += freq;
}
@Override
public final boolean next() throws IOException {
// we remember to skip the remaining positions of the current
// document lazily
lazySkipProxCount += proxCount;
if (super.next()) { // run super
proxCount = freq; // note frequency
position = 0; // reset position
return true;
}
return false;
}
@Override
public final int read(final int[] docs, final int[] freqs) {
throw new UnsupportedOperationException("TermPositions does not support processing multiple documents in one call. Use TermDocs instead.");
}
/** Called by super.skipTo(). */
@Override
protected void skipProx(long proxPointer, int payloadLength) throws IOException {
// we save the pointer, we might have to skip there lazily
lazySkipPointer = proxPointer;
lazySkipProxCount = 0;
proxCount = 0;
this.payloadLength = payloadLength;
needToLoadPayload = false;
}
private void skipPositions(int n) throws IOException {
assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
for (int f = n; f > 0; f--) { // skip unread positions
readDeltaPosition();
skipPayload();
}
}
private void skipPayload() throws IOException {
if (needToLoadPayload && payloadLength > 0) {
proxStream.seek(proxStream.getFilePointer() + payloadLength);
}
needToLoadPayload = false;
}
// It is not always necessary to move the prox pointer
// to a new document after the freq pointer has been moved.
// Consider for example a phrase query with two terms:
// the freq pointer for term 1 has to move to document x
// to answer the question if the term occurs in that document. But
// only if term 2 also matches document x, the positions have to be
// read to figure out if term 1 and term 2 appear next
// to each other in document x and thus satisfy the query.
// So we move the prox pointer lazily to the document
// as soon as positions are requested.
private void lazySkip() throws IOException {
if (proxStream == null) {
// clone lazily
proxStream = (IndexInput)proxStreamOrig.clone();
}
// we might have to skip the current payload
// if it was not read yet
skipPayload();
if (lazySkipPointer != -1) {
proxStream.seek(lazySkipPointer);
lazySkipPointer = -1;
}
if (lazySkipProxCount != 0) {
skipPositions(lazySkipProxCount);
lazySkipProxCount = 0;
}
}
public int getPayloadLength() {
return payloadLength;
}
public byte[] getPayload(byte[] data, int offset) throws IOException {
if (!needToLoadPayload) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
}
// read payloads lazily
byte[] retArray;
int retOffset;
if (data == null || data.length - offset < payloadLength) {
// the array is too small to store the payload data,
// so we allocate a new one
retArray = new byte[payloadLength];
retOffset = 0;
} else {
retArray = data;
retOffset = offset;
}
proxStream.readBytes(retArray, retOffset, payloadLength);
needToLoadPayload = false;
return retArray;
}
public boolean isPayloadAvailable() {
return needToLoadPayload && payloadLength > 0;
}
}

View File

@ -1,128 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.Comparator;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.FieldInfos;
/**
* @lucene.experimental
* @deprecated (4.0)
*/
@Deprecated
final class TermBuffer implements Cloneable {
private String field;
private Term term; // cached
private BytesRef bytes = new BytesRef(10);
// Cannot be -1 since (strangely) we write that
// fieldNumber into index for first indexed term:
private int currentFieldNumber = -2;
private static final Comparator<BytesRef> utf8AsUTF16Comparator = BytesRef.getUTF8SortedAsUTF16Comparator();
int newSuffixStart; // only valid right after .read is called
public int compareTo(TermBuffer other) {
if (field == other.field) // fields are interned
// (only by PreFlex codec)
return utf8AsUTF16Comparator.compare(bytes, other.bytes);
else
return field.compareTo(other.field);
}
public void read(IndexInput input, FieldInfos fieldInfos)
throws IOException {
this.term = null; // invalidate cache
newSuffixStart = input.readVInt();
int length = input.readVInt();
int totalLength = newSuffixStart + length;
if (bytes.bytes.length < totalLength) {
bytes.grow(totalLength);
}
bytes.length = totalLength;
input.readBytes(bytes.bytes, newSuffixStart, length);
final int fieldNumber = input.readVInt();
if (fieldNumber != currentFieldNumber) {
currentFieldNumber = fieldNumber;
// NOTE: too much sneakiness here, seriously this is a negative vint?!
if (currentFieldNumber == -1) {
field = "";
} else {
assert fieldInfos.fieldInfo(currentFieldNumber) != null : currentFieldNumber;
field = fieldInfos.fieldInfo(currentFieldNumber).name.intern();
}
} else {
assert field.equals(fieldInfos.fieldInfo(fieldNumber).name) : "currentFieldNumber=" + currentFieldNumber + " field=" + field + " vs " + fieldInfos.fieldInfo(fieldNumber) == null ? "null" : fieldInfos.fieldInfo(fieldNumber).name;
}
}
public void set(Term term) {
if (term == null) {
reset();
return;
}
bytes.copyBytes(term.bytes());
field = term.field().intern();
currentFieldNumber = -1;
this.term = term;
}
public void set(TermBuffer other) {
field = other.field;
currentFieldNumber = other.currentFieldNumber;
// dangerous to copy Term over, since the underlying
// BytesRef could subsequently be modified:
term = null;
bytes.copyBytes(other.bytes);
}
public void reset() {
field = null;
term = null;
currentFieldNumber= -1;
}
public Term toTerm() {
if (field == null) // unset
return null;
if (term == null) {
term = new Term(field, BytesRef.deepCopyOf(bytes));
}
return term;
}
@Override
protected TermBuffer clone() {
TermBuffer clone = null;
try {
clone = (TermBuffer)super.clone();
} catch (CloneNotSupportedException e) {}
clone.bytes = BytesRef.deepCopyOf(bytes);
return clone;
}
}

View File

@ -1,63 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.
*/
/** A TermInfo is the record of information stored for a
* term
* @deprecated (4.0) This class is no longer used in flexible
* indexing. */
@Deprecated
class TermInfo {
/** The number of documents which contain the term. */
public int docFreq = 0;
public long freqPointer = 0;
public long proxPointer = 0;
public int skipOffset;
public TermInfo() {}
public TermInfo(int df, long fp, long pp) {
docFreq = df;
freqPointer = fp;
proxPointer = pp;
}
public TermInfo(TermInfo ti) {
docFreq = ti.docFreq;
freqPointer = ti.freqPointer;
proxPointer = ti.proxPointer;
skipOffset = ti.skipOffset;
}
public final void set(int docFreq,
long freqPointer, long proxPointer, int skipOffset) {
this.docFreq = docFreq;
this.freqPointer = freqPointer;
this.proxPointer = proxPointer;
this.skipOffset = skipOffset;
}
public final void set(TermInfo ti) {
docFreq = ti.docFreq;
freqPointer = ti.freqPointer;
proxPointer = ti.proxPointer;
skipOffset = ti.skipOffset;
}
}

View File

@ -1,351 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.Comparator;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.DoubleBarrelLRUCache;
/** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
* Directory. Pairs are accessed either by Term or by ordinal position the
* set
* @deprecated (4.0) This class has been replaced by
* FormatPostingsTermsDictReader, except for reading old segments.
* @lucene.experimental
*/
@Deprecated
final class TermInfosReader {
private final Directory directory;
private final String segment;
private final FieldInfos fieldInfos;
private final CloseableThreadLocal<ThreadResources> threadResources = new CloseableThreadLocal<ThreadResources>();
private final SegmentTermEnum origEnum;
private final long size;
private final TermInfosReaderIndex index;
private final int indexLength;
private final int totalIndexInterval;
private final static int DEFAULT_CACHE_SIZE = 1024;
// Just adds term's ord to TermInfo
private final static class TermInfoAndOrd extends TermInfo {
final long termOrd;
public TermInfoAndOrd(TermInfo ti, long termOrd) {
super(ti);
assert termOrd >= 0;
this.termOrd = termOrd;
}
}
private static class CloneableTerm extends DoubleBarrelLRUCache.CloneableKey {
Term term;
public CloneableTerm(Term t) {
this.term = t;
}
@Override
public boolean equals(Object other) {
CloneableTerm t = (CloneableTerm) other;
return this.term.equals(t.term);
}
@Override
public int hashCode() {
return term.hashCode();
}
@Override
public CloneableTerm clone() {
return new CloneableTerm(term);
}
}
private final DoubleBarrelLRUCache<CloneableTerm,TermInfoAndOrd> termsCache = new DoubleBarrelLRUCache<CloneableTerm,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
/**
* Per-thread resources managed by ThreadLocal
*/
private static final class ThreadResources {
SegmentTermEnum termEnum;
}
TermInfosReader(Directory dir, String seg, FieldInfos fis, IOContext context, int indexDivisor)
throws CorruptIndexException, IOException {
boolean success = false;
if (indexDivisor < 1 && indexDivisor != -1) {
throw new IllegalArgumentException("indexDivisor must be -1 (don't load terms index) or greater than 0: got " + indexDivisor);
}
try {
directory = dir;
segment = seg;
fieldInfos = fis;
origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_EXTENSION),
context), fieldInfos, false);
size = origEnum.size;
if (indexDivisor != -1) {
// Load terms index
totalIndexInterval = origEnum.indexInterval * indexDivisor;
final String indexFileName = IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION);
final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(indexFileName,
context), fieldInfos, true);
try {
index = new TermInfosReaderIndex(indexEnum, indexDivisor, dir.fileLength(indexFileName), totalIndexInterval);
indexLength = index.length();
} finally {
indexEnum.close();
}
} else {
// Do not load terms index:
totalIndexInterval = -1;
index = null;
indexLength = -1;
}
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
// fine) to hit a FileNotFound exception above. In
// this case, we want to explicitly close any subset
// of things that were opened so that we don't have to
// wait for a GC to do so.
if (!success) {
close();
}
}
}
public int getSkipInterval() {
return origEnum.skipInterval;
}
public int getMaxSkipLevels() {
return origEnum.maxSkipLevels;
}
void close() throws IOException {
if (origEnum != null)
origEnum.close();
threadResources.close();
}
/** Returns the number of term/value pairs in the set. */
long size() {
return size;
}
private ThreadResources getThreadResources() {
ThreadResources resources = threadResources.get();
if (resources == null) {
resources = new ThreadResources();
resources.termEnum = terms();
threadResources.set(resources);
}
return resources;
}
private static final Comparator<BytesRef> legacyComparator =
BytesRef.getUTF8SortedAsUTF16Comparator();
private final int compareAsUTF16(Term term1, Term term2) {
if (term1.field().equals(term2.field())) {
return legacyComparator.compare(term1.bytes(), term2.bytes());
} else {
return term1.field().compareTo(term2.field());
}
}
/** Returns the TermInfo for a Term in the set, or null. */
TermInfo get(Term term) throws IOException {
return get(term, false);
}
/** Returns the TermInfo for a Term in the set, or null. */
private TermInfo get(Term term, boolean mustSeekEnum) throws IOException {
if (size == 0) return null;
ensureIndexIsRead();
TermInfoAndOrd tiOrd = termsCache.get(new CloneableTerm(term));
ThreadResources resources = getThreadResources();
if (!mustSeekEnum && tiOrd != null) {
return tiOrd;
}
return seekEnum(resources.termEnum, term, tiOrd, true);
}
public void cacheCurrentTerm(SegmentTermEnum enumerator) {
termsCache.put(new CloneableTerm(enumerator.term()),
new TermInfoAndOrd(enumerator.termInfo,
enumerator.position));
}
TermInfo seekEnum(SegmentTermEnum enumerator, Term term, boolean useCache) throws IOException {
if (useCache) {
return seekEnum(enumerator, term,
termsCache.get(new CloneableTerm(term.deepCopyOf())),
useCache);
} else {
return seekEnum(enumerator, term, null, useCache);
}
}
TermInfo seekEnum(SegmentTermEnum enumerator, Term term, TermInfoAndOrd tiOrd, boolean useCache) throws IOException {
if (size == 0) {
return null;
}
// optimize sequential access: first try scanning cached enum w/o seeking
if (enumerator.term() != null // term is at or past current
&& ((enumerator.prev() != null && compareAsUTF16(term, enumerator.prev())> 0)
|| compareAsUTF16(term, enumerator.term()) >= 0)) {
int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
if (indexLength == enumOffset // but before end of block
|| index.compareTo(term, enumOffset) < 0) {
// no need to seek
final TermInfo ti;
int numScans = enumerator.scanTo(term);
if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
ti = enumerator.termInfo;
if (numScans > 1) {
// we only want to put this TermInfo into the cache if
// scanEnum skipped more than one dictionary entry.
// This prevents RangeQueries or WildcardQueries to
// wipe out the cache when they iterate over a large numbers
// of terms in order
if (tiOrd == null) {
if (useCache) {
termsCache.put(new CloneableTerm(term.deepCopyOf()),
new TermInfoAndOrd(ti, enumerator.position));
}
} else {
assert sameTermInfo(ti, tiOrd, enumerator);
assert (int) enumerator.position == tiOrd.termOrd;
}
}
} else {
ti = null;
}
return ti;
}
}
// random-access: must seek
final int indexPos;
if (tiOrd != null) {
indexPos = (int) (tiOrd.termOrd / totalIndexInterval);
} else {
// Must do binary search:
indexPos = index.getIndexOffset(term);
}
index.seekEnum(enumerator, indexPos);
enumerator.scanTo(term);
final TermInfo ti;
if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
ti = enumerator.termInfo;
if (tiOrd == null) {
if (useCache) {
termsCache.put(new CloneableTerm(term.deepCopyOf()),
new TermInfoAndOrd(ti, enumerator.position));
}
} else {
assert sameTermInfo(ti, tiOrd, enumerator);
assert enumerator.position == tiOrd.termOrd;
}
} else {
ti = null;
}
return ti;
}
// called only from asserts
private boolean sameTermInfo(TermInfo ti1, TermInfo ti2, SegmentTermEnum enumerator) {
if (ti1.docFreq != ti2.docFreq) {
return false;
}
if (ti1.freqPointer != ti2.freqPointer) {
return false;
}
if (ti1.proxPointer != ti2.proxPointer) {
return false;
}
// skipOffset is only valid when docFreq >= skipInterval:
if (ti1.docFreq >= enumerator.skipInterval &&
ti1.skipOffset != ti2.skipOffset) {
return false;
}
return true;
}
private void ensureIndexIsRead() {
if (index == null) {
throw new IllegalStateException("terms index was not loaded when this reader was created");
}
}
/** Returns the position of a Term in the set or -1. */
long getPosition(Term term) throws IOException {
if (size == 0) return -1;
ensureIndexIsRead();
int indexOffset = index.getIndexOffset(term);
SegmentTermEnum enumerator = getThreadResources().termEnum;
index.seekEnum(enumerator, indexOffset);
while(compareAsUTF16(term, enumerator.term()) > 0 && enumerator.next()) {}
if (compareAsUTF16(term, enumerator.term()) == 0)
return enumerator.position;
else
return -1;
}
/** Returns an enumeration of all the Terms and TermInfos in the set. */
public SegmentTermEnum terms() {
return origEnum.clone();
}
/** Returns an enumeration of terms starting at or after the named term. */
public SegmentTermEnum terms(Term term) throws IOException {
get(term, true);
return getThreadResources().termEnum.clone();
}
}

View File

@ -1,257 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
import org.apache.lucene.util.PagedBytes.PagedBytesDataOutput;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.GrowableWriter;
import org.apache.lucene.util.packed.PackedInts;
/**
* This stores a monotonically increasing set of <Term, TermInfo> pairs in an
* index segment. Pairs are accessed either by Term or by ordinal position the
* set. The Terms and TermInfo are actually serialized and stored into a byte
* array and pointers to the position of each are stored in a int array.
* @deprecated
*/
@Deprecated
class TermInfosReaderIndex {
private static final int MAX_PAGE_BITS = 18; // 256 KB block
private Term[] fields;
private int totalIndexInterval;
private Comparator<BytesRef> comparator = BytesRef.getUTF8SortedAsUTF16Comparator();
private final PagedBytesDataInput dataInput;
private final PackedInts.Reader indexToDataOffset;
private final int indexSize;
private final int skipInterval;
/**
* Loads the segment information at segment load time.
*
* @param indexEnum
* the term enum.
* @param indexDivisor
* the index divisor.
* @param tiiFileLength
* the size of the tii file, used to approximate the size of the
* buffer.
* @param totalIndexInterval
* the total index interval.
*/
TermInfosReaderIndex(SegmentTermEnum indexEnum, int indexDivisor, long tiiFileLength, int totalIndexInterval) throws IOException {
this.totalIndexInterval = totalIndexInterval;
indexSize = 1 + ((int) indexEnum.size - 1) / indexDivisor;
skipInterval = indexEnum.skipInterval;
// this is only an inital size, it will be GCed once the build is complete
long initialSize = (long) (tiiFileLength * 1.5) / indexDivisor;
PagedBytes dataPagedBytes = new PagedBytes(estimatePageBits(initialSize));
PagedBytesDataOutput dataOutput = dataPagedBytes.getDataOutput();
final int bitEstimate = 1+MathUtil.log(tiiFileLength, 2);
GrowableWriter indexToTerms = new GrowableWriter(bitEstimate, indexSize, PackedInts.DEFAULT);
String currentField = null;
List<String> fieldStrs = new ArrayList<String>();
int fieldCounter = -1;
for (int i = 0; indexEnum.next(); i++) {
Term term = indexEnum.term();
if (currentField == null || !currentField.equals(term.field())) {
currentField = term.field();
fieldStrs.add(currentField);
fieldCounter++;
}
TermInfo termInfo = indexEnum.termInfo();
indexToTerms.set(i, dataOutput.getPosition());
dataOutput.writeVInt(fieldCounter);
dataOutput.writeString(term.text());
dataOutput.writeVInt(termInfo.docFreq);
if (termInfo.docFreq >= skipInterval) {
dataOutput.writeVInt(termInfo.skipOffset);
}
dataOutput.writeVLong(termInfo.freqPointer);
dataOutput.writeVLong(termInfo.proxPointer);
dataOutput.writeVLong(indexEnum.indexPointer);
for (int j = 1; j < indexDivisor; j++) {
if (!indexEnum.next()) {
break;
}
}
}
fields = new Term[fieldStrs.size()];
for (int i = 0; i < fields.length; i++) {
fields[i] = new Term(fieldStrs.get(i));
}
dataPagedBytes.freeze(true);
dataInput = dataPagedBytes.getDataInput();
indexToDataOffset = indexToTerms.getMutable();
}
private static int estimatePageBits(long estSize) {
return Math.max(Math.min(64 - BitUtil.nlz(estSize), MAX_PAGE_BITS), 4);
}
void seekEnum(SegmentTermEnum enumerator, int indexOffset) throws IOException {
PagedBytesDataInput input = dataInput.clone();
input.setPosition(indexToDataOffset.get(indexOffset));
// read the term
int fieldId = input.readVInt();
Term field = fields[fieldId];
Term term = new Term(field.field(), input.readString());
// read the terminfo
TermInfo termInfo = new TermInfo();
termInfo.docFreq = input.readVInt();
if (termInfo.docFreq >= skipInterval) {
termInfo.skipOffset = input.readVInt();
} else {
termInfo.skipOffset = 0;
}
termInfo.freqPointer = input.readVLong();
termInfo.proxPointer = input.readVLong();
long pointer = input.readVLong();
// perform the seek
enumerator.seek(pointer, ((long) indexOffset * totalIndexInterval) - 1, term, termInfo);
}
/**
* Binary search for the given term.
*
* @param term
* the term to locate.
* @throws IOException
*/
int getIndexOffset(Term term) throws IOException {
int lo = 0;
int hi = indexSize - 1;
PagedBytesDataInput input = dataInput.clone();
BytesRef scratch = new BytesRef();
while (hi >= lo) {
int mid = (lo + hi) >>> 1;
int delta = compareTo(term, mid, input, scratch);
if (delta < 0)
hi = mid - 1;
else if (delta > 0)
lo = mid + 1;
else
return mid;
}
return hi;
}
/**
* Gets the term at the given position. For testing.
*
* @param termIndex
* the position to read the term from the index.
* @return the term.
* @throws IOException
*/
Term getTerm(int termIndex) throws IOException {
PagedBytesDataInput input = dataInput.clone();
input.setPosition(indexToDataOffset.get(termIndex));
// read the term
int fieldId = input.readVInt();
Term field = fields[fieldId];
return new Term(field.field(), input.readString());
}
/**
* Returns the number of terms.
*
* @return int.
*/
int length() {
return indexSize;
}
/**
* The compares the given term against the term in the index specified by the
* term index. ie It returns negative N when term is less than index term;
*
* @param term
* the given term.
* @param termIndex
* the index of the of term to compare.
* @return int.
* @throws IOException
*/
int compareTo(Term term, int termIndex) throws IOException {
return compareTo(term, termIndex, dataInput.clone(), new BytesRef());
}
/**
* Compare the fields of the terms first, and if not equals return from
* compare. If equal compare terms.
*
* @param term
* the term to compare.
* @param termIndex
* the position of the term in the input to compare
* @param input
* the input buffer.
* @return int.
* @throws IOException
*/
private int compareTo(Term term, int termIndex, PagedBytesDataInput input, BytesRef reuse) throws IOException {
// if term field does not equal mid's field index, then compare fields
// else if they are equal, compare term's string values...
int c = compareField(term, termIndex, input);
if (c == 0) {
reuse.length = input.readVInt();
reuse.grow(reuse.length);
input.readBytes(reuse.bytes, 0, reuse.length);
return comparator.compare(term.bytes(), reuse);
}
return c;
}
/**
* Compares the fields before checking the text of the terms.
*
* @param term
* the given term.
* @param termIndex
* the term that exists in the data block.
* @param input
* the data block.
* @return int.
* @throws IOException
*/
private int compareField(Term term, int termIndex, PagedBytesDataInput input) throws IOException {
input.setPosition(indexToDataOffset.get(termIndex));
return term.field().compareTo(fields[input.readVInt()].field());
}
}

View File

@ -1,25 +0,0 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codec to support Lucene 3.x indexes (readonly)
</body>
</html>

View File

@ -28,7 +28,7 @@ public class IndexFormatTooOldException extends CorruptIndexException {
/** @lucene.internal */
public IndexFormatTooOldException(String resourceDesc, String version) {
super("Format version is not supported (resource: " + resourceDesc + "): " +
version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
version + ". This version of Lucene only supports indexes created with release 4.0 and later.");
assert resourceDesc != null;
}
@ -41,7 +41,7 @@ public class IndexFormatTooOldException extends CorruptIndexException {
public IndexFormatTooOldException(String resourceDesc, int version, int minVersion, int maxVersion) {
super("Format version is not supported (resource: " + resourceDesc + "): " +
version + " (needs to be between " + minVersion + " and " + maxVersion +
"). This version of Lucene only supports indexes created with release 3.0 and later.");
"). This version of Lucene only supports indexes created with release 4.0 and later.");
assert resourceDesc != null;
}

View File

@ -33,8 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
import org.apache.lucene.codecs.lucene3x.Lucene3xSegmentInfoFormat;
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@ -2223,7 +2221,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
String newSegName = newSegmentName();
String dsName = Lucene3xSegmentInfoFormat.getDocStoreSegment(info.info);
String dsName = info.info.name;
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process segment origName=" + info.info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
@ -2355,6 +2353,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
/** Copies the segment files as-is into the IndexWriter's directory. */
// TODO: this can be substantially simplified now that 3.x support/shared docstores is removed!
private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName,
Map<String, String> dsNames, Set<String> dsFilesCopied, IOContext context,
Set<String> copiedFiles)
@ -2363,7 +2362,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// only relevant for segments that share doc store with others,
// because the DS might have been copied already, in which case we
// just want to update the DS name of this SegmentInfo.
final String dsName = Lucene3xSegmentInfoFormat.getDocStoreSegment(info.info);
final String dsName = info.info.name;
assert dsName != null;
final String newDsName;
if (dsNames.containsKey(dsName)) {
@ -2377,8 +2376,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// so we don't pass a null value to the si writer
FieldInfos fis = getFieldInfos(info.info);
Set<String> docStoreFiles3xOnly = Lucene3xCodec.getDocStoreFiles(info.info);
final Map<String,String> attributes;
// copy the attributes map, we might modify it below.
// also we need to ensure its read-write, since we will invoke the SIwriter (which might want to set something).
@ -2387,12 +2384,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} else {
attributes = new HashMap<String,String>(info.info.attributes());
}
if (docStoreFiles3xOnly != null) {
// only violate the codec this way if it's preflex &
// shares doc stores
// change docStoreSegment to newDsName
attributes.put(Lucene3xSegmentInfoFormat.DS_NAME_KEY, newDsName);
}
//System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion());
// Same SI as before but we change directory, name and docStoreSegment:
@ -2407,11 +2398,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// before writing SegmentInfo:
for (String file: info.files()) {
final String newFileName;
if (docStoreFiles3xOnly != null && docStoreFiles3xOnly.contains(file)) {
newFileName = newDsName + IndexFileNames.stripSegmentName(file);
} else {
newFileName = segName + IndexFileNames.stripSegmentName(file);
}
newFileName = segName + IndexFileNames.stripSegmentName(file);
segFiles.add(newFileName);
}
newInfo.setFiles(segFiles);
@ -2432,16 +2419,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Copy the segment's files
for (String file: info.files()) {
final String newFileName;
if (docStoreFiles3xOnly != null && docStoreFiles3xOnly.contains(file)) {
newFileName = newDsName + IndexFileNames.stripSegmentName(file);
if (dsFilesCopied.contains(newFileName)) {
continue;
}
dsFilesCopied.add(newFileName);
} else {
newFileName = segName + IndexFileNames.stripSegmentName(file);
}
final String newFileName = segName + IndexFileNames.stripSegmentName(file);
if (siFiles.contains(newFileName)) {
// We already rewrote this above
@ -4016,7 +3994,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + fileName);
}
assert Lucene3xSegmentInfoFormat.getDocStoreOffset(info) == -1;
// Now merge all added files
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);

View File

@ -232,7 +232,6 @@ public abstract class LogMergePolicy extends MergePolicy {
assert w != null;
boolean hasDeletions = w.numDeletedDocs(info) > 0;
return !hasDeletions &&
!info.info.hasSeparateNorms() &&
info.info.dir == w.getDirectory() &&
(info.info.getUseCompoundFile() == useCompoundFile || noCFSRatio < 1.0);
}

View File

@ -26,7 +26,6 @@ import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene3x.Lucene3xSegmentInfoFormat;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.TrackingDirectoryWrapper;
@ -106,14 +105,6 @@ public final class SegmentInfo {
return sizeInBytes;
}
/**
* @deprecated separate norms are not supported in >= 4.0
*/
@Deprecated
boolean hasSeparateNorms() {
return getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_KEY) != null;
}
/**
* Mark whether this segment is stored as a compound file.
*

View File

@ -33,9 +33,6 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
import org.apache.lucene.codecs.lucene3x.Lucene3xSegmentInfoFormat;
import org.apache.lucene.codecs.lucene3x.Lucene3xSegmentInfoReader;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexOutput;
import org.apache.lucene.store.DataOutput; // javadocs
@ -274,32 +271,29 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
ChecksumIndexInput input = new ChecksumIndexInput(directory.openInput(segmentFileName, IOContext.READ));
try {
final int format = input.readInt();
if (format == CodecUtil.CODEC_MAGIC) {
// 4.0+
CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_40);
version = input.readLong();
counter = input.readInt();
int numSegments = input.readInt();
for(int seg=0;seg<numSegments;seg++) {
String segName = input.readString();
Codec codec = Codec.forName(input.readString());
//System.out.println("SIS.read seg=" + seg + " codec=" + codec);
SegmentInfo info = codec.segmentInfoFormat().getSegmentInfosReader().read(directory, segName, IOContext.READ);
info.setCodec(codec);
long delGen = input.readLong();
int delCount = input.readInt();
assert delCount <= info.getDocCount();
add(new SegmentInfoPerCommit(info, delCount, delGen));
}
userData = input.readStringStringMap();
} else {
Lucene3xSegmentInfoReader.readLegacyInfos(this, directory, input, format);
Codec codec = Codec.forName("Lucene3x");
for (SegmentInfoPerCommit info : this) {
info.info.setCodec(codec);
}
// NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
// to read the magic ourselves.
int magic = input.readInt();
if (magic != CodecUtil.CODEC_MAGIC) {
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
// 4.0+
CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_40);
version = input.readLong();
counter = input.readInt();
int numSegments = input.readInt();
for(int seg=0;seg<numSegments;seg++) {
String segName = input.readString();
Codec codec = Codec.forName(input.readString());
//System.out.println("SIS.read seg=" + seg + " codec=" + codec);
SegmentInfo info = codec.segmentInfoFormat().getSegmentInfosReader().read(directory, segName, IOContext.READ);
info.setCodec(codec);
long delGen = input.readLong();
int delCount = input.readInt();
assert delCount <= info.getDocCount();
add(new SegmentInfoPerCommit(info, delCount, delGen));
}
userData = input.readStringStringMap();
final long checksumNow = input.getChecksum();
final long checksumThen = input.readLong();
@ -351,8 +345,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
ChecksumIndexOutput segnOutput = null;
boolean success = false;
final Set<String> upgradedSIFiles = new HashSet<String>();
try {
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
CodecUtil.writeHeader(segnOutput, "segments", VERSION_40);
@ -368,16 +360,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
assert si.dir == directory;
assert siPerCommit.getDelCount() <= si.getDocCount();
// If this segment is pre-4.x, perform a one-time
// "ugprade" to write the .si file for it:
String version = si.getVersion();
if (version == null || StringHelper.getVersionComparator().compare(version, "4.0") < 0) {
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfoFormat.UPGRADED_SI_EXTENSION);
if (!directory.fileExists(fileName)) {
upgradedSIFiles.add(write3xInfo(directory, si, IOContext.DEFAULT));
}
}
}
segnOutput.writeStringStringMap(userData);
pendingSegnOutput = segnOutput;
@ -388,14 +370,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
// but suppress any exception:
IOUtils.closeWhileHandlingException(segnOutput);
for(String fileName : upgradedSIFiles) {
try {
directory.deleteFile(fileName);
} catch (Throwable t) {
// Suppress so we keep throwing the original exception
}
}
try {
// Try not to leave a truncated segments_N file in
// the index:
@ -407,49 +381,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
}
}
@Deprecated
public static String write3xInfo(Directory dir, SegmentInfo si, IOContext context) throws IOException {
// NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfoFormat.UPGRADED_SI_EXTENSION);
si.addFile(fileName);
//System.out.println("UPGRADE write " + fileName);
boolean success = false;
IndexOutput output = dir.createOutput(fileName, context);
try {
// we are about to write this SI in 3.x format, dropping all codec information, etc.
// so it had better be a 3.x segment or you will get very confusing errors later.
assert si.getCodec() instanceof Lucene3xCodec : "broken test, trying to mix preflex with other codecs";
CodecUtil.writeHeader(output, Lucene3xSegmentInfoFormat.UPGRADED_SI_CODEC_NAME,
Lucene3xSegmentInfoFormat.UPGRADED_SI_VERSION_CURRENT);
// Write the Lucene version that created this segment, since 3.1
output.writeString(si.getVersion());
output.writeInt(si.getDocCount());
output.writeStringStringMap(si.attributes());
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeStringStringMap(si.getDiagnostics());
output.writeStringSet(si.files());
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
try {
si.dir.deleteFile(fileName);
} catch (Throwable t) {
// Suppress so we keep throwing the original exception
}
} else {
output.close();
}
}
return fileName;
}
/**
* Returns a copy of this instance, also copying each
* SegmentInfo.

View File

@ -629,7 +629,6 @@ public class TieredMergePolicy extends MergePolicy {
assert w != null;
boolean hasDeletions = w.numDeletedDocs(info) > 0;
return !hasDeletions &&
!info.info.hasSeparateNorms() &&
info.info.dir == w.getDirectory() &&
(info.info.getUseCompoundFile() == useCompoundFile || noCFSRatio < 1.0);
}

View File

@ -14,6 +14,5 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40Codec
org.apache.lucene.codecs.lucene3x.Lucene3xCodec
org.apache.lucene.codecs.simpletext.SimpleTextCodec
org.apache.lucene.codecs.appending.AppendingCodec

View File

@ -1,34 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene3x.PreFlexRWCodec;
import org.apache.lucene.util.LuceneTestCase;
/**
* Test that the SPI magic is returning "PreFlexRWCodec" for Lucene3x
*
* @lucene.experimental
*/
public class TestImpersonation extends LuceneTestCase {
public void test() throws Exception {
Codec codec = Codec.forName("Lucene3x");
assertTrue(codec instanceof PreFlexRWCodec);
}
}

View File

@ -1,356 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.store.*;
import org.apache.lucene.codecs.lucene3x.PreFlexRWCodec;
import org.apache.lucene.document.*;
import org.apache.lucene.analysis.*;
import org.apache.lucene.index.*;
import org.apache.lucene.util.*;
import java.util.*;
import java.io.IOException;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestSurrogates extends LuceneTestCase {
/** we will manually instantiate preflex-rw here */
@BeforeClass
public static void beforeClass() {
LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE = true;
}
private static String makeDifficultRandomUnicodeString(Random r) {
final int end = r.nextInt(20);
if (end == 0) {
// allow 0 length
return "";
}
final char[] buffer = new char[end];
for (int i = 0; i < end; i++) {
int t = r.nextInt(5);
if (0 == t && i < end - 1) {
// hi
buffer[i++] = (char) (0xd800 + r.nextInt(2));
// lo
buffer[i] = (char) (0xdc00 + r.nextInt(2));
} else if (t <= 3) {
buffer[i] = (char) ('a' + r.nextInt(2));
} else if (4 == t) {
buffer[i] = (char) (0xe000 + r.nextInt(2));
}
}
return new String(buffer, 0, end);
}
private String toHexString(Term t) {
return t.field() + ":" + UnicodeUtil.toHexString(t.text());
}
private String getRandomString(Random r) {
String s;
if (r.nextInt(5) == 1) {
if (r.nextInt(3) == 1) {
s = makeDifficultRandomUnicodeString(r);
} else {
s = _TestUtil.randomUnicodeString(r);
}
} else {
s = _TestUtil.randomRealisticUnicodeString(r);
}
return s;
}
private static class SortTermAsUTF16Comparator implements Comparator<Term> {
private static final Comparator<BytesRef> legacyComparator =
BytesRef.getUTF8SortedAsUTF16Comparator();
public int compare(Term term1, Term term2) {
if (term1.field().equals(term2.field())) {
return legacyComparator.compare(term1.bytes(), term2.bytes());
} else {
return term1.field().compareTo(term2.field());
}
}
}
private static final SortTermAsUTF16Comparator termAsUTF16Comparator = new SortTermAsUTF16Comparator();
// single straight enum
private void doTestStraightEnum(List<Term> fieldTerms, IndexReader reader, int uniqueTermCount) throws IOException {
if (VERBOSE) {
System.out.println("\nTEST: top now enum reader=" + reader);
}
FieldsEnum fieldsEnum = MultiFields.getFields(reader).iterator();
{
// Test straight enum:
String field;
int termCount = 0;
while((field = fieldsEnum.next()) != null) {
Terms terms = fieldsEnum.terms();
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
BytesRef text;
BytesRef lastText = null;
while((text = termsEnum.next()) != null) {
Term exp = fieldTerms.get(termCount);
if (VERBOSE) {
System.out.println(" got term=" + field + ":" + UnicodeUtil.toHexString(text.utf8ToString()));
System.out.println(" exp=" + exp.field() + ":" + UnicodeUtil.toHexString(exp.text().toString()));
System.out.println();
}
if (lastText == null) {
lastText = BytesRef.deepCopyOf(text);
} else {
assertTrue(lastText.compareTo(text) < 0);
lastText.copyBytes(text);
}
assertEquals(exp.field(), field);
assertEquals(exp.bytes(), text);
termCount++;
}
if (VERBOSE) {
System.out.println(" no more terms for field=" + field);
}
}
assertEquals(uniqueTermCount, termCount);
}
}
// randomly seeks to term that we know exists, then next's
// from there
private void doTestSeekExists(Random r, List<Term> fieldTerms, IndexReader reader) throws IOException {
final Map<String,TermsEnum> tes = new HashMap<String,TermsEnum>();
// Test random seek to existing term, then enum:
if (VERBOSE) {
System.out.println("\nTEST: top now seek");
}
int num = atLeast(100);
for (int iter = 0; iter < num; iter++) {
// pick random field+term
int spot = r.nextInt(fieldTerms.size());
Term term = fieldTerms.get(spot);
String field = term.field();
if (VERBOSE) {
System.out.println("TEST: exist seek field=" + field + " term=" + UnicodeUtil.toHexString(term.text()));
}
// seek to it
TermsEnum te = tes.get(field);
if (te == null) {
te = MultiFields.getTerms(reader, field).iterator(null);
tes.put(field, te);
}
if (VERBOSE) {
System.out.println(" done get enum");
}
// seek should find the term
assertEquals(TermsEnum.SeekStatus.FOUND,
te.seekCeil(term.bytes()));
// now .next() this many times:
int ct = _TestUtil.nextInt(r, 5, 100);
for(int i=0;i<ct;i++) {
if (VERBOSE) {
System.out.println("TEST: now next()");
}
if (1+spot+i >= fieldTerms.size()) {
break;
}
term = fieldTerms.get(1+spot+i);
if (!term.field().equals(field)) {
assertNull(te.next());
break;
} else {
BytesRef t = te.next();
if (VERBOSE) {
System.out.println(" got term=" + (t == null ? null : UnicodeUtil.toHexString(t.utf8ToString())));
System.out.println(" exp=" + UnicodeUtil.toHexString(term.text().toString()));
}
assertEquals(term.bytes(), t);
}
}
}
}
private void doTestSeekDoesNotExist(Random r, int numField, List<Term> fieldTerms, Term[] fieldTermsArray, IndexReader reader) throws IOException {
final Map<String,TermsEnum> tes = new HashMap<String,TermsEnum>();
if (VERBOSE) {
System.out.println("TEST: top random seeks");
}
{
int num = atLeast(100);
for (int iter = 0; iter < num; iter++) {
// seek to random spot
String field = ("f" + r.nextInt(numField)).intern();
Term tx = new Term(field, getRandomString(r));
int spot = Arrays.binarySearch(fieldTermsArray, tx);
if (spot < 0) {
if (VERBOSE) {
System.out.println("TEST: non-exist seek to " + field + ":" + UnicodeUtil.toHexString(tx.text()));
}
// term does not exist:
TermsEnum te = tes.get(field);
if (te == null) {
te = MultiFields.getTerms(reader, field).iterator(null);
tes.put(field, te);
}
if (VERBOSE) {
System.out.println(" got enum");
}
spot = -spot - 1;
if (spot == fieldTerms.size() || !fieldTerms.get(spot).field().equals(field)) {
assertEquals(TermsEnum.SeekStatus.END, te.seekCeil(tx.bytes()));
} else {
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(tx.bytes()));
if (VERBOSE) {
System.out.println(" got term=" + UnicodeUtil.toHexString(te.term().utf8ToString()));
System.out.println(" exp term=" + UnicodeUtil.toHexString(fieldTerms.get(spot).text()));
}
assertEquals(fieldTerms.get(spot).bytes(),
te.term());
// now .next() this many times:
int ct = _TestUtil.nextInt(r, 5, 100);
for(int i=0;i<ct;i++) {
if (VERBOSE) {
System.out.println("TEST: now next()");
}
if (1+spot+i >= fieldTerms.size()) {
break;
}
Term term = fieldTerms.get(1+spot+i);
if (!term.field().equals(field)) {
assertNull(te.next());
break;
} else {
BytesRef t = te.next();
if (VERBOSE) {
System.out.println(" got term=" + (t == null ? null : UnicodeUtil.toHexString(t.utf8ToString())));
System.out.println(" exp=" + UnicodeUtil.toHexString(term.text().toString()));
}
assertEquals(term.bytes(), t);
}
}
}
}
}
}
}
@Test
public void testSurrogatesOrder() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(),
dir,
newIndexWriterConfig( TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setCodec(new PreFlexRWCodec()));
final int numField = _TestUtil.nextInt(random(), 2, 5);
int uniqueTermCount = 0;
int tc = 0;
List<Term> fieldTerms = new ArrayList<Term>();
for(int f=0;f<numField;f++) {
String field = "f" + f;
final int numTerms = atLeast(200);
final Set<String> uniqueTerms = new HashSet<String>();
for(int i=0;i<numTerms;i++) {
String term = getRandomString(random()) + "_ " + (tc++);
uniqueTerms.add(term);
fieldTerms.add(new Term(field, term));
Document doc = new Document();
doc.add(newField(field, term, StringField.TYPE_UNSTORED));
w.addDocument(doc);
}
uniqueTermCount += uniqueTerms.size();
}
IndexReader reader = w.getReader();
if (VERBOSE) {
Collections.sort(fieldTerms, termAsUTF16Comparator);
System.out.println("\nTEST: UTF16 order");
for(Term t: fieldTerms) {
System.out.println(" " + toHexString(t));
}
}
// sorts in code point order:
Collections.sort(fieldTerms);
if (VERBOSE) {
System.out.println("\nTEST: codepoint order");
for(Term t: fieldTerms) {
System.out.println(" " + toHexString(t));
}
}
Term[] fieldTermsArray = fieldTerms.toArray(new Term[fieldTerms.size()]);
//SegmentInfo si = makePreFlexSegment(r, "_0", dir, fieldInfos, codec, fieldTerms);
//FieldsProducer fields = codec.fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, 1));
//assertNotNull(fields);
doTestStraightEnum(fieldTerms, reader, uniqueTermCount);
doTestSeekExists(random(), fieldTerms, reader);
doTestSeekDoesNotExist(random(), numField, fieldTerms, fieldTermsArray, reader);
reader.close();
w.close();
dir.close();
}
}

View File

@ -1,208 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
public class TestTermInfosReaderIndex extends LuceneTestCase {
private static int NUMBER_OF_DOCUMENTS;
private static int NUMBER_OF_FIELDS;
private static TermInfosReaderIndex index;
private static Directory directory;
private static SegmentTermEnum termEnum;
private static int indexDivisor;
private static int termIndexInterval;
private static IndexReader reader;
private static List<Term> sampleTerms;
/** we will manually instantiate preflex-rw here */
@BeforeClass
public static void beforeClass() throws Exception {
LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE = true;
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
termIndexInterval = config.getTermIndexInterval();
indexDivisor = _TestUtil.nextInt(random(), 1, 10);
NUMBER_OF_DOCUMENTS = atLeast(100);
NUMBER_OF_FIELDS = atLeast(Math.max(10, 3*termIndexInterval*indexDivisor/NUMBER_OF_DOCUMENTS));
directory = newDirectory();
config.setCodec(new PreFlexRWCodec());
LogMergePolicy mp = newLogMergePolicy();
// turn off compound file, this test will open some index files directly.
mp.setUseCompoundFile(false);
config.setMergePolicy(mp);
populate(directory, config);
DirectoryReader r0 = IndexReader.open(directory);
SegmentReader r = LuceneTestCase.getOnlySegmentReader(r0);
String segment = r.getSegmentName();
r.close();
FieldInfosReader infosReader = new PreFlexRWCodec().fieldInfosFormat().getFieldInfosReader();
FieldInfos fieldInfos = infosReader.read(directory, segment, IOContext.READONCE);
String segmentFileName = IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION);
long tiiFileLength = directory.fileLength(segmentFileName);
IndexInput input = directory.openInput(segmentFileName, newIOContext(random()));
termEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_EXTENSION), newIOContext(random())), fieldInfos, false);
int totalIndexInterval = termEnum.indexInterval * indexDivisor;
SegmentTermEnum indexEnum = new SegmentTermEnum(input, fieldInfos, true);
index = new TermInfosReaderIndex(indexEnum, indexDivisor, tiiFileLength, totalIndexInterval);
indexEnum.close();
input.close();
reader = IndexReader.open(directory);
sampleTerms = sample(reader,1000);
}
@AfterClass
public static void afterClass() throws Exception {
termEnum.close();
reader.close();
directory.close();
termEnum = null;
reader = null;
directory = null;
index = null;
sampleTerms = null;
}
public void testSeekEnum() throws CorruptIndexException, IOException {
int indexPosition = 3;
SegmentTermEnum clone = termEnum.clone();
Term term = findTermThatWouldBeAtIndex(clone, indexPosition);
SegmentTermEnum enumerator = clone;
index.seekEnum(enumerator, indexPosition);
assertEquals(term, enumerator.term());
clone.close();
}
public void testCompareTo() throws IOException {
Term term = new Term("field" + random().nextInt(NUMBER_OF_FIELDS) ,getText());
for (int i = 0; i < index.length(); i++) {
Term t = index.getTerm(i);
int compareTo = term.compareTo(t);
assertEquals(compareTo, index.compareTo(term, i));
}
}
public void testRandomSearchPerformance() throws CorruptIndexException, IOException {
IndexSearcher searcher = new IndexSearcher(reader);
for (Term t : sampleTerms) {
TermQuery query = new TermQuery(t);
TopDocs topDocs = searcher.search(query, 10);
assertTrue(topDocs.totalHits > 0);
}
}
private static List<Term> sample(IndexReader reader, int size) throws IOException {
List<Term> sample = new ArrayList<Term>();
Random random = new Random();
FieldsEnum fieldsEnum = MultiFields.getFields(reader).iterator();
String field;
while((field = fieldsEnum.next()) != null) {
Terms terms = fieldsEnum.terms();
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
while (termsEnum.next() != null) {
if (sample.size() >= size) {
int pos = random.nextInt(size);
sample.set(pos, new Term(field, termsEnum.term()));
} else {
sample.add(new Term(field, termsEnum.term()));
}
}
}
Collections.shuffle(sample);
return sample;
}
private Term findTermThatWouldBeAtIndex(SegmentTermEnum termEnum, int index) throws IOException {
int termPosition = index * termIndexInterval * indexDivisor;
for (int i = 0; i < termPosition; i++) {
// TODO: this test just uses random terms, so this is always possible
assumeTrue("ran out of terms", termEnum.next());
}
final Term term = termEnum.term();
// An indexed term is only written when the term after
// it exists, so, if the number of terms is 0 mod
// termIndexInterval, the last index term will not be
// written; so we require a term after this term
// as well:
assumeTrue("ran out of terms", termEnum.next());
return term;
}
private static void populate(Directory directory, IndexWriterConfig config) throws CorruptIndexException, LockObtainFailedException, IOException {
RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config);
for (int i = 0; i < NUMBER_OF_DOCUMENTS; i++) {
Document document = new Document();
for (int f = 0; f < NUMBER_OF_FIELDS; f++) {
document.add(newField("field" + f, getText(), StringField.TYPE_UNSTORED));
}
writer.addDocument(document);
}
writer.forceMerge(1);
writer.close();
}
private static String getText() {
return Long.toString(random().nextLong(),Character.MAX_RADIX);
}
}

View File

@ -34,13 +34,9 @@ import java.util.List;
import java.util.Random;
import org.junit.Ignore;
// NOTE: this test will fail w/ PreFlexRW codec! (Because
// this test uses full binary term space, but PreFlex cannot
// handle this since it requires the terms are UTF8 bytes).
//
// Also, SimpleText codec will consume very large amounts of
// NOTE: SimpleText codec will consume very large amounts of
// disk (but, should run successfully). Best to run w/
// -Dtests.codec=Standard, and w/ plenty of RAM, eg:
// -Dtests.codec=<current codec>, and w/ plenty of RAM, eg:
//
// ant test -Dtest.slow=true -Dtests.heapsize=8g
//
@ -144,9 +140,6 @@ public class Test2BTerms extends LuceneTestCase {
@Slow
public void test2BTerms() throws IOException {
if ("Lucene3x".equals(Codec.getDefault().getName())) {
throw new RuntimeException("this test cannot run with PreFlex codec");
}
System.out.println("Starting Test2B");
final long TERM_COUNT = ((long) Integer.MAX_VALUE) + 100000000;

View File

@ -57,12 +57,13 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
/*
Verify we can read the pre-4.0 file format, do searches
Verify we can read the pre-5.0 file format, do searches
against it, and add documents to it.
*/
// don't use 3.x codec, its unrealistic since it means
// note: add this if we make a 4.x impersonator
// TODO: don't use 4.x codec, its unrealistic since it means
// we won't even be running the actual code, only the impostor
@SuppressCodecs("Lucene3x")
// @SuppressCodecs("Lucene4x")
public class TestBackwardsCompatibility extends LuceneTestCase {
// Uncomment these cases & run them on an older Lucene
@ -87,7 +88,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
/*
// These are only needed for the special upgrade test to verify
// that also single-segment indexes are correctly upgraded by IndexUpgrader.
// You don't need them to be build for non-3.1 (the test is happy with just one
// You don't need them to be build for non-4.0 (the test is happy with just one
// "old" segment format, version is unimportant:
public void testCreateSingleSegmentCFS() throws IOException {
@ -99,14 +100,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
}
*/
final static String[] oldNames = {"30.cfs",
"30.nocfs",
"31.cfs",
"31.nocfs",
"32.cfs",
"32.nocfs",
"34.cfs",
"34.nocfs",
final static String[] oldNames = {"40.cfs",
"40.nocfs",
};
final String[] unsupportedNames = {"19.cfs",
@ -123,10 +118,18 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
"24.nocfs",
"29.cfs",
"29.nocfs",
"30.cfs",
"30.nocfs",
"31.cfs",
"31.nocfs",
"32.cfs",
"32.nocfs",
"34.cfs",
"34.nocfs"
};
final static String[] oldSingleSegmentNames = {"31.optimized.cfs",
"31.optimized.nocfs",
final static String[] oldSingleSegmentNames = {"40.optimized.cfs",
"40.optimized.nocfs",
};
static Map<String,Directory> oldIndexDirs;
@ -341,16 +344,15 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
ScoreDoc[] hits = searcher.search(new TermQuery(new Term(new String("content"), "aaa")), null, 1000).scoreDocs;
// First document should be #21 since it's norm was
// increased:
// First document should be #0
Document d = searcher.getIndexReader().document(hits[0].doc);
assertEquals("didn't get the right document first", "21", d.get("id"));
assertEquals("didn't get the right document first", "0", d.get("id"));
doTestHits(hits, 34, searcher.getIndexReader());
hits = searcher.search(new TermQuery(new Term("utf8", "\u0000")), null, 1000).scoreDocs;
assertEquals(34, hits.length);
hits = searcher.search(new TermQuery(new Term(new String("utf8"), "Lu\uD834\uDD1Ece\uD834\uDD60ne")), null, 1000).scoreDocs;
hits = searcher.search(new TermQuery(new Term(new String("utf8"), "lu\uD834\uDD1Ece\uD834\uDD60ne")), null, 1000).scoreDocs;
assertEquals(34, hits.length);
hits = searcher.search(new TermQuery(new Term("utf8", "ab\ud917\udc17cd")), null, 1000).scoreDocs;
assertEquals(34, hits.length);
@ -366,7 +368,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
public void changeIndexWithAdds(Random random, Directory dir, String origOldName) throws IOException {
// open writer
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
// add 10 docs
for(int i=0;i<10;i++) {
addDoc(writer, 35+i);
@ -387,12 +389,12 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
IndexSearcher searcher = new IndexSearcher(reader);
ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
Document d = searcher.getIndexReader().document(hits[0].doc);
assertEquals("wrong first document", "21", d.get("id"));
assertEquals("wrong first document", "0", d.get("id"));
doTestHits(hits, 44, searcher.getIndexReader());
reader.close();
// fully merge
writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
writer.forceMerge(1);
writer.close();
@ -402,7 +404,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
assertEquals("wrong number of hits", 44, hits.length);
d = searcher.doc(hits[0].doc);
doTestHits(hits, 44, searcher.getIndexReader());
assertEquals("wrong first document", "21", d.get("id"));
assertEquals("wrong first document", "0", d.get("id"));
reader.close();
}
@ -413,7 +415,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
assertEquals("wrong number of hits", 34, hits.length);
Document d = searcher.doc(hits[0].doc);
assertEquals("wrong first document", "21", d.get("id"));
assertEquals("wrong first document", "0", d.get("id"));
reader.close();
// fully merge
@ -760,16 +762,4 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
dir.close();
}
}
public static final String surrogatesIndexName = "index.36.surrogates.zip";
public void testSurrogates() throws Exception {
File oldIndexDir = _TestUtil.getTempDir("surrogates");
_TestUtil.unzip(getDataFile(surrogatesIndexName), oldIndexDir);
Directory dir = newFSDirectory(oldIndexDir);
// TODO: more tests
_TestUtil.checkIndex(dir);
dir.close();
}
}

View File

@ -29,12 +29,10 @@ import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
/**
* Test indexing and searching some byte[] terms
*/
@SuppressCodecs("Lucene3x")
public class TestBinaryTerms extends LuceneTestCase {
public void testBinary() throws IOException {
Directory dir = newDirectory();

View File

@ -28,7 +28,6 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
@ -254,7 +253,7 @@ public class TestCodecs extends LuceneTestCase {
final FieldData[] fields = new FieldData[] {field};
final FieldInfos fieldInfos = builder.finish();
final Directory dir = newDirectory();
this.write(fieldInfos, dir, fields, true);
this.write(fieldInfos, dir, fields);
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
@ -310,7 +309,7 @@ public class TestCodecs extends LuceneTestCase {
System.out.println("TEST: now write postings");
}
this.write(fieldInfos, dir, fields, false);
this.write(fieldInfos, dir, fields);
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000,
false, codec, null, null);
@ -454,10 +453,6 @@ public class TestCodecs extends LuceneTestCase {
for(int iter=0;iter<NUM_TEST_ITER;iter++) {
final FieldData field = fields[random().nextInt(fields.length)];
final TermsEnum termsEnum = termsDict.terms(field.fieldInfo.name).iterator(null);
if (si.getCodec() instanceof Lucene3xCodec) {
// code below expects unicode sort order
continue;
}
int upto = 0;
// Test straight enum of the terms:
@ -613,7 +608,7 @@ public class TestCodecs extends LuceneTestCase {
}
}
private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields, boolean allowPreFlex) throws Throwable {
private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
final int termIndexInterval = _TestUtil.nextInt(random(), 13, 27);
final Codec codec = Codec.getDefault();
@ -623,10 +618,6 @@ public class TestCodecs extends LuceneTestCase {
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
Arrays.sort(fields);
for (final FieldData field : fields) {
if (!allowPreFlex && codec instanceof Lucene3xCodec) {
// code below expects unicode sort order
continue;
}
field.write(consumer);
}
consumer.close();

View File

@ -38,7 +38,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
*
*/
// TODO: what is the problem with SimpleText
@SuppressCodecs({ "SimpleText", "Lucene3x" })
@SuppressCodecs("SimpleText")
public class TestCustomNorms extends LuceneTestCase {
final String floatTestField = "normsTestFloat";
final String exceptionTestField = "normsTestExcp";

View File

@ -24,12 +24,10 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
/**
* Tests the Terms.docCount statistic
*/
@SuppressCodecs("Lucene3x")
public class TestDocCount extends LuceneTestCase {
public void testSimple() throws Exception {
Directory dir = newDirectory();

View File

@ -22,7 +22,6 @@ import java.util.*;
import java.util.Map.Entry;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.ByteDocValuesField;
import org.apache.lucene.document.DerefBytesDocValuesField;
import org.apache.lucene.document.Document;
@ -56,7 +55,6 @@ import org.junit.Before;
* Tests DocValues integration into IndexWriter & Codecs
*
*/
@SuppressCodecs("Lucene3x")
public class TestDocValuesIndexing extends LuceneTestCase {
/*
* - add test for multi segment case with deletes

View File

@ -39,7 +39,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
@BeforeClass
public static void beforeClass() throws Exception {
lineDocFile = new LineFileDocs(random(), defaultCodecSupportsDocValues());
lineDocFile = new LineFileDocs(random(), true);
}
@AfterClass

View File

@ -59,7 +59,7 @@ public class TestForceMergeForever extends LuceneTestCase {
// Try to make an index that requires merging:
w.getConfig().setMaxBufferedDocs(_TestUtil.nextInt(random(), 2, 11));
final int numStartDocs = atLeast(20);
final LineFileDocs docs = new LineFileDocs(random(), defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random(), true);
for(int docIDX=0;docIDX<numStartDocs;docIDX++) {
w.addDocument(docs.nextDoc());
}

View File

@ -65,9 +65,12 @@ public class TestIndexWriterOnJRECrash extends TestNRTThreads {
return;
}
} else {
// note: re-enable this if we create a 4.x impersonator,
// and if its format is actually different than the real 4.x (unlikely)
// TODO: the non-fork code could simply enable impersonation?
assumeFalse("does not support PreFlex, see LUCENE-3992",
Codec.getDefault().getName().equals("Lucene3x"));
// assumeFalse("does not support PreFlex, see LUCENE-3992",
// Codec.getDefault().getName().equals("Lucene4x"));
// we are the fork, setup a crashing thread
final int crashTime = _TestUtil.nextInt(random(), 3000, 4000);
Thread t = new Thread() {

View File

@ -971,9 +971,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
public void testNoTermsIndex() throws Exception {
// Some Codecs don't honor the ReaderTermsIndexDivisor, so skip the test if
// they're picked.
assumeFalse("PreFlex codec does not support ReaderTermsIndexDivisor!",
"Lucene3x".equals(Codec.getDefault().getName()));
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setReaderTermsIndexDivisor(-1);

View File

@ -27,9 +27,7 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
@SuppressCodecs("Lucene3x")
public class TestMixedCodecs extends LuceneTestCase {
public void test() throws Exception {

View File

@ -183,7 +183,7 @@ public class TestNorms extends LuceneTestCase {
Similarity provider = new MySimProvider(writeNorms);
config.setSimilarity(provider);
RandomIndexWriter writer = new RandomIndexWriter(random, dir, config);
final LineFileDocs docs = new LineFileDocs(random, defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random, true);
int num = atLeast(100);
for (int i = 0; i < num; i++) {
Document doc = docs.nextDoc();

View File

@ -46,7 +46,7 @@ import org.apache.lucene.util._TestUtil;
// TODO: we really need to test indexingoffsets, but then getting only docs / docs + freqs.
// not all codecs store prx separate...
// TODO: fix sep codec to index offsets so we can greatly reduce this list!
@SuppressCodecs({"Lucene3x", "MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
public class TestPostingsOffsets extends LuceneTestCase {
IndexWriterConfig iwc;

View File

@ -37,10 +37,10 @@ public class TestRollingUpdates extends LuceneTestCase {
Random random = new Random(random().nextLong());
final MockDirectoryWrapper dir = newDirectory();
dir.setCheckIndexOnClose(false); // we use a custom codec provider
final LineFileDocs docs = new LineFileDocs(random, defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random, true);
//provider.register(new MemoryCodec());
if ( (!"Lucene3x".equals(Codec.getDefault().getName())) && random().nextBoolean()) {
if (random().nextBoolean()) {
Codec.setDefault(_TestUtil.alwaysPostingsFormat(new MemoryPostingsFormat(random().nextBoolean())));
}

View File

@ -44,7 +44,7 @@ public class TestTermsEnum extends LuceneTestCase {
public void test() throws Exception {
Random random = new Random(random().nextLong());
final LineFileDocs docs = new LineFileDocs(random, defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random, true);
final Directory d = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), d);
final int numDocs = atLeast(10);

View File

@ -55,9 +55,7 @@ public class TestTermsEnum2 extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
// we generate aweful regexps: good for testing.
// but for preflex codec, the test can be very slow, so use less iterations.
numIterations = Codec.getDefault().getName().equals("Lucene3x") ? 10 * RANDOM_MULTIPLIER : atLeast(50);
numIterations = atLeast(50);
dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,

View File

@ -43,7 +43,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
@SuppressCodecs("Lucene3x")
public class TestTypePromotion extends LuceneTestCase {
private static EnumSet<Type> INTEGERS = EnumSet.of(Type.VAR_INTS,

View File

@ -43,7 +43,6 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
* In the example, a docvalues field is used as a per-document boost (separate from the norm)
* @lucene.experimental
*/
@SuppressCodecs("Lucene3x")
public class TestDocValuesScoring extends LuceneTestCase {
private static final float SCORE_EPSILON = 0.001f; /* for comparing floats */

View File

@ -59,10 +59,7 @@ public class TestPrefixRandom extends LuceneTestCase {
Field field = newField("field", "", StringField.TYPE_UNSTORED);
doc.add(field);
// we generate aweful prefixes: good for testing.
// but for preflex codec, the test can be very slow, so use less iterations.
final String codec = Codec.getDefault().getName();
int num = codec.equals("Lucene3x") ? 200 * RANDOM_MULTIPLIER : atLeast(1000);
int num = atLeast(1000);
for (int i = 0; i < num; i++) {
field.setStringValue(_TestUtil.randomUnicodeString(random(), 10));
writer.addDocument(doc);

View File

@ -139,9 +139,7 @@ public class TestRegexpRandom2 extends LuceneTestCase {
/** test a bunch of random regular expressions */
public void testRegexps() throws Exception {
// we generate aweful regexps: good for testing.
// but for preflex codec, the test can be very slow, so use less iterations.
int num = Codec.getDefault().getName().equals("Lucene3x") ? 100 * RANDOM_MULTIPLIER : atLeast(1000);
int num = atLeast(1000);
for (int i = 0; i < num; i++) {
String reg = AutomatonTestUtil.randomRegexp(random());
if (VERBOSE) {

View File

@ -48,8 +48,6 @@ public class TestSearchAfter extends LuceneTestCase {
private Directory dir;
private IndexReader reader;
private IndexSearcher searcher;
boolean supportsDocValues = Codec.getDefault().getName().equals("Lucene3x") == false;
private static SortField useDocValues(SortField field) {
field.setUseIndexValues(true);
@ -77,13 +75,11 @@ public class TestSearchAfter extends LuceneTestCase {
document.add(newField("bytesval", _TestUtil.randomRealisticUnicodeString(random()), StringField.TYPE_UNSTORED));
document.add(new DoubleField("double", random().nextDouble()));
if (supportsDocValues) {
document.add(new IntDocValuesField("intdocvalues", random().nextInt()));
document.add(new FloatDocValuesField("floatdocvalues", random().nextFloat()));
document.add(new SortedBytesDocValuesField("sortedbytesdocvalues", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
document.add(new SortedBytesDocValuesField("sortedbytesdocvaluesval", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
document.add(new StraightBytesDocValuesField("straightbytesdocvalues", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
}
document.add(new IntDocValuesField("intdocvalues", random().nextInt()));
document.add(new FloatDocValuesField("floatdocvalues", random().nextFloat()));
document.add(new SortedBytesDocValuesField("sortedbytesdocvalues", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
document.add(new SortedBytesDocValuesField("sortedbytesdocvaluesval", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
document.add(new StraightBytesDocValuesField("straightbytesdocvalues", new BytesRef(_TestUtil.randomRealisticUnicodeString(random()))));
iw.addDocument(document);
}
@ -131,13 +127,11 @@ public class TestSearchAfter extends LuceneTestCase {
assertQuery(query, filter, new Sort(new SortField[] {new SortField("double", SortField.Type.DOUBLE, reversed)}));
assertQuery(query, filter, new Sort(new SortField[] {new SortField("bytes", SortField.Type.STRING, reversed)}));
assertQuery(query, filter, new Sort(new SortField[] {new SortField("bytesval", SortField.Type.STRING_VAL, reversed)}));
if (supportsDocValues) {
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("intdocvalues", SortField.Type.INT, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("floatdocvalues", SortField.Type.FLOAT, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("sortedbytesdocvalues", SortField.Type.STRING, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("sortedbytesdocvaluesval", SortField.Type.STRING_VAL, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("straightbytesdocvalues", SortField.Type.STRING_VAL, reversed))}));
}
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("intdocvalues", SortField.Type.INT, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("floatdocvalues", SortField.Type.FLOAT, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("sortedbytesdocvalues", SortField.Type.STRING, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("sortedbytesdocvaluesval", SortField.Type.STRING_VAL, reversed))}));
assertQuery(query, filter, new Sort(new SortField[] {useDocValues(new SortField("straightbytesdocvalues", SortField.Type.STRING_VAL, reversed))}));
}
}

View File

@ -74,8 +74,6 @@ import org.junit.BeforeClass;
*/
public class TestSort extends LuceneTestCase {
// true if our codec supports docvalues: true unless codec is preflex (3.x)
boolean supportsDocValues = Codec.getDefault().getName().equals("Lucene3x") == false;
private static int NUM_STRINGS;
private IndexSearcher full;
private IndexSearcher searchX;
@ -158,20 +156,15 @@ public class TestSort extends LuceneTestCase {
doc.add (new TextField ("contents", data[i][1]));
if (data[i][2] != null) {
doc.add(new StringField ("int", data[i][2]));
if (supportsDocValues) {
doc.add(new PackedLongDocValuesField("int", Integer.parseInt(data[i][2])));
}
doc.add(new PackedLongDocValuesField("int", Integer.parseInt(data[i][2])));
}
if (data[i][3] != null) {
doc.add(new StringField ("float", data[i][3]));
if (supportsDocValues) {
doc.add(new FloatDocValuesField("float", Float.parseFloat(data[i][3])));
}
doc.add(new FloatDocValuesField("float", Float.parseFloat(data[i][3])));
}
if (data[i][4] != null) {
doc.add(new StringField ("string", data[i][4]));
if (supportsDocValues) {
switch(stringDVType) {
switch(stringDVType) {
case BYTES_FIXED_SORTED:
doc.add(new SortedBytesDocValuesField("string", new BytesRef(data[i][4]), true));
break;
@ -192,7 +185,6 @@ public class TestSort extends LuceneTestCase {
break;
default:
throw new IllegalStateException("unknown type " + stringDVType);
}
}
}
if (data[i][5] != null) doc.add (new StringField ("custom", data[i][5]));
@ -200,9 +192,7 @@ public class TestSort extends LuceneTestCase {
if (data[i][7] != null) doc.add (new StringField ("long", data[i][7]));
if (data[i][8] != null) {
doc.add(new StringField ("double", data[i][8]));
if (supportsDocValues) {
doc.add(new DoubleDocValuesField("double", Double.parseDouble(data[i][8])));
}
doc.add(new DoubleDocValuesField("double", Double.parseDouble(data[i][8])));
}
if (data[i][9] != null) doc.add (new StringField ("short", data[i][9]));
if (data[i][10] != null) doc.add (new StringField ("byte", data[i][10]));
@ -246,14 +236,10 @@ public class TestSort extends LuceneTestCase {
doc.add (new Field ("tracer", num, onlyStored));
//doc.add (new Field ("contents", Integer.toString(i), Field.Store.NO, Field.Index.ANALYZED));
doc.add(new StringField("string", num));
if (supportsDocValues) {
doc.add(new SortedBytesDocValuesField("string", new BytesRef(num)));
}
doc.add(new SortedBytesDocValuesField("string", new BytesRef(num)));
String num2 = getRandomCharString(getRandomNumber(1, 4), 48, 50);
doc.add(new StringField ("string2", num2));
if (supportsDocValues) {
doc.add(new SortedBytesDocValuesField("string2", new BytesRef(num2)));
}
doc.add(new SortedBytesDocValuesField("string2", new BytesRef(num2)));
doc.add (new Field ("tracer2", num2, onlyStored));
for(IndexableField f2 : doc.getFields()) {
if (!f2.fieldType().omitNorms()) {
@ -265,14 +251,10 @@ public class TestSort extends LuceneTestCase {
doc.add (new Field ("fixed_tracer", numFixed, onlyStored));
//doc.add (new Field ("contents", Integer.toString(i), Field.Store.NO, Field.Index.ANALYZED));
doc.add(new StringField("string_fixed", numFixed));
if (supportsDocValues) {
doc.add(new SortedBytesDocValuesField("string_fixed", new BytesRef(numFixed), true));
}
doc.add(new SortedBytesDocValuesField("string_fixed", new BytesRef(numFixed), true));
String num2Fixed = getRandomCharString(fixedLen2, 48, 52);
doc.add(new StringField ("string2_fixed", num2Fixed));
if (supportsDocValues) {
doc.add(new SortedBytesDocValuesField("string2_fixed", new BytesRef(num2Fixed), true));
}
doc.add(new SortedBytesDocValuesField("string2_fixed", new BytesRef(num2Fixed), true));
doc.add (new Field ("tracer2_fixed", num2Fixed, onlyStored));
for(IndexableField f2 : doc.getFields()) {
@ -412,23 +394,21 @@ public class TestSort extends LuceneTestCase {
assertMatches (full, queryX, sort, "AIGEC");
assertMatches (full, queryY, sort, "DJHFB");
if (supportsDocValues) {
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "IGAEC");
assertMatches (full, queryY, sort, "DHFJB");
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "IGAEC");
assertMatches (full, queryY, sort, "DHFJB");
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "GCIEA");
assertMatches (full, queryY, sort, "DHJFB");
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "GCIEA");
assertMatches (full, queryY, sort, "DHJFB");
sort.setSort (useDocValues(new SortField ("double", SortField.Type.DOUBLE)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "AGICE");
assertMatches (full, queryY, sort, "DJHBF");
sort.setSort (useDocValues(new SortField ("double", SortField.Type.DOUBLE)), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "AGICE");
assertMatches (full, queryY, sort, "DJHBF");
sort.setSort (useDocValues(new SortField ("string", getDVStringSortType())), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "AIGEC");
assertMatches (full, queryY, sort, "DJHFB");
}
sort.setSort (useDocValues(new SortField ("string", getDVStringSortType())), SortField.FIELD_DOC );
assertMatches (full, queryX, sort, "AIGEC");
assertMatches (full, queryY, sort, "DJHFB");
}
private SortField.Type getDVStringSortType() {
@ -520,8 +500,6 @@ public class TestSort extends LuceneTestCase {
verifyStringSort(sort);
// Doc values field, var length
assumeFalse("cannot work with preflex codec",
"Lucene3x".equals(Codec.getDefault().getName()));
sort.setSort(
useDocValues(new SortField("string", getDVStringSortType())),
useDocValues(new SortField("string2", getDVStringSortType(), true)),
@ -781,19 +759,17 @@ public class TestSort extends LuceneTestCase {
assertMatches (full, queryX, sort, "CEGIA");
assertMatches (full, queryY, sort, "BFHJD");
if (supportsDocValues) {
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT, true)) );
assertMatches (full, queryX, sort, "CAEGI");
assertMatches (full, queryY, sort, "BJFHD");
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT, true)) );
assertMatches (full, queryX, sort, "CAEGI");
assertMatches (full, queryY, sort, "BJFHD");
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT, true)) );
assertMatches (full, queryX, sort, "AECIG");
assertMatches (full, queryY, sort, "BFJHD");
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT, true)) );
assertMatches (full, queryX, sort, "AECIG");
assertMatches (full, queryY, sort, "BFJHD");
sort.setSort (useDocValues(new SortField ("string", getDVStringSortType(), true)) );
assertMatches (full, queryX, sort, "CEGIA");
assertMatches (full, queryY, sort, "BFHJD");
}
sort.setSort (useDocValues(new SortField ("string", getDVStringSortType(), true)) );
assertMatches (full, queryX, sort, "CEGIA");
assertMatches (full, queryY, sort, "BFHJD");
}
// test sorting when the sort field is empty (undefined) for some of the documents
@ -864,19 +840,17 @@ public class TestSort extends LuceneTestCase {
sort.setSort (new SortField ("float", SortField.Type.FLOAT), new SortField ("string", SortField.Type.STRING) );
assertMatches (full, queryX, sort, "GICEA");
if (supportsDocValues) {
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT)),
useDocValues(new SortField ("float", SortField.Type.FLOAT)));
assertMatches (full, queryX, sort, "IGEAC");
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT)),
useDocValues(new SortField ("float", SortField.Type.FLOAT)));
assertMatches (full, queryX, sort, "IGEAC");
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT, true)),
useDocValues(new SortField (null, SortField.Type.DOC, true)));
assertMatches (full, queryX, sort, "CEAGI");
sort.setSort (useDocValues(new SortField ("int", SortField.Type.INT, true)),
useDocValues(new SortField (null, SortField.Type.DOC, true)));
assertMatches (full, queryX, sort, "CEAGI");
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT)),
useDocValues(new SortField ("string", getDVStringSortType())));
assertMatches (full, queryX, sort, "GICEA");
}
sort.setSort (useDocValues(new SortField ("float", SortField.Type.FLOAT)),
useDocValues(new SortField ("string", getDVStringSortType())));
assertMatches (full, queryX, sort, "GICEA");
}
// test a variety of sorts using a parallel multisearcher
@ -1189,53 +1163,51 @@ public class TestSort extends LuceneTestCase {
sort.setSort(new SortField ("string", SortField.Type.STRING, true));
assertMatches(multi, queryF, sort, "IJZ");
if (supportsDocValues) {
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)));
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)));
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)), SortField.FIELD_DOC);
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)), SortField.FIELD_DOC);
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT)));
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT)));
expected = isFull ? "IDHFGJABEC" : "IDHFGJAEBC";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField ("float", SortField.Type.FLOAT)), SortField.FIELD_DOC);
assertMatches(multi, queryA, sort, "GDHJCIEFAB");
sort.setSort(useDocValues(new SortField ("float", SortField.Type.FLOAT)), SortField.FIELD_DOC);
assertMatches(multi, queryA, sort, "GDHJCIEFAB");
sort.setSort(useDocValues(new SortField("float", SortField.Type.FLOAT)));
assertMatches(multi, queryA, sort, "GDHJCIEFAB");
sort.setSort(useDocValues(new SortField("float", SortField.Type.FLOAT)));
assertMatches(multi, queryA, sort, "GDHJCIEFAB");
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT, true)));
expected = isFull ? "CABEJGFHDI" : "CAEBJGFHDI";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT, true)));
expected = isFull ? "CABEJGFHDI" : "CAEBJGFHDI";
assertMatches(multi, queryA, sort, expected);
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT)), useDocValues(new SortField("float", SortField.Type.FLOAT)));
assertMatches(multi, queryA, sort, "IDHFGJEABC");
sort.setSort(useDocValues(new SortField("int", SortField.Type.INT)), useDocValues(new SortField("float", SortField.Type.FLOAT)));
assertMatches(multi, queryA, sort, "IDHFGJEABC");
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)));
assertMatches(multi, queryF, sort, "IZJ");
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT)));
assertMatches(multi, queryF, sort, "IZJ");
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT, true)));
assertMatches(multi, queryF, sort, "JZI");
sort.setSort(useDocValues(new SortField ("int", SortField.Type.INT, true)));
assertMatches(multi, queryF, sort, "JZI");
sort.setSort(useDocValues(new SortField("string", getDVStringSortType())));
assertMatches(multi, queryA, sort, "DJAIHGFEBC");
sort.setSort(useDocValues(new SortField("string", getDVStringSortType())));
assertMatches(multi, queryA, sort, "DJAIHGFEBC");
sort.setSort(useDocValues(new SortField("string", getDVStringSortType(), true)));
assertMatches(multi, queryA, sort, "CBEFGHIAJD");
sort.setSort(useDocValues(new SortField("string", getDVStringSortType(), true)));
assertMatches(multi, queryA, sort, "CBEFGHIAJD");
sort.setSort(useDocValues(new SortField("float", SortField.Type.FLOAT)),useDocValues(new SortField("string", getDVStringSortType())));
assertMatches(multi, queryA, sort, "GDHJICEFAB");
sort.setSort(useDocValues(new SortField("float", SortField.Type.FLOAT)),useDocValues(new SortField("string", getDVStringSortType())));
assertMatches(multi, queryA, sort, "GDHJICEFAB");
sort.setSort(useDocValues(new SortField ("string", getDVStringSortType())));
assertMatches(multi, queryF, sort, "ZJI");
sort.setSort(useDocValues(new SortField ("string", getDVStringSortType())));
assertMatches(multi, queryF, sort, "ZJI");
sort.setSort(useDocValues(new SortField ("string", getDVStringSortType(), true)));
assertMatches(multi, queryF, sort, "IJZ");
}
sort.setSort(useDocValues(new SortField ("string", getDVStringSortType(), true)));
assertMatches(multi, queryF, sort, "IJZ");
// up to this point, all of the searches should have "sane"
// FieldCache behavior, and should have reused hte cache in several cases
@ -1370,8 +1342,6 @@ public class TestSort extends LuceneTestCase {
public void testRandomStringSort() throws Exception {
Random random = new Random(random().nextLong());
assumeTrue("cannot work with Lucene3x codec",
defaultCodecSupportsDocValues());
final int NUM_DOCS = atLeast(100);
final Directory dir = newDirectory();

View File

@ -559,9 +559,6 @@ public class TestSimilarityBase extends LuceneTestCase {
/** Test whether all similarities return document 3 before documents 7 and 8. */
public void testHeartRanking() throws IOException {
assumeFalse("PreFlex codec does not support the stats necessary for this test!",
"Lucene3x".equals(Codec.getDefault().getName()));
Query q = new TermQuery(new Term(FIELD_BODY, "heart"));
for (SimilarityBase sim : sims) {

View File

@ -48,8 +48,7 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
NRTCachingDirectory cachedDir = new NRTCachingDirectory(dir, 2.0, 25.0);
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
RandomIndexWriter w = new RandomIndexWriter(random(), cachedDir, conf);
final LineFileDocs docs = new LineFileDocs(random(),
defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random(), true);
final int numDocs = _TestUtil.nextInt(random(), 100, 400);
if (VERBOSE) {

View File

@ -1097,7 +1097,7 @@ public class TestFSTs extends LuceneTestCase {
Codec.setDefault(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat()));
}
final LineFileDocs docs = new LineFileDocs(random(), defaultCodecSupportsDocValues());
final LineFileDocs docs = new LineFileDocs(random(), true);
final int RUN_TIME_MSEC = atLeast(500);
final IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMaxBufferedDocs(-1).setRAMBufferSizeMB(64);
final File tempDir = _TestUtil.getTempDir("fstlines");

View File

@ -54,7 +54,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = true;
Type valueType = vts[random().nextInt(vts.length)];
// 0
@ -202,8 +202,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())));
boolean preFlex = "Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = !preFlex;
boolean canUseIDV = true;
Type valueType = vts[random().nextInt(vts.length)];
Document doc = new Document();
@ -301,7 +300,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
if (SlowCompositeReaderWrapper.class.isAssignableFrom(s.getIndexReader().getClass())) {
canUseIDV = false;
} else {
canUseIDV = !preFlex;
canUseIDV = true;
}
for (int contentID = 0; contentID < 3; contentID++) {

View File

@ -50,7 +50,7 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = true;
// 0
Document doc = new Document();

View File

@ -61,7 +61,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
boolean canUseDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseDV = true;
DocValues.Type dvType = canUseDV ? dvTypes[random.nextInt(dvTypes.length)] : null;
Document doc = new Document();
@ -417,7 +417,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy())
);
boolean canUseDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseDV = true;
DocValues.Type dvType = canUseDV ? dvTypes[random.nextInt(dvTypes.length)] : null;
int numDocs = 86 + random.nextInt(1087) * RANDOM_MULTIPLIER;

View File

@ -48,7 +48,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
boolean canUseDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseDV = true;
boolean useDv = canUseDV && random().nextBoolean();
// 0
@ -360,7 +360,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
new MockAnalyzer(random)
)
);
boolean canUseDV = !"Lucene3x".equals(writer.w.getConfig().getCodec().getName());
boolean canUseDV = true;
boolean useDv = canUseDV && random.nextBoolean();
Document doc = new Document();

View File

@ -51,7 +51,7 @@ public class GroupingSearchTest extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = true;
List<Document> documents = new ArrayList<Document>();
// 0
Document doc = new Document();

View File

@ -73,7 +73,7 @@ public class TestGrouping extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = true;
// 0
Document doc = new Document();
addGroupField(doc, groupField, "author1", canUseIDV);
@ -701,8 +701,7 @@ public class TestGrouping extends LuceneTestCase {
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())));
final boolean preFlex = "Lucene3x".equals(w.w.getConfig().getCodec().getName());
boolean canUseIDV = !preFlex;
boolean canUseIDV = true;
Document doc = new Document();
Document docNoGroup = new Document();
@ -778,7 +777,7 @@ public class TestGrouping extends LuceneTestCase {
if (SlowCompositeReaderWrapper.class.isAssignableFrom(s.getIndexReader().getClass())) {
canUseIDV = false;
} else {
canUseIDV = !preFlex;
canUseIDV = true;
}
final ShardState shards = new ShardState(s);
@ -953,7 +952,7 @@ public class TestGrouping extends LuceneTestCase {
ValueHolder<Boolean> idvBasedImplsUsedSharded = new ValueHolder<Boolean>(false);
final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort,
groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, canUseIDV, preFlex, idvBasedImplsUsedSharded);
groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, canUseIDV, false, idvBasedImplsUsedSharded);
final AbstractSecondPassGroupingCollector<?> c2;
if (topGroups != null) {

View File

@ -44,7 +44,7 @@ public class TestBalancedSegmentMergePolicy extends LuceneTestCase {
mp.setMergePolicyParams(newMergePolicyParams(random()));
iwc.setMergePolicy(mp);
iw = new RandomIndexWriter(random(), dir, iwc);
LineFileDocs docs = new LineFileDocs(random(), !Codec.getDefault().getName().equals("Lucene3x"));
LineFileDocs docs = new LineFileDocs(random(), true);
int numDocs = atLeast(200);
for (int i = 0; i < numDocs; i++) {
iw.addDocument(docs.nextDoc());

View File

@ -311,13 +311,8 @@ public class TestValueSources extends LuceneTestCase {
}
public void testSumTotalTermFreq() throws Exception {
if (Codec.getDefault().getName().equals("Lucene3x")) {
assertHits(new FunctionQuery(new SumTotalTermFreqValueSource("text")),
new float[] { -1f, -1f });
} else {
assertHits(new FunctionQuery(new SumTotalTermFreqValueSource("text")),
assertHits(new FunctionQuery(new SumTotalTermFreqValueSource("text")),
new float[] { 8f, 8f });
}
}
public void testTermFreq() throws Exception {
@ -346,15 +341,9 @@ public class TestValueSources extends LuceneTestCase {
}
public void testTotalTermFreq() throws Exception {
if (Codec.getDefault().getName().equals("Lucene3x")) {
assertHits(new FunctionQuery(
new TotalTermFreqValueSource("bogus", "bogus", "text", new BytesRef("test"))),
new float[] { -1f, -1f });
} else {
assertHits(new FunctionQuery(
new TotalTermFreqValueSource("bogus", "bogus", "text", new BytesRef("test"))),
new float[] { 4f, 4f });
}
assertHits(new FunctionQuery(
new TotalTermFreqValueSource("bogus", "bogus", "text", new BytesRef("test"))),
new float[] { 4f, 4f });
}
void assertHits(Query q, float scores[]) throws Exception {

View File

@ -1,106 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.util.LuceneTestCase;
/**
* Writes 3.x-like indexes (not perfect emulation yet) for testing only!
* @lucene.experimental
*/
public class PreFlexRWCodec extends Lucene3xCodec {
private final PostingsFormat postings = new PreFlexRWPostingsFormat();
private final Lucene3xNormsFormat norms = new PreFlexRWNormsFormat();
private final FieldInfosFormat fieldInfos = new PreFlexRWFieldInfosFormat();
private final TermVectorsFormat termVectors = new PreFlexRWTermVectorsFormat();
private final SegmentInfoFormat segmentInfos = new PreFlexRWSegmentInfoFormat();
private final StoredFieldsFormat storedFields = new PreFlexRWStoredFieldsFormat();
// TODO: this should really be a different impl
private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
@Override
public PostingsFormat postingsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return postings;
} else {
return super.postingsFormat();
}
}
@Override
public NormsFormat normsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return norms;
} else {
return super.normsFormat();
}
}
@Override
public SegmentInfoFormat segmentInfoFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return segmentInfos ;
} else {
return super.segmentInfoFormat();
}
}
@Override
public FieldInfosFormat fieldInfosFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return fieldInfos;
} else {
return super.fieldInfosFormat();
}
}
@Override
public TermVectorsFormat termVectorsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return termVectors;
} else {
return super.termVectorsFormat();
}
}
@Override
public LiveDocsFormat liveDocsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return liveDocs;
} else {
return super.liveDocsFormat();
}
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return storedFields;
} else {
return super.storedFieldsFormat();
}
}
}

View File

@ -1,41 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
/**
*
* @lucene.internal
* @lucene.experimental
*/
class PreFlexRWFieldInfosFormat extends Lucene3xFieldInfosFormat {
@Override
public FieldInfosReader getFieldInfosReader() throws IOException {
return new PreFlexRWFieldInfosReader();
}
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
return new PreFlexRWFieldInfosWriter();
}
}

View File

@ -1,112 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.Set;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
/**
* @lucene.internal
* @lucene.experimental
*/
class PreFlexRWFieldInfosReader extends FieldInfosReader {
static final int FORMAT_MINIMUM = PreFlexRWFieldInfosWriter.FORMAT_START;
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", PreFlexRWFieldInfosWriter.FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
try {
final int format = input.readVInt();
if (format > FORMAT_MINIMUM) {
throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, PreFlexRWFieldInfosWriter.FORMAT_CURRENT);
}
if (format < PreFlexRWFieldInfosWriter.FORMAT_CURRENT && format != PreFlexRWFieldInfosWriter.FORMAT_PREFLEX_RW) {
throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, PreFlexRWFieldInfosWriter.FORMAT_CURRENT);
}
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
for (int i = 0; i < size; i++) {
String name = input.readString();
final int fieldNumber = format == PreFlexRWFieldInfosWriter.FORMAT_PREFLEX_RW ? input.readInt() : i;
byte bits = input.readByte();
boolean isIndexed = (bits & PreFlexRWFieldInfosWriter.IS_INDEXED) != 0;
boolean storeTermVector = (bits & PreFlexRWFieldInfosWriter.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & PreFlexRWFieldInfosWriter.OMIT_NORMS) != 0;
boolean storePayloads = (bits & PreFlexRWFieldInfosWriter.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & PreFlexRWFieldInfosWriter.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & PreFlexRWFieldInfosWriter.OMIT_POSITIONS) != 0) {
if (format <= PreFlexRWFieldInfosWriter.FORMAT_OMIT_POSITIONS) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else {
throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
}
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
}
// LUCENE-3027: past indices were able to write
// storePayloads=true when omitTFAP is also true,
// which is invalid. We correct that, here:
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
storePayloads = false;
}
Type normType = isIndexed && !omitNorms ? Type.FIXED_INTS_8 : null;
if (format == PreFlexRWFieldInfosWriter.FORMAT_PREFLEX_RW && normType != null) {
// RW can have norms but doesn't write them
normType = input.readByte() != 0 ? Type.FIXED_INTS_8 : null;
}
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, null, normType, null);
}
if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
return new FieldInfos(infos);
} finally {
input.close();
}
}
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexRWFieldInfosWriter.FIELD_INFOS_EXTENSION));
}
}

View File

@ -1,98 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.FieldInfosWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
/**
* @lucene.internal
* @lucene.experimental
*/
class PreFlexRWFieldInfosWriter extends FieldInfosWriter {
// TODO move to test-framework preflex RW?
/** Extension of field infos */
static final String FIELD_INFOS_EXTENSION = "fnm";
// First used in 2.9; prior to 2.9 there was no format header
static final int FORMAT_START = -2;
// First used in 3.4: omit only positional information
static final int FORMAT_OMIT_POSITIONS = -3;
static final int FORMAT_PREFLEX_RW = Integer.MIN_VALUE;
// whenever you add a new format, make it 1 smaller (negative version logic)!
static final int FORMAT_CURRENT = FORMAT_OMIT_POSITIONS;
static final byte IS_INDEXED = 0x1;
static final byte STORE_TERMVECTOR = 0x2;
static final byte OMIT_NORMS = 0x10;
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
static final byte OMIT_POSITIONS = -128;
@Override
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
try {
output.writeVInt(FORMAT_PREFLEX_RW);
output.writeVInt(infos.size());
for (FieldInfo fi : infos) {
byte bits = 0x0;
if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
if (fi.omitsNorms()) bits |= OMIT_NORMS;
if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
if (fi.isIndexed()) {
bits |= IS_INDEXED;
assert fi.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.hasPayloads();
if (fi.getIndexOptions() == IndexOptions.DOCS_ONLY) {
bits |= OMIT_TERM_FREQ_AND_POSITIONS;
} else if (fi.getIndexOptions() == IndexOptions.DOCS_AND_FREQS) {
bits |= OMIT_POSITIONS;
}
}
output.writeString(fi.name);
/*
* we need to write the field number since IW tries
* to stabelize the field numbers across segments so the
* FI ordinal is not necessarily equivalent to the field number
*/
output.writeInt(fi.number);
output.writeByte(bits);
if (fi.isIndexed() && !fi.omitsNorms()) {
// to allow null norm types we need to indicate if norms are written
// only in RW case
output.writeByte((byte) (fi.getNormType() == null ? 0 : 1));
}
assert fi.attributes() == null; // not used or supported
}
} finally {
output.close();
}
}
}

View File

@ -1,224 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.Comparator;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
class PreFlexRWFieldsWriter extends FieldsConsumer {
private final TermInfosWriter termsOut;
private final IndexOutput freqOut;
private final IndexOutput proxOut;
private final PreFlexRWSkipListWriter skipListWriter;
private final int totalNumDocs;
public PreFlexRWFieldsWriter(SegmentWriteState state) throws IOException {
termsOut = new TermInfosWriter(state.directory,
state.segmentInfo.name,
state.fieldInfos,
state.termIndexInterval);
boolean success = false;
try {
final String freqFile = IndexFileNames.segmentFileName(state.segmentInfo.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION);
freqOut = state.directory.createOutput(freqFile, state.context);
totalNumDocs = state.segmentInfo.getDocCount();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(termsOut);
}
}
success = false;
try {
if (state.fieldInfos.hasProx()) {
final String proxFile = IndexFileNames.segmentFileName(state.segmentInfo.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
proxOut = state.directory.createOutput(proxFile, state.context);
} else {
proxOut = null;
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(termsOut, freqOut);
}
}
skipListWriter = new PreFlexRWSkipListWriter(termsOut.skipInterval,
termsOut.maxSkipLevels,
totalNumDocs,
freqOut,
proxOut);
//System.out.println("\nw start seg=" + segment);
}
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
assert field.number != -1;
if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
throw new UnsupportedOperationException("this codec cannot index offsets");
}
//System.out.println("w field=" + field.name + " storePayload=" + field.storePayloads + " number=" + field.number);
return new PreFlexTermsWriter(field);
}
@Override
public void close() throws IOException {
IOUtils.close(termsOut, freqOut, proxOut);
}
private class PreFlexTermsWriter extends TermsConsumer {
private final FieldInfo fieldInfo;
private final boolean omitTF;
private final boolean storePayloads;
private final TermInfo termInfo = new TermInfo();
private final PostingsWriter postingsWriter = new PostingsWriter();
public PreFlexTermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
omitTF = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY;
storePayloads = fieldInfo.hasPayloads();
}
private class PostingsWriter extends PostingsConsumer {
private int lastDocID;
private int lastPayloadLength = -1;
private int lastPosition;
private int df;
public PostingsWriter reset() {
df = 0;
lastDocID = 0;
lastPayloadLength = -1;
return this;
}
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
//System.out.println(" w doc=" + docID);
final int delta = docID - lastDocID;
if (docID < 0 || (df > 0 && delta <= 0)) {
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
}
if ((++df % termsOut.skipInterval) == 0) {
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
skipListWriter.bufferSkip(df);
}
lastDocID = docID;
assert docID < totalNumDocs: "docID=" + docID + " totalNumDocs=" + totalNumDocs;
if (omitTF) {
freqOut.writeVInt(delta);
} else {
final int code = delta << 1;
if (termDocFreq == 1) {
freqOut.writeVInt(code|1);
} else {
freqOut.writeVInt(code);
freqOut.writeVInt(termDocFreq);
}
}
lastPosition = 0;
}
@Override
public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
assert proxOut != null;
assert startOffset == -1;
assert endOffset == -1;
//System.out.println(" w pos=" + position + " payl=" + payload);
final int delta = position - lastPosition;
lastPosition = position;
if (storePayloads) {
final int payloadLength = payload == null ? 0 : payload.length;
if (payloadLength != lastPayloadLength) {
//System.out.println(" write payload len=" + payloadLength);
lastPayloadLength = payloadLength;
proxOut.writeVInt((delta<<1)|1);
proxOut.writeVInt(payloadLength);
} else {
proxOut.writeVInt(delta << 1);
}
if (payloadLength > 0) {
proxOut.writeBytes(payload.bytes, payload.offset, payload.length);
}
} else {
proxOut.writeVInt(delta);
}
}
@Override
public void finishDoc() throws IOException {
}
}
@Override
public PostingsConsumer startTerm(BytesRef text) throws IOException {
//System.out.println(" w term=" + text.utf8ToString());
skipListWriter.resetSkip();
termInfo.freqPointer = freqOut.getFilePointer();
if (proxOut != null) {
termInfo.proxPointer = proxOut.getFilePointer();
}
return postingsWriter.reset();
}
@Override
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
if (stats.docFreq > 0) {
long skipPointer = skipListWriter.writeSkip(freqOut);
termInfo.docFreq = stats.docFreq;
termInfo.skipOffset = (int) (skipPointer - termInfo.freqPointer);
//System.out.println(" w finish term=" + text.utf8ToString() + " fnum=" + fieldInfo.number);
termsOut.add(fieldInfo.number,
text,
termInfo);
}
}
@Override
public void finish(long sumTotalTermCount, long sumDocFreq, int docCount) throws IOException {
}
@Override
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUTF16Comparator();
}
}
}

View File

@ -1,293 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
/**
* Writes and Merges Lucene 3.x norms format
* @lucene.experimental
*/
class PreFlexRWNormsConsumer extends PerDocConsumer {
/** norms header placeholder */
private static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
/** Extension of norms file */
private static final String NORMS_EXTENSION = "nrm";
/** Extension of separate norms file
* @deprecated */
@Deprecated
private static final String SEPARATE_NORMS_EXTENSION = "s";
private final Directory directory;
private final String segment;
private final IOContext context;
private NormsWriter writer;
public PreFlexRWNormsConsumer(Directory directory, String segment, IOContext context){
this.directory = directory;
this.segment = segment;
this.context = context;
}
@Override
public void merge(MergeState mergeState) throws IOException {
getNormsWriter().merge(mergeState);
}
@Override
public void close() throws IOException {
if (writer != null) {
writer.finish();
}
}
@Override
protected boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
throws IOException {
if (type != Type.FIXED_INTS_8) {
throw new UnsupportedOperationException("Codec only supports single byte norm values. Type give: " + type);
}
return new Lucene3xNormsDocValuesConsumer(fieldInfo);
}
class Lucene3xNormsDocValuesConsumer extends DocValuesConsumer {
// Holds all docID/norm pairs we've seen
private int[] docIDs = new int[1];
private byte[] norms = new byte[1];
private int upto;
private final FieldInfo fi;
Lucene3xNormsDocValuesConsumer(FieldInfo fieldInfo) {
fi = fieldInfo;
}
@Override
public void finish(int docCount) throws IOException {
final NormsWriter normsWriter = getNormsWriter();
boolean success = false;
try {
int uptoDoc = 0;
normsWriter.setNumTotalDocs(docCount);
if (upto > 0) {
normsWriter.startField(fi);
int docID = 0;
for (; docID < docCount; docID++) {
if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
normsWriter.writeNorm(norms[uptoDoc]);
uptoDoc++;
} else {
normsWriter.writeNorm((byte) 0);
}
}
// we should have consumed every norm
assert uptoDoc == upto;
} else {
// Fill entire field with default norm:
normsWriter.startField(fi);
for (; upto < docCount; upto++)
normsWriter.writeNorm((byte) 0);
}
success = true;
} finally {
if (!success) {
normsWriter.abort();
}
}
}
@Override
public void add(int docID, IndexableField docValue) throws IOException {
add(docID, docValue.numericValue().longValue());
}
protected void add(int docID, long value) {
if (docIDs.length <= upto) {
assert docIDs.length == upto;
docIDs = ArrayUtil.grow(docIDs, 1 + upto);
}
if (norms.length <= upto) {
assert norms.length == upto;
norms = ArrayUtil.grow(norms, 1 + upto);
}
norms[upto] = (byte) value;
docIDs[upto] = docID;
upto++;
}
@Override
protected Type getType() {
return Type.FIXED_INTS_8;
}
}
public NormsWriter getNormsWriter() throws IOException {
if (writer == null) {
writer = new NormsWriter(directory, segment, context);
}
return writer;
}
private static class NormsWriter {
private final IndexOutput output;
private int normCount = 0;
private int numTotalDocs = 0;
public NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
boolean success = false;
IndexOutput out = null;
try {
out = directory.createOutput(normsFileName, context);
output = out;
output.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(out);
}
}
}
public void setNumTotalDocs(int numTotalDocs) {
assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
this.numTotalDocs = numTotalDocs;
}
public void startField(FieldInfo info) throws IOException {
assert info.omitsNorms() == false;
normCount++;
}
public void writeNorm(byte norm) throws IOException {
output.writeByte(norm);
}
public void abort() throws IOException {
IOUtils.close(output);
}
public void finish() throws IOException {
IOUtils.close(output);
if (4+normCount*(long)numTotalDocs != output.getFilePointer()) {
throw new IOException(".nrm file size mismatch: expected=" + (4+normCount*(long)numTotalDocs) + " actual=" + output.getFilePointer());
}
}
// TODO: we can actually use the defaul DV merge here and drop this specific stuff entirely
/** we override merge and bulk-merge norms when there are no deletions */
public void merge(MergeState mergeState) throws IOException {
int numMergedDocs = 0;
for (FieldInfo fi : mergeState.fieldInfos) {
if (fi.hasNorms()) {
startField(fi);
int numMergedDocsForField = 0;
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
final int maxDoc = reader.reader.maxDoc();
byte[] normBuffer;
DocValues normValues = reader.reader.normValues(fi.name);
if (normValues == null) {
// Can be null if this segment doesn't have
// any docs with this field
normBuffer = new byte[maxDoc];
Arrays.fill(normBuffer, (byte)0);
} else {
Source directSource = normValues.getDirectSource();
assert directSource.hasArray();
normBuffer = (byte[]) directSource.getArray();
}
if (reader.liveDocs == null) {
//optimized case for segments without deleted docs
output.writeBytes(normBuffer, maxDoc);
numMergedDocsForField += maxDoc;
} else {
// this segment has deleted docs, so we have to
// check for every doc if it is deleted or not
final Bits liveDocs = reader.liveDocs;
for (int k = 0; k < maxDoc; k++) {
if (liveDocs.get(k)) {
numMergedDocsForField++;
output.writeByte(normBuffer[k]);
}
}
}
mergeState.checkAbort.work(maxDoc);
}
assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
numMergedDocs = numMergedDocsForField;
}
}
this.numTotalDocs = numMergedDocs;
}
}
@Override
public void abort() {
try {
try {
if (writer != null) {
writer.abort();
}
} finally {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
NORMS_EXTENSION));
}
} catch (IOException e) {
// ignore
}
}
}

View File

@ -1,35 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.PerDocConsumer;
import org.apache.lucene.index.PerDocWriteState;
/**
* @lucene.internal
* @lucene.experimental
*/
class PreFlexRWNormsFormat extends Lucene3xNormsFormat {
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new PreFlexRWNormsConsumer(state.directory, state.segmentInfo.name, state.context);
}
}

View File

@ -1,74 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.LuceneTestCase;
/** Codec, only for testing, that can write and read the
* pre-flex index format.
*
* @lucene.experimental
*/
class PreFlexRWPostingsFormat extends Lucene3xPostingsFormat {
public PreFlexRWPostingsFormat() {
// NOTE: we impersonate the PreFlex codec so that it can
// read the segments we write!
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new PreFlexRWFieldsWriter(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
// Whenever IW opens readers, eg for merging, we have to
// keep terms order in UTF16:
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
// we are part of a "merge", we must sort by UTF16:
boolean unicodeSortOrder = true;
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
//System.out.println(trace[i].getClassName());
if ("merge".equals(trace[i].getMethodName())) {
unicodeSortOrder = false;
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: PreFlexRW codec: forcing legacy UTF16 term sort order");
}
break;
}
}
return unicodeSortOrder;
}
};
}
}

View File

@ -1,32 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
import org.apache.lucene.codecs.SegmentInfoWriter;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* @lucene.experimental
*/
class PreFlexRWSegmentInfoFormat extends Lucene3xSegmentInfoFormat {
private final SegmentInfoWriter writer = new PreFlexRWSegmentInfoWriter();
@Override
public SegmentInfoWriter getSegmentInfosWriter() {
return writer;
}
}

View File

@ -1,45 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.SegmentInfoWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* PreFlex implementation of {@link SegmentInfoWriter}.
* @lucene.experimental
*/
class PreFlexRWSegmentInfoWriter extends SegmentInfoWriter {
// NOTE: this is not "really" 3.x format, because we are
// writing each SI to its own file, vs 3.x where the list
// of segments and SI for each segment is written into a
// single segments_N file
/** Save a single segment's info. */
@Override
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
SegmentInfos.write3xInfo(dir, si, ioContext);
}
}

View File

@ -1,127 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
/**
* PreFlexRW skiplist implementation.
* @lucene.experimental
*/
public class PreFlexRWSkipListWriter extends MultiLevelSkipListWriter {
private int[] lastSkipDoc;
private int[] lastSkipPayloadLength;
private long[] lastSkipFreqPointer;
private long[] lastSkipProxPointer;
private IndexOutput freqOutput;
private IndexOutput proxOutput;
private int curDoc;
private boolean curStorePayloads;
private int curPayloadLength;
private long curFreqPointer;
private long curProxPointer;
public PreFlexRWSkipListWriter(int skipInterval, int numberOfSkipLevels, int docCount, IndexOutput freqOutput, IndexOutput proxOutput) {
super(skipInterval, numberOfSkipLevels, docCount);
this.freqOutput = freqOutput;
this.proxOutput = proxOutput;
lastSkipDoc = new int[numberOfSkipLevels];
lastSkipPayloadLength = new int[numberOfSkipLevels];
lastSkipFreqPointer = new long[numberOfSkipLevels];
lastSkipProxPointer = new long[numberOfSkipLevels];
}
/**
* Sets the values for the current skip data.
*/
public void setSkipData(int doc, boolean storePayloads, int payloadLength) {
this.curDoc = doc;
this.curStorePayloads = storePayloads;
this.curPayloadLength = payloadLength;
this.curFreqPointer = freqOutput.getFilePointer();
if (proxOutput != null)
this.curProxPointer = proxOutput.getFilePointer();
}
@Override
public void resetSkip() {
super.resetSkip();
Arrays.fill(lastSkipDoc, 0);
Arrays.fill(lastSkipPayloadLength, -1); // we don't have to write the first length in the skip list
Arrays.fill(lastSkipFreqPointer, freqOutput.getFilePointer());
if (proxOutput != null)
Arrays.fill(lastSkipProxPointer, proxOutput.getFilePointer());
}
@Override
protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
// To efficiently store payloads in the posting lists we do not store the length of
// every payload. Instead we omit the length for a payload if the previous payload had
// the same length.
// However, in order to support skipping the payload length at every skip point must be known.
// So we use the same length encoding that we use for the posting lists for the skip data as well:
// Case 1: current field does not store payloads
// SkipDatum --> DocSkip, FreqSkip, ProxSkip
// DocSkip,FreqSkip,ProxSkip --> VInt
// DocSkip records the document number before every SkipInterval th document in TermFreqs.
// Document numbers are represented as differences from the previous value in the sequence.
// Case 2: current field stores payloads
// SkipDatum --> DocSkip, PayloadLength?, FreqSkip,ProxSkip
// DocSkip,FreqSkip,ProxSkip --> VInt
// PayloadLength --> VInt
// In this case DocSkip/2 is the difference between
// the current and the previous value. If DocSkip
// is odd, then a PayloadLength encoded as VInt follows,
// if DocSkip is even, then it is assumed that the
// current payload length equals the length at the previous
// skip point
if (curStorePayloads) {
int delta = curDoc - lastSkipDoc[level];
if (curPayloadLength == lastSkipPayloadLength[level]) {
// the current payload length equals the length at the previous skip point,
// so we don't store the length again
skipBuffer.writeVInt(delta * 2);
} else {
// the payload length is different from the previous one. We shift the DocSkip,
// set the lowest bit and store the current payload length as VInt.
skipBuffer.writeVInt(delta * 2 + 1);
skipBuffer.writeVInt(curPayloadLength);
lastSkipPayloadLength[level] = curPayloadLength;
}
} else {
// current field does not store payloads
skipBuffer.writeVInt(curDoc - lastSkipDoc[level]);
}
skipBuffer.writeVInt((int) (curFreqPointer - lastSkipFreqPointer[level]));
skipBuffer.writeVInt((int) (curProxPointer - lastSkipProxPointer[level]));
lastSkipDoc[level] = curDoc;
lastSkipFreqPointer[level] = curFreqPointer;
lastSkipProxPointer[level] = curProxPointer;
}
}

View File

@ -1,33 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.StoredFieldsWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
class PreFlexRWStoredFieldsFormat extends Lucene3xStoredFieldsFormat {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new PreFlexRWStoredFieldsWriter(directory, segmentInfo.name, context);
}
}

View File

@ -1,156 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed 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.StoredFieldsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** @lucene.experimental */
final class PreFlexRWStoredFieldsWriter extends StoredFieldsWriter {
private final Directory directory;
private final String segment;
private IndexOutput fieldsStream;
private IndexOutput indexStream;
public PreFlexRWStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
assert directory != null;
this.directory = directory;
this.segment = segment;
boolean success = false;
try {
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION), context);
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION), context);
fieldsStream.writeInt(Lucene3xStoredFieldsReader.FORMAT_CURRENT);
indexStream.writeInt(Lucene3xStoredFieldsReader.FORMAT_CURRENT);
success = true;
} finally {
if (!success) {
abort();
}
}
}
// Writes the contents of buffer into the fields stream
// and adds a new entry for this document into the index
// stream. This assumes the buffer was already written
// in the correct fields format.
public void startDocument(int numStoredFields) throws IOException {
indexStream.writeLong(fieldsStream.getFilePointer());
fieldsStream.writeVInt(numStoredFields);
}
public void close() throws IOException {
try {
IOUtils.close(fieldsStream, indexStream);
} finally {
fieldsStream = indexStream = null;
}
}
public void abort() {
try {
close();
} catch (IOException ignored) {}
IOUtils.deleteFilesIgnoringExceptions(directory,
IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_EXTENSION),
IndexFileNames.segmentFileName(segment, "", Lucene3xStoredFieldsReader.FIELDS_INDEX_EXTENSION));
}
public void writeField(FieldInfo info, IndexableField field) throws IOException {
fieldsStream.writeVInt(info.number);
int bits = 0;
final BytesRef bytes;
final String string;
// TODO: maybe a field should serialize itself?
// this way we don't bake into indexer all these
// specific encodings for different fields? and apps
// can customize...
Number number = field.numericValue();
if (number != null) {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_INT;
} else if (number instanceof Long) {
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_LONG;
} else if (number instanceof Float) {
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_FLOAT;
} else if (number instanceof Double) {
bits |= Lucene3xStoredFieldsReader.FIELD_IS_NUMERIC_DOUBLE;
} else {
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
}
string = null;
bytes = null;
} else {
bytes = field.binaryValue();
if (bytes != null) {
bits |= Lucene3xStoredFieldsReader.FIELD_IS_BINARY;
string = null;
} else {
string = field.stringValue();
if (string == null) {
throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
}
}
}
fieldsStream.writeByte((byte) bits);
if (bytes != null) {
fieldsStream.writeVInt(bytes.length);
fieldsStream.writeBytes(bytes.bytes, bytes.offset, bytes.length);
} else if (string != null) {
fieldsStream.writeString(field.stringValue());
} else {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
fieldsStream.writeInt(number.intValue());
} else if (number instanceof Long) {
fieldsStream.writeLong(number.longValue());
} else if (number instanceof Float) {
fieldsStream.writeInt(Float.floatToIntBits(number.floatValue()));
} else if (number instanceof Double) {
fieldsStream.writeLong(Double.doubleToLongBits(number.doubleValue()));
} else {
assert false;
}
}
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
if (4+((long) numDocs)*8 != indexStream.getFilePointer())
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and
// throw an exception to prevent the corruption from
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("fdx size mismatch: docCount is " + numDocs + " but fdx file size is " + indexStream.getFilePointer() + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
}
}

View File

@ -1,62 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.LuceneTestCase;
class PreFlexRWTermVectorsFormat extends Lucene3xTermVectorsFormat {
@Override
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new PreFlexRWTermVectorsWriter(directory, segmentInfo.name, context);
}
@Override
public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context) {
@Override
protected boolean sortTermsByUnicode() {
// We carefully peek into stack track above us: if
// we are part of a "merge", we must sort by UTF16:
boolean unicodeSortOrder = true;
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
//System.out.println(trace[i].getClassName());
if ("merge".equals(trace[i].getMethodName())) {
unicodeSortOrder = false;
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: PreFlexRW codec: forcing legacy UTF16 vector term sort order");
}
break;
}
}
return unicodeSortOrder;
}
};
}
}

View File

@ -1,221 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* 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.Comparator;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
final class PreFlexRWTermVectorsWriter extends TermVectorsWriter {
private final Directory directory;
private final String segment;
private IndexOutput tvx = null, tvd = null, tvf = null;
public PreFlexRWTermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
this.directory = directory;
this.segment = segment;
boolean success = false;
try {
// Open files for TermVector storage
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION), context);
tvx.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION), context);
tvd.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION), context);
tvf.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
success = true;
} finally {
if (!success) {
abort();
}
}
}
@Override
public void startDocument(int numVectorFields) throws IOException {
lastFieldName = null;
this.numVectorFields = numVectorFields;
tvx.writeLong(tvd.getFilePointer());
tvx.writeLong(tvf.getFilePointer());
tvd.writeVInt(numVectorFields);
fieldCount = 0;
fps = ArrayUtil.grow(fps, numVectorFields);
}
private long fps[] = new long[10]; // pointers to the tvf before writing each field
private int fieldCount = 0; // number of fields we have written so far for this document
private int numVectorFields = 0; // total number of fields we will write for this document
private String lastFieldName;
@Override
public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException {
assert lastFieldName == null || info.name.compareTo(lastFieldName) > 0: "fieldName=" + info.name + " lastFieldName=" + lastFieldName;
lastFieldName = info.name;
this.positions = positions;
this.offsets = offsets;
lastTerm.length = 0;
fps[fieldCount++] = tvf.getFilePointer();
tvd.writeVInt(info.number);
tvf.writeVInt(numTerms);
byte bits = 0x0;
if (positions)
bits |= Lucene3xTermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
if (offsets)
bits |= Lucene3xTermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
tvf.writeByte(bits);
assert fieldCount <= numVectorFields;
if (fieldCount == numVectorFields) {
// last field of the document
// this is crazy because the file format is crazy!
for (int i = 1; i < fieldCount; i++) {
tvd.writeVLong(fps[i] - fps[i-1]);
}
}
}
private final BytesRef lastTerm = new BytesRef(10);
// NOTE: we override addProx, so we don't need to buffer when indexing.
// we also don't buffer during bulk merges.
private int offsetStartBuffer[] = new int[10];
private int offsetEndBuffer[] = new int[10];
private int offsetIndex = 0;
private int offsetFreq = 0;
private boolean positions = false;
private boolean offsets = false;
@Override
public void startTerm(BytesRef term, int freq) throws IOException {
final int prefix = StringHelper.bytesDifference(lastTerm, term);
final int suffix = term.length - prefix;
tvf.writeVInt(prefix);
tvf.writeVInt(suffix);
tvf.writeBytes(term.bytes, term.offset + prefix, suffix);
tvf.writeVInt(freq);
lastTerm.copyBytes(term);
lastPosition = lastOffset = 0;
if (offsets && positions) {
// we might need to buffer if its a non-bulk merge
offsetStartBuffer = ArrayUtil.grow(offsetStartBuffer, freq);
offsetEndBuffer = ArrayUtil.grow(offsetEndBuffer, freq);
offsetIndex = 0;
offsetFreq = freq;
}
}
int lastPosition = 0;
int lastOffset = 0;
@Override
public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
// TODO: technically we could just copy bytes and not re-encode if we knew the length...
if (positions != null) {
for (int i = 0; i < numProx; i++) {
tvf.writeVInt(positions.readVInt());
}
}
if (offsets != null) {
for (int i = 0; i < numProx; i++) {
tvf.writeVInt(offsets.readVInt());
tvf.writeVInt(offsets.readVInt());
}
}
}
@Override
public void addPosition(int position, int startOffset, int endOffset) throws IOException {
if (positions && offsets) {
// write position delta
tvf.writeVInt(position - lastPosition);
lastPosition = position;
// buffer offsets
offsetStartBuffer[offsetIndex] = startOffset;
offsetEndBuffer[offsetIndex] = endOffset;
offsetIndex++;
// dump buffer if we are done
if (offsetIndex == offsetFreq) {
for (int i = 0; i < offsetIndex; i++) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
}
}
} else if (positions) {
// write position delta
tvf.writeVInt(position - lastPosition);
lastPosition = position;
} else if (offsets) {
// write offset deltas
tvf.writeVInt(startOffset - lastOffset);
tvf.writeVInt(endOffset - startOffset);
lastOffset = endOffset;
}
}
@Override
public void abort() {
try {
close();
} catch (IOException ignored) {}
IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION),
IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION),
IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
if (4+((long) numDocs)*16 != tvx.getFilePointer())
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and
// throw an exception to prevent the corruption from
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("tvx size mismatch: mergedDocs is " + numDocs + " but tvx size is " + tvx.getFilePointer() + " file=" + tvx.toString() + "; now aborting this merge to prevent index corruption");
}
/** Close all streams. */
@Override
public void close() throws IOException {
// make an effort to close all streams we can but remember and re-throw
// the first exception encountered in this process
IOUtils.close(tvx, tvd, tvf);
tvx = tvd = tvf = null;
}
@Override
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUTF16Comparator();
}
}

View File

@ -1,281 +0,0 @@
package org.apache.lucene.codecs.lucene3x;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import 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.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.UnicodeUtil;
/** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
Directory. A TermInfos can be written once, in order. */
final class TermInfosWriter implements Closeable {
/** The file format version, a negative number. */
public static final int FORMAT = -3;
// Changed strings to true utf8 with length-in-bytes not
// length-in-chars
public static final int FORMAT_VERSION_UTF8_LENGTH_IN_BYTES = -4;
// NOTE: always change this if you switch to a new format!
public static final int FORMAT_CURRENT = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
private FieldInfos fieldInfos;
private IndexOutput output;
private TermInfo lastTi = new TermInfo();
private long size;
// TODO: the default values for these two parameters should be settable from
// IndexWriter. However, once that's done, folks will start setting them to
// ridiculous values and complaining that things don't work well, as with
// mergeFactor. So, let's wait until a number of folks find that alternate
// values work better. Note that both of these values are stored in the
// segment, so that it's safe to change these w/o rebuilding all indexes.
/** Expert: The fraction of terms in the "dictionary" which should be stored
* in RAM. Smaller values use more memory, but make searching slightly
* faster, while larger values use less memory and make searching slightly
* slower. Searching is typically not dominated by dictionary lookup, so
* tweaking this is rarely useful.*/
int indexInterval = 128;
/** Expert: The fraction of {@link TermDocs} entries stored in skip tables,
* used to accelerate {@link TermDocs#skipTo(int)}. Larger values result in
* smaller indexes, greater acceleration, but fewer accelerable cases, while
* smaller values result in bigger indexes, less acceleration and more
* accelerable cases. More detailed experiments would be useful here. */
int skipInterval = 16;
/** Expert: The maximum number of skip levels. Smaller values result in
* slightly smaller indexes, but slower skipping in big posting lists.
*/
int maxSkipLevels = 10;
private long lastIndexPointer;
private boolean isIndex;
private final BytesRef lastTerm = new BytesRef();
private int lastFieldNumber = -1;
private TermInfosWriter other;
TermInfosWriter(Directory directory, String segment, FieldInfos fis,
int interval)
throws IOException {
initialize(directory, segment, fis, interval, false);
boolean success = false;
try {
other = new TermInfosWriter(directory, segment, fis, interval, true);
other.other = this;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
(isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
: Lucene3xPostingsFormat.TERMS_EXTENSION)));
} catch (IOException ignored) {
}
}
}
}
private TermInfosWriter(Directory directory, String segment, FieldInfos fis,
int interval, boolean isIndex) throws IOException {
initialize(directory, segment, fis, interval, isIndex);
}
private void initialize(Directory directory, String segment, FieldInfos fis,
int interval, boolean isi) throws IOException {
indexInterval = interval;
fieldInfos = fis;
isIndex = isi;
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "",
(isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
: Lucene3xPostingsFormat.TERMS_EXTENSION)), IOContext.DEFAULT);
boolean success = false;
try {
output.writeInt(FORMAT_CURRENT); // write format
output.writeLong(0); // leave space for size
output.writeInt(indexInterval); // write indexInterval
output.writeInt(skipInterval); // write skipInterval
output.writeInt(maxSkipLevels); // write maxSkipLevels
assert initUTF16Results();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
(isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
: Lucene3xPostingsFormat.TERMS_EXTENSION)));
} catch (IOException ignored) {
}
}
}
}
// Currently used only by assert statements
CharsRef utf16Result1;
CharsRef utf16Result2;
private final BytesRef scratchBytes = new BytesRef();
// Currently used only by assert statements
private boolean initUTF16Results() {
utf16Result1 = new CharsRef(10);
utf16Result2 = new CharsRef(10);
return true;
}
/** note: -1 is the empty field: "" !!!! */
static String fieldName(FieldInfos infos, int fieldNumber) {
FieldInfo fi = infos.fieldInfo(fieldNumber);
return (fi != null) ? fi.name : "";
}
// Currently used only by assert statement
private int compareToLastTerm(int fieldNumber, BytesRef term) {
if (lastFieldNumber != fieldNumber) {
final int cmp = fieldName(fieldInfos, lastFieldNumber).compareTo(fieldName(fieldInfos, fieldNumber));
// If there is a field named "" (empty string) then we
// will get 0 on this comparison, yet, it's "OK". But
// it's not OK if two different field numbers map to
// the same name.
if (cmp != 0 || lastFieldNumber != -1)
return cmp;
}
scratchBytes.copyBytes(term);
assert lastTerm.offset == 0;
UnicodeUtil.UTF8toUTF16(lastTerm.bytes, 0, lastTerm.length, utf16Result1);
assert scratchBytes.offset == 0;
UnicodeUtil.UTF8toUTF16(scratchBytes.bytes, 0, scratchBytes.length, utf16Result2);
final int len;
if (utf16Result1.length < utf16Result2.length)
len = utf16Result1.length;
else
len = utf16Result2.length;
for(int i=0;i<len;i++) {
final char ch1 = utf16Result1.chars[i];
final char ch2 = utf16Result2.chars[i];
if (ch1 != ch2)
return ch1-ch2;
}
if (utf16Result1.length == 0 && lastFieldNumber == -1) {
// If there is a field named "" (empty string) with a term text of "" (empty string) then we
// will get 0 on this comparison, yet, it's "OK".
return -1;
}
return utf16Result1.length - utf16Result2.length;
}
/** Adds a new <<fieldNumber, termBytes>, TermInfo> pair to the set.
Term must be lexicographically greater than all previous Terms added.
TermInfo pointers must be positive and greater than all previous.*/
public void add(int fieldNumber, BytesRef term, TermInfo ti)
throws IOException {
assert compareToLastTerm(fieldNumber, term) < 0 ||
(isIndex && term.length == 0 && lastTerm.length == 0) :
"Terms are out of order: field=" + fieldName(fieldInfos, fieldNumber) + " (number " + fieldNumber + ")" +
" lastField=" + fieldName(fieldInfos, lastFieldNumber) + " (number " + lastFieldNumber + ")" +
" text=" + term.utf8ToString() + " lastText=" + lastTerm.utf8ToString();
assert ti.freqPointer >= lastTi.freqPointer: "freqPointer out of order (" + ti.freqPointer + " < " + lastTi.freqPointer + ")";
assert ti.proxPointer >= lastTi.proxPointer: "proxPointer out of order (" + ti.proxPointer + " < " + lastTi.proxPointer + ")";
if (!isIndex && size % indexInterval == 0) {
other.add(lastFieldNumber, lastTerm, lastTi); // add an index term
}
writeTerm(fieldNumber, term); // write term
output.writeVInt(ti.docFreq); // write doc freq
output.writeVLong(ti.freqPointer - lastTi.freqPointer); // write pointers
output.writeVLong(ti.proxPointer - lastTi.proxPointer);
if (ti.docFreq >= skipInterval) {
output.writeVInt(ti.skipOffset);
}
if (isIndex) {
output.writeVLong(other.output.getFilePointer() - lastIndexPointer);
lastIndexPointer = other.output.getFilePointer(); // write pointer
}
lastFieldNumber = fieldNumber;
lastTi.set(ti);
size++;
}
private void writeTerm(int fieldNumber, BytesRef term)
throws IOException {
//System.out.println(" tiw.write field=" + fieldNumber + " term=" + term.utf8ToString());
// TODO: UTF16toUTF8 could tell us this prefix
// Compute prefix in common with last term:
int start = 0;
final int limit = term.length < lastTerm.length ? term.length : lastTerm.length;
while(start < limit) {
if (term.bytes[start+term.offset] != lastTerm.bytes[start+lastTerm.offset])
break;
start++;
}
final int length = term.length - start;
output.writeVInt(start); // write shared prefix length
output.writeVInt(length); // write delta length
output.writeBytes(term.bytes, start+term.offset, length); // write delta bytes
output.writeVInt(fieldNumber); // write field num
lastTerm.copyBytes(term);
}
/** Called to complete TermInfos creation. */
public void close() throws IOException {
try {
output.seek(4); // write size after format
output.writeLong(size);
} finally {
try {
output.close();
} finally {
if (!isIndex) {
other.close();
}
}
}
}
}

View File

@ -1,30 +0,0 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for generating test indexes in the Lucene 3.x index format.
<p>
NOTE: This is not a perfect simulation of the 3.x format, but its close.
Particularly, indexes generated with this codec cannot actually be read
with Lucene 3.x
</p>
</body>
</html>

Some files were not shown because too many files have changed in this diff Show More