LUCENE-4399: Remove AppendingCodec

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1396020 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-10-09 13:12:35 +00:00
parent 15b7ffd6bf
commit 0f062118e1
31 changed files with 114 additions and 643 deletions

View File

@ -44,6 +44,11 @@ New Features
the suggester to ignore such variations. (Robert Muir, Sudarshan
Gaikaiwari, Mike McCandless)
API Changes
* LUCENE-4399: Deprecated AppendingCodec. Lucene's term dictionaries
no longer seek when writing. (Adrien Grand, Robert Muir)
Bug Fixes
* LUCENE-1822: BaseFragListBuilder hard-coded 6 char margin is too naive.

View File

@ -1,44 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
/**
* This codec uses an index format that is very similar to
* {@link Lucene40Codec} but works on append-only outputs, such as plain output
* streams and append-only filesystems.
*
* @lucene.experimental
*/
public final class AppendingCodec extends FilterCodec {
public AppendingCodec() {
super("Appending", new Lucene40Codec());
}
private final PostingsFormat postings = new AppendingPostingsFormat();
@Override
public PostingsFormat postingsFormat() {
return postings;
}
}

View File

@ -1,80 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* Appending postings impl.
*/
final class AppendingPostingsFormat extends PostingsFormat {
public static String CODEC_NAME = "Appending";
public AppendingPostingsFormat() {
super(CODEC_NAME);
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
success = true;
return ret;
} finally {
if (!success) {
docsWriter.close();
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new AppendingTermsReader(
state.dir,
state.fieldInfos,
state.segmentInfo,
postings,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
success = true;
return ret;
} finally {
if (!success) {
postings.close();
}
}
}
}

View File

@ -1,62 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.BlockTreeTermsReader;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PostingsReaderBase;
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.store.IndexInput;
/**
* Reads append-only terms from {@link AppendingTermsWriter}
* @lucene.experimental
*/
public class AppendingTermsReader extends BlockTreeTermsReader {
public AppendingTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info, PostingsReaderBase postingsReader,
IOContext ioContext, String segmentSuffix, int indexDivisor) throws IOException {
super(dir, fieldInfos, info, postingsReader, ioContext, segmentSuffix, indexDivisor);
}
@Override
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, AppendingTermsWriter.TERMS_CODEC_NAME,
AppendingTermsWriter.TERMS_VERSION_START,
AppendingTermsWriter.TERMS_VERSION_CURRENT);
}
@Override
protected void readIndexHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, AppendingTermsWriter.TERMS_INDEX_CODEC_NAME,
AppendingTermsWriter.TERMS_INDEX_VERSION_START,
AppendingTermsWriter.TERMS_INDEX_VERSION_CURRENT);
}
@Override
protected void seekDir(IndexInput input, long dirOffset) throws IOException {
input.seek(input.length() - Long.SIZE / 8);
long offset = input.readLong();
input.seek(offset);
}
}

View File

@ -1,64 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.BlockTreeTermsWriter;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
/**
* Append-only version of {@link BlockTreeTermsWriter}
* @lucene.experimental
*/
public class AppendingTermsWriter extends BlockTreeTermsWriter {
final static String TERMS_CODEC_NAME = "APPENDING_TERMS_DICT";
final static int TERMS_VERSION_START = 0;
final static int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
final static String TERMS_INDEX_CODEC_NAME = "APPENDING_TERMS_INDEX";
final static int TERMS_INDEX_VERSION_START = 0;
final static int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_START;
public AppendingTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter, int minItemsInBlock, int maxItemsInBlock) throws IOException {
super(state, postingsWriter, minItemsInBlock, maxItemsInBlock);
}
@Override
protected void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
}
@Override
protected void writeIndexHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT);
}
@Override
protected void writeTrailer(IndexOutput out, long dirStart) throws IOException {
out.writeLong(dirStart);
}
@Override
protected void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
indexOut.writeLong(dirStart);
}
}

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 for on append-only outputs, such as plain output streams and append-only filesystems.
</body>
</html>

View File

@ -118,8 +118,8 @@ import org.apache.lucene.util.packed.PackedInts;
*
* <ul>
* <!-- TODO: expand on this, its not really correct and doesnt explain sub-blocks etc -->
* <li>TermDictionary(.tim) --&gt; Header, DirOffset, PostingsHeader, PackedBlockSize,
* &lt;Block&gt;<sup>NumBlocks</sup>, FieldSummary</li>
* <li>TermDictionary(.tim) --&gt; Header, PostingsHeader, PackedBlockSize,
* &lt;Block&gt;<sup>NumBlocks</sup>, FieldSummary, DirOffset</li>
* <li>Block --&gt; SuffixBlock, StatsBlock, MetadataBlock</li>
* <li>SuffixBlock --&gt; EntryCount, SuffixLength, {@link DataOutput#writeByte byte}<sup>SuffixLength</sup></li>
* <li>StatsBlock --&gt; StatsLength, &lt;DocFreq, TotalTermFreq&gt;<sup>EntryCount</sup></li>

View File

@ -76,7 +76,9 @@ public class BlockTermsReader extends FieldsProducer {
private TermsIndexReaderBase indexReader;
// keeps the dirStart offset
protected long dirOffset;
private long dirOffset;
private final int version;
// Used as key for the terms cache
private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
@ -123,7 +125,7 @@ public class BlockTermsReader extends FieldsProducer {
boolean success = false;
try {
readHeader(in);
version = readHeader(in);
// Have PostingsReader init itself
postingsReader.init(in);
@ -168,15 +170,21 @@ public class BlockTermsReader extends FieldsProducer {
this.indexReader = indexReader;
}
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
BlockTermsWriter.VERSION_START,
BlockTermsWriter.VERSION_CURRENT);
dirOffset = input.readLong();
if (version < BlockTermsWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
protected void seekDir(IndexInput input, long dirOffset)
throws IOException {
private void seekDir(IndexInput input, long dirOffset) throws IOException {
if (version >= BlockTermsWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(dirOffset);
}

View File

@ -58,8 +58,8 @@ public class BlockTermsWriter extends FieldsConsumer {
// Initial format
public static final int VERSION_START = 0;
public static final int VERSION_CURRENT = VERSION_START;
public static final int VERSION_APPEND_ONLY = 1;
public static final int VERSION_CURRENT = VERSION_APPEND_ONLY;
/** Extension of terms file */
static final String TERMS_EXTENSION = "tib";
@ -98,10 +98,8 @@ public class BlockTermsWriter extends FieldsConsumer {
}
}
protected void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
private void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
}
@Override
@ -148,8 +146,7 @@ public class BlockTermsWriter extends FieldsConsumer {
}
}
protected void writeTrailer(long dirStart) throws IOException {
out.seek(CodecUtil.headerLength(CODEC_NAME));
private void writeTrailer(long dirStart) throws IOException {
out.writeLong(dirStart);
}

View File

@ -70,7 +70,9 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
// start of the field info data
protected long dirOffset;
private long dirOffset;
private final int version;
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
throws IOException {
@ -85,7 +87,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
try {
readHeader(in);
version = readHeader(in);
indexInterval = in.readInt();
if (indexInterval < 1) {
throw new CorruptIndexException("invalid indexInterval: " + indexInterval + " (resource=" + in + ")");
@ -148,10 +150,13 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
return indexDivisor;
}
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_START);
dirOffset = input.readLong();
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_CURRENT);
if (version < FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
private class IndexEnum extends FieldIndexEnum {
@ -409,7 +414,11 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
}
}
protected void seekDir(IndexInput input, long dirOffset) throws IOException {
private void seekDir(IndexInput input, long dirOffset) throws IOException {
if (version >= FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(dirOffset);
}
}

View File

@ -49,7 +49,8 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
final static String CODEC_NAME = "SIMPLE_STANDARD_TERMS_INDEX";
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
final static int VERSION_APPEND_ONLY = 1;
final static int VERSION_CURRENT = VERSION_APPEND_ONLY;
final private int termIndexInterval;
@ -74,10 +75,8 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
}
}
protected void writeHeader(IndexOutput out) throws IOException {
private void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
// Placeholder for dir offset
out.writeLong(0);
}
@Override
@ -250,8 +249,7 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
}
}
protected void writeTrailer(long dirStart) throws IOException {
out.seek(CodecUtil.headerLength(CODEC_NAME));
private void writeTrailer(long dirStart) throws IOException {
out.writeLong(dirStart);
}
}

View File

@ -54,7 +54,9 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
// start of the field info data
protected long dirOffset;
private long dirOffset;
private final int version;
final String segment;
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
@ -66,7 +68,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
try {
readHeader(in);
version = readHeader(in);
this.indexDivisor = indexDivisor;
seekDir(in, dirOffset);
@ -103,10 +105,13 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
return indexDivisor;
}
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
VariableGapTermsIndexWriter.VERSION_START, VariableGapTermsIndexWriter.VERSION_START);
dirOffset = input.readLong();
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
VariableGapTermsIndexWriter.VERSION_START, VariableGapTermsIndexWriter.VERSION_CURRENT);
if (version < VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
private static class IndexEnum extends FieldIndexEnum {
@ -229,7 +234,11 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
}
}
protected void seekDir(IndexInput input, long dirOffset) throws IOException {
private void seekDir(IndexInput input, long dirOffset) throws IOException {
if (version >= VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(dirOffset);
}
}

View File

@ -52,7 +52,8 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
final static String CODEC_NAME = "VARIABLE_GAP_TERMS_INDEX";
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
final static int VERSION_APPEND_ONLY = 1;
final static int VERSION_CURRENT = VERSION_APPEND_ONLY;
private final List<FSTFieldWriter> fields = new ArrayList<FSTFieldWriter>();
@ -189,10 +190,8 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
}
}
protected void writeHeader(IndexOutput out) throws IOException {
private void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
// Placeholder for dir offset
out.writeLong(0);
}
@Override
@ -316,8 +315,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
}
}
protected void writeTrailer(long dirStart) throws IOException {
out.seek(CodecUtil.headerLength(CODEC_NAME));
private void writeTrailer(long dirStart) throws IOException {
out.writeLong(dirStart);
}
}

View File

@ -14,4 +14,3 @@
# limitations under the License.
org.apache.lucene.codecs.simpletext.SimpleTextCodec
org.apache.lucene.codecs.appending.AppendingCodec

View File

@ -1,167 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.StoredDocument;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestAppendingCodec extends LuceneTestCase {
private static class AppendingIndexOutputWrapper extends IndexOutput {
IndexOutput wrapped;
public AppendingIndexOutputWrapper(IndexOutput wrapped) {
this.wrapped = wrapped;
}
@Override
public void close() throws IOException {
wrapped.close();
}
@Override
public void flush() throws IOException {
wrapped.flush();
}
@Override
public long getFilePointer() {
return wrapped.getFilePointer();
}
@Override
public long length() throws IOException {
return wrapped.length();
}
@Override
public void seek(long pos) throws IOException {
throw new UnsupportedOperationException("seek() is unsupported");
}
@Override
public void writeByte(byte b) throws IOException {
wrapped.writeByte(b);
}
@Override
public void writeBytes(byte[] b, int offset, int length) throws IOException {
wrapped.writeBytes(b, offset, length);
}
}
@SuppressWarnings("serial")
private static class AppendingRAMDirectory extends MockDirectoryWrapper {
public AppendingRAMDirectory(Random random, Directory delegate) {
super(random, delegate);
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return new AppendingIndexOutputWrapper(super.createOutput(name, context));
}
}
private static final String text = "the quick brown fox jumped over the lazy dog";
public void testCodec() throws Exception {
Directory dir = new AppendingRAMDirectory(random(), new RAMDirectory());
IndexWriterConfig cfg = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
cfg.setCodec(new AppendingCodec());
((TieredMergePolicy)cfg.getMergePolicy()).setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(dir, cfg);
Document doc = new Document();
FieldType storedTextType = new FieldType(TextField.TYPE_STORED);
storedTextType.setStoreTermVectors(true);
storedTextType.setStoreTermVectorPositions(true);
storedTextType.setStoreTermVectorOffsets(true);
doc.add(newField("f", text, storedTextType));
writer.addDocument(doc);
writer.commit();
writer.addDocument(doc);
writer.forceMerge(1);
writer.close();
IndexReader reader = DirectoryReader.open(dir, 1);
assertEquals(2, reader.numDocs());
StoredDocument doc2 = reader.document(0);
assertEquals(text, doc2.get("f"));
Fields fields = MultiFields.getFields(reader);
Terms terms = fields.terms("f");
assertNotNull(terms);
TermsEnum te = terms.iterator(null);
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("quick")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("brown")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("fox")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("jumped")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("over")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("lazy")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("dog")));
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("the")));
DocsEnum de = te.docs(null, null);
assertTrue(de.advance(0) != DocIdSetIterator.NO_MORE_DOCS);
assertEquals(2, de.freq());
assertTrue(de.advance(1) != DocIdSetIterator.NO_MORE_DOCS);
assertTrue(de.advance(2) == DocIdSetIterator.NO_MORE_DOCS);
reader.close();
}
public void testCompoundFile() throws Exception {
Directory dir = new AppendingRAMDirectory(random(), new RAMDirectory());
IndexWriterConfig cfg = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
TieredMergePolicy mp = new TieredMergePolicy();
mp.setUseCompoundFile(true);
mp.setNoCFSRatio(1.0);
cfg.setMergePolicy(mp);
cfg.setCodec(new AppendingCodec());
IndexWriter writer = new IndexWriter(dir, cfg);
Document doc = new Document();
writer.addDocument(doc);
writer.close();
assertTrue(dir.fileExists("_0.cfs"));
dir.close();
}
}

View File

@ -1,34 +0,0 @@
package org.apache.lucene.codecs.appending;
/*
* 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.lucene40.Lucene40Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
* Tests AppendingPostingsFormat
*/
public class TestAppendingPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = new AppendingCodec();
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.block;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.appending.AppendingCodec;
import org.apache.lucene.codecs.block.BlockPostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;

View File

@ -100,12 +100,14 @@ public class BlockTreeTermsReader extends FieldsProducer {
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
/** File offset where the directory starts in the terms file. */
protected long dirOffset;
private long dirOffset;
/** File offset where the directory starts in the index file. */
protected long indexDirOffset;
private long indexDirOffset;
private String segment;
private final int version;
/** Sole constructor. */
public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
@ -123,11 +125,14 @@ public class BlockTreeTermsReader extends FieldsProducer {
IndexInput indexIn = null;
try {
readHeader(in);
version = readHeader(in);
if (indexDivisor != -1) {
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
ioContext);
readIndexHeader(indexIn);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
}
}
// Have PostingsReader init itself
@ -186,24 +191,34 @@ public class BlockTreeTermsReader extends FieldsProducer {
}
/** Reads terms file header. */
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
BlockTreeTermsWriter.TERMS_VERSION_START,
BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
dirOffset = input.readLong();
if (version < BlockTreeTermsWriter.TERMS_VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
/** Reads index file header. */
protected void readIndexHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
private int readIndexHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
BlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
indexDirOffset = input.readLong();
if (version < BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
indexDirOffset = input.readLong();
}
return version;
}
/** Seek {@code input} to the directory offset. */
protected void seekDir(IndexInput input, long dirOffset)
private void seekDir(IndexInput input, long dirOffset)
throws IOException {
if (version >= BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(dirOffset);
}

View File

@ -107,9 +107,12 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
/** Initial terms format. */
public static final int TERMS_VERSION_START = 0;
/** Append-only */
public static final int TERMS_VERSION_APPEND_ONLY = 1;
/** Current terms format. */
public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_APPEND_ONLY;
/** Extension of terms index file */
static final String TERMS_INDEX_EXTENSION = "tip";
@ -117,9 +120,12 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
/** Initial index format. */
public static final int TERMS_INDEX_VERSION_START = 0;
/** Append-only */
public static final int TERMS_INDEX_VERSION_APPEND_ONLY = 1;
/** Current index format. */
public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_START;
public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_APPEND_ONLY;
private final IndexOutput out;
private final IndexOutput indexOut;
@ -189,26 +195,22 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
}
/** Writes the terms file header. */
protected void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
private void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
}
/** Writes the index file header. */
protected void writeIndexHeader(IndexOutput out) throws IOException {
private void writeIndexHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
}
/** Writes the terms file trailer. */
protected void writeTrailer(IndexOutput out, long dirStart) throws IOException {
out.seek(CodecUtil.headerLength(TERMS_CODEC_NAME));
private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
out.writeLong(dirStart);
}
/** Writes the index file trailer. */
protected void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
indexOut.seek(CodecUtil.headerLength(TERMS_INDEX_CODEC_NAME));
private void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
indexOut.writeLong(dirStart);
}

View File

@ -68,9 +68,9 @@ import org.apache.lucene.util.fst.FST; // javadocs
*
* <ul>
* <!-- TODO: expand on this, its not really correct and doesnt explain sub-blocks etc -->
* <li>TermsDict (.tim) --&gt; Header, DirOffset, PostingsHeader, SkipInterval,
* <li>TermsDict (.tim) --&gt; Header, PostingsHeader, SkipInterval,
* MaxSkipLevels, SkipMinimum, Block<sup>NumBlocks</sup>,
* FieldSummary</li>
* FieldSummary, DirOffset</li>
* <li>Block --&gt; SuffixBlock, StatsBlock, MetadataBlock</li>
* <li>SuffixBlock --&gt; EntryCount, SuffixLength, Byte<sup>SuffixLength</sup></li>
* <li>StatsBlock --&gt; StatsLength, &lt;DocFreq, TotalTermFreq&gt;<sup>EntryCount</sup></li>
@ -131,12 +131,13 @@ import org.apache.lucene.util.fst.FST; // javadocs
* accessed randomly. The index is also used to determine
* when a given term cannot exist on disk (in the .tim file), saving a disk seek.</p>
* <ul>
* <li>TermsIndex (.tip) --&gt; Header, &lt;IndexStartFP&gt;<sup>NumFields</sup>,
* FSTIndex<sup>NumFields</sup></li>
* <li>TermsIndex (.tip) --&gt; Header, FSTIndex<sup>NumFields</sup>,
* &lt;IndexStartFP&gt;<sup>NumFields</sup>, DirOffset</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>IndexStartFP --&gt; {@link DataOutput#writeVLong VLong}</li>
* <!-- TODO: better describe FST output here -->
* <li>FSTIndex --&gt; {@link FST FST&lt;byte[]&gt;}</li>
* <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
* </ul>
* <p>Notes:</p>
* <ul>
@ -145,6 +146,8 @@ import org.apache.lucene.util.fst.FST; // javadocs
* block that holds all terms starting with that
* prefix. Each field's IndexStartFP points to its
* FST.</li>
* <li>DirOffset is a pointer to the start of the IndexStartFPs
* for all fields</li>
* <li>It's possible that an on-disk block would contain
* too many terms (more than the allowed maximum
* (default: 48)). When this happens, the block is

View File

@ -27,9 +27,6 @@ public abstract class BufferedIndexOutput extends IndexOutput {
private long bufferStart = 0; // position in file of buffer
private int bufferPosition = 0; // position in buffer
/** Writes a single byte.
* @see IndexInput#readByte()
*/
@Override
public void writeByte(byte b) throws IOException {
if (bufferPosition >= BUFFER_SIZE)
@ -37,11 +34,6 @@ public abstract class BufferedIndexOutput extends IndexOutput {
buffer[bufferPosition++] = b;
}
/** Writes an array of bytes.
* @param b the bytes to write
* @param length the number of bytes to write
* @see IndexInput#readBytes(byte[],int,int)
*/
@Override
public void writeBytes(byte[] b, int offset, int length) throws IOException {
int bytesLeft = BUFFER_SIZE - bufferPosition;
@ -82,7 +74,6 @@ public abstract class BufferedIndexOutput extends IndexOutput {
}
}
/** Forces any buffered output to be written. */
@Override
public void flush() throws IOException {
flushBuffer(buffer, bufferPosition);
@ -107,31 +98,16 @@ public abstract class BufferedIndexOutput extends IndexOutput {
*/
protected abstract void flushBuffer(byte[] b, int offset, int len) throws IOException;
/** Closes this stream to further operations. */
@Override
public void close() throws IOException {
flush();
}
/** Returns the current position in this file, where the next write will
* occur.
* @see #seek(long)
*/
@Override
public long getFilePointer() {
return bufferStart + bufferPosition;
}
/** Sets current position in this file, where the next write will occur.
* @see #getFilePointer()
*/
@Override
public void seek(long pos) throws IOException {
flush();
bufferStart = pos;
}
/** The number of bytes in the file. */
@Override
public abstract long length() throws IOException;

View File

@ -22,7 +22,7 @@ import java.util.zip.CRC32;
import java.util.zip.Checksum;
/** Writes bytes through to a primary IndexOutput, computing
* checksum. Note that you cannot use seek().
* checksum.
*
* @lucene.internal
*/
@ -66,11 +66,6 @@ public class ChecksumIndexOutput extends IndexOutput {
return main.getFilePointer();
}
@Override
public void seek(long pos) {
throw new UnsupportedOperationException();
}
/** writes the checksum */
public void finishCommit() throws IOException {
main.writeLong(getChecksum());

View File

@ -324,12 +324,6 @@ final class CompoundFileWriter implements Closeable{
return delegate.getFilePointer() - offset;
}
@Override
public void seek(long pos) throws IOException {
assert !closed;
delegate.seek(offset + pos);
}
@Override
public long length() throws IOException {
assert !closed;

View File

@ -548,13 +548,6 @@ public abstract class FSDirectory extends Directory {
}
}
/** Random-access methods */
@Override
public void seek(long pos) throws IOException {
super.seek(pos);
file.seek(pos);
}
@Override
public long length() throws IOException {
return file.length();

View File

@ -39,14 +39,9 @@ public abstract class IndexOutput extends DataOutput implements Closeable {
/** Returns the current position in this file, where the next write will
* occur.
* @see #seek(long)
*/
public abstract long getFilePointer();
/** Sets current position in this file, where the next write will occur.
* @see #getFilePointer()
*/
public abstract void seek(long pos) throws IOException;
/** The number of bytes in the file. */
public abstract long length() throws IOException;

View File

@ -102,19 +102,6 @@ public class RAMOutputStream extends IndexOutput {
flush();
}
@Override
public void seek(long pos) throws IOException {
// set the file length in case we seek back
// and flush() has not been called yet
setFileLength();
if (pos < bufferStart || pos >= bufferStart + bufferLength) {
currentBufferIndex = (int) (pos / BUFFER_SIZE);
switchCurrentBuffer();
}
bufferPosition = (int) (pos % BUFFER_SIZE);
}
@Override
public long length() {
return file.length;

View File

@ -154,7 +154,6 @@ public class SlowRAMDirectory extends RAMDirectory {
@Override public void flush() throws IOException { io.flush(); }
@Override public long getFilePointer() { return io.getFilePointer(); }
@Override public long length() throws IOException { return io.length(); }
@Override public void seek(long pos) throws IOException { io.seek(pos); }
}
}

View File

@ -236,26 +236,6 @@ public class NativeUnixDirectory extends FSDirectory {
return filePos + bufferPos;
}
// TODO: seek is fragile at best; it can only properly
// handle seek & then change bytes that fit entirely
// within one buffer
@Override
public void seek(long pos) throws IOException {
if (pos != getFilePointer()) {
dump();
final long alignedPos = pos & ALIGN_NOT_MASK;
filePos = alignedPos;
int n = (int) NativePosixUtil.pread(fos.getFD(), filePos, buffer);
if (n < bufferSize) {
buffer.limit(n);
}
//System.out.println("seek refill=" + n);
final int delta = (int) (pos - alignedPos);
buffer.position(delta);
bufferPos = delta;
}
}
@Override
public long length() {
return fileLength + bufferPos;

View File

@ -139,11 +139,6 @@ public class MockIndexOutputWrapper extends IndexOutput {
return delegate.getFilePointer();
}
@Override
public void seek(long pos) throws IOException {
delegate.seek(pos);
}
@Override
public long length() throws IOException {
return delegate.length();

View File

@ -29,7 +29,6 @@ import java.util.TimeZone;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.appending.AppendingCodec;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
@ -162,8 +161,6 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
};
} else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && !shouldAvoidCodec("SimpleText"))) {
codec = new SimpleTextCodec();
} else if ("Appending".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 8 && !shouldAvoidCodec("Appending"))) {
codec = new AppendingCodec();
} else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("Asserting"))) {
codec = new AssertingCodec();
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {

View File

@ -88,12 +88,6 @@ public class ThrottledIndexOutput extends IndexOutput {
return delegate.getFilePointer();
}
@Override
public void seek(long pos) throws IOException {
sleep(seekDelayMillis);
delegate.seek(pos);
}
@Override
public long length() throws IOException {
return delegate.length();