diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java b/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java deleted file mode 100644 index 50a6ceac575..00000000000 --- a/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java +++ /dev/null @@ -1,169 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.util.HashMap; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.HashSet; -import java.io.IOException; - -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; - -/** This is just a "splitter" class: it lets you wrap two - * DocFieldConsumer instances as a single consumer. */ - -final class DocFieldConsumers extends DocFieldConsumer { - final DocFieldConsumer one; - final DocFieldConsumer two; - - public DocFieldConsumers(DocFieldConsumer one, DocFieldConsumer two) { - this.one = one; - this.two = two; - } - - @Override - void setFieldInfos(FieldInfos fieldInfos) { - super.setFieldInfos(fieldInfos); - one.setFieldInfos(fieldInfos); - two.setFieldInfos(fieldInfos); - } - - @Override - public void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException { - - Map> oneThreadsAndFields = new HashMap>(); - Map> twoThreadsAndFields = new HashMap>(); - - for (Map.Entry> entry : threadsAndFields.entrySet()) { - - final DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey(); - - final Collection fields = entry.getValue(); - - Iterator fieldsIt = fields.iterator(); - Collection oneFields = new HashSet(); - Collection twoFields = new HashSet(); - while(fieldsIt.hasNext()) { - DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next(); - oneFields.add(perField.one); - twoFields.add(perField.two); - } - - oneThreadsAndFields.put(perThread.one, oneFields); - twoThreadsAndFields.put(perThread.two, twoFields); - } - - - one.flush(oneThreadsAndFields, state); - two.flush(twoThreadsAndFields, state); - } - - @Override - public void closeDocStore(SegmentWriteState state) throws IOException { - try { - one.closeDocStore(state); - } finally { - two.closeDocStore(state); - } - } - - @Override - public void abort() { - try { - one.abort(); - } finally { - two.abort(); - } - } - - @Override - public boolean freeRAM() { - boolean any = one.freeRAM(); - any |= two.freeRAM(); - return any; - } - - @Override - public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException { - return new DocFieldConsumersPerThread(docFieldProcessorPerThread, this, one.addThread(docFieldProcessorPerThread), two.addThread(docFieldProcessorPerThread)); - } - - PerDoc[] docFreeList = new PerDoc[1]; - int freeCount; - int allocCount; - - synchronized PerDoc getPerDoc() { - if (freeCount == 0) { - allocCount++; - if (allocCount > docFreeList.length) { - // Grow our free list up front to make sure we have - // enough space to recycle all outstanding PerDoc - // instances - assert allocCount == 1+docFreeList.length; - docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - } - return new PerDoc(); - } else - return docFreeList[--freeCount]; - } - - synchronized void freePerDoc(PerDoc perDoc) { - assert freeCount < docFreeList.length; - docFreeList[freeCount++] = perDoc; - } - - class PerDoc extends DocumentsWriter.DocWriter { - - DocumentsWriter.DocWriter writerOne; - DocumentsWriter.DocWriter writerTwo; - - @Override - public long sizeInBytes() { - return writerOne.sizeInBytes() + writerTwo.sizeInBytes(); - } - - @Override - public void finish() throws IOException { - try { - try { - writerOne.finish(); - } finally { - writerTwo.finish(); - } - } finally { - freePerDoc(this); - } - } - - @Override - public void abort() { - try { - try { - writerOne.abort(); - } finally { - writerTwo.abort(); - } - } finally { - freePerDoc(this); - } - } - } -} diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java b/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java deleted file mode 100644 index e75891f5556..00000000000 --- a/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java +++ /dev/null @@ -1,49 +0,0 @@ -package org.apache.lucene.index; - -/** - * 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.document.Fieldable; - -final class DocFieldConsumersPerField extends DocFieldConsumerPerField { - - final DocFieldConsumerPerField one; - final DocFieldConsumerPerField two; - final DocFieldConsumersPerThread perThread; - - public DocFieldConsumersPerField(DocFieldConsumersPerThread perThread, DocFieldConsumerPerField one, DocFieldConsumerPerField two) { - this.perThread = perThread; - this.one = one; - this.two = two; - } - - @Override - public void processFields(Fieldable[] fields, int count) throws IOException { - one.processFields(fields, count); - two.processFields(fields, count); - } - - @Override - public void abort() { - try { - one.abort(); - } finally { - two.abort(); - } - } -} diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java b/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java deleted file mode 100644 index 99d56ee725d..00000000000 --- a/lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java +++ /dev/null @@ -1,75 +0,0 @@ -package org.apache.lucene.index; - -/** - * 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; - -final class DocFieldConsumersPerThread extends DocFieldConsumerPerThread { - - final DocFieldConsumerPerThread one; - final DocFieldConsumerPerThread two; - final DocFieldConsumers parent; - final DocumentsWriter.DocState docState; - - public DocFieldConsumersPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, - DocFieldConsumers parent, DocFieldConsumerPerThread one, DocFieldConsumerPerThread two) { - this.parent = parent; - this.one = one; - this.two = two; - docState = docFieldProcessorPerThread.docState; - } - - @Override - public void startDocument() throws IOException { - one.startDocument(); - two.startDocument(); - } - - @Override - public void abort() { - try { - one.abort(); - } finally { - two.abort(); - } - } - - @Override - public DocumentsWriter.DocWriter finishDocument() throws IOException { - final DocumentsWriter.DocWriter oneDoc = one.finishDocument(); - final DocumentsWriter.DocWriter twoDoc = two.finishDocument(); - if (oneDoc == null) - return twoDoc; - else if (twoDoc == null) - return oneDoc; - else { - DocFieldConsumers.PerDoc both = parent.getPerDoc(); - both.docID = docState.docID; - assert oneDoc.docID == docState.docID; - assert twoDoc.docID == docState.docID; - both.writerOne = oneDoc; - both.writerTwo = twoDoc; - return both; - } - } - - @Override - public DocFieldConsumerPerField addField(FieldInfo fi) { - return new DocFieldConsumersPerField(this, one.addField(fi), two.addField(fi)); - } -} diff --git a/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java b/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java deleted file mode 100644 index 5eb0badf513..00000000000 --- a/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java +++ /dev/null @@ -1,279 +0,0 @@ -package org.apache.lucene.index; - -/** - * 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.BufferedIndexInput; -import org.apache.lucene.store.IndexInput; - -/** - * This abstract class reads skip lists with multiple levels. - * - * See {@link MultiLevelSkipListWriter} for the information about the encoding - * of the multi level skip lists. - * - * Subclasses must implement the abstract method {@link #readSkipData(int, IndexInput)} - * which defines the actual format of the skip data. - */ -abstract class MultiLevelSkipListReader { - // the maximum number of skip levels possible for this index - private int maxNumberOfSkipLevels; - - // number of levels in this skip list - private int numberOfSkipLevels; - - // Expert: defines the number of top skip levels to buffer in memory. - // Reducing this number results in less memory usage, but possibly - // slower performance due to more random I/Os. - // Please notice that the space each level occupies is limited by - // the skipInterval. The top level can not contain more than - // skipLevel entries, the second top level can not contain more - // than skipLevel^2 entries and so forth. - private int numberOfLevelsToBuffer = 1; - - private int docCount; - private boolean haveSkipped; - - private IndexInput[] skipStream; // skipStream for each level - private long skipPointer[]; // the start pointer of each skip level - private int skipInterval[]; // skipInterval of each level - private int[] numSkipped; // number of docs skipped per level - - private int[] skipDoc; // doc id of current skip entry per level - private int lastDoc; // doc id of last read skip entry with docId <= target - private long[] childPointer; // child pointer of current skip entry per level - private long lastChildPointer; // childPointer of last read skip entry with docId <= target - - private boolean inputIsBuffered; - - public MultiLevelSkipListReader(IndexInput skipStream, int maxSkipLevels, int skipInterval) { - this.skipStream = new IndexInput[maxSkipLevels]; - this.skipPointer = new long[maxSkipLevels]; - this.childPointer = new long[maxSkipLevels]; - this.numSkipped = new int[maxSkipLevels]; - this.maxNumberOfSkipLevels = maxSkipLevels; - this.skipInterval = new int[maxSkipLevels]; - this.skipStream [0]= skipStream; - this.inputIsBuffered = (skipStream instanceof BufferedIndexInput); - this.skipInterval[0] = skipInterval; - for (int i = 1; i < maxSkipLevels; i++) { - // cache skip intervals - this.skipInterval[i] = this.skipInterval[i - 1] * skipInterval; - } - skipDoc = new int[maxSkipLevels]; - } - - - /** Returns the id of the doc to which the last call of {@link #skipTo(int)} - * has skipped. */ - int getDoc() { - return lastDoc; - } - - - /** Skips entries to the first beyond the current whose document number is - * greater than or equal to target. Returns the current doc count. - */ - int skipTo(int target) throws IOException { - if (!haveSkipped) { - // first time, load skip levels - loadSkipLevels(); - haveSkipped = true; - } - - // walk up the levels until highest level is found that has a skip - // for this target - int level = 0; - while (level < numberOfSkipLevels - 1 && target > skipDoc[level + 1]) { - level++; - } - - while (level >= 0) { - if (target > skipDoc[level]) { - if (!loadNextSkip(level)) { - continue; - } - } else { - // no more skips on this level, go down one level - if (level > 0 && lastChildPointer > skipStream[level - 1].getFilePointer()) { - seekChild(level - 1); - } - level--; - } - } - - return numSkipped[0] - skipInterval[0] - 1; - } - - private boolean loadNextSkip(int level) throws IOException { - // we have to skip, the target document is greater than the current - // skip list entry - setLastSkipData(level); - - numSkipped[level] += skipInterval[level]; - - if (numSkipped[level] > docCount) { - // this skip list is exhausted - skipDoc[level] = Integer.MAX_VALUE; - if (numberOfSkipLevels > level) numberOfSkipLevels = level; - return false; - } - - // read next skip entry - skipDoc[level] += readSkipData(level, skipStream[level]); - - if (level != 0) { - // read the child pointer if we are not on the leaf level - childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1]; - } - - return true; - - } - - /** Seeks the skip entry on the given level */ - protected void seekChild(int level) throws IOException { - skipStream[level].seek(lastChildPointer); - numSkipped[level] = numSkipped[level + 1] - skipInterval[level + 1]; - skipDoc[level] = lastDoc; - if (level > 0) { - childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1]; - } - } - - void close() throws IOException { - for (int i = 1; i < skipStream.length; i++) { - if (skipStream[i] != null) { - skipStream[i].close(); - } - } - } - - /** initializes the reader */ - void init(long skipPointer, int df) { - this.skipPointer[0] = skipPointer; - this.docCount = df; - Arrays.fill(skipDoc, 0); - Arrays.fill(numSkipped, 0); - Arrays.fill(childPointer, 0); - - haveSkipped = false; - for (int i = 1; i < numberOfSkipLevels; i++) { - skipStream[i] = null; - } - } - - /** Loads the skip levels */ - private void loadSkipLevels() throws IOException { - numberOfSkipLevels = docCount == 0 ? 0 : (int) Math.floor(Math.log(docCount) / Math.log(skipInterval[0])); - if (numberOfSkipLevels > maxNumberOfSkipLevels) { - numberOfSkipLevels = maxNumberOfSkipLevels; - } - - skipStream[0].seek(skipPointer[0]); - - int toBuffer = numberOfLevelsToBuffer; - - for (int i = numberOfSkipLevels - 1; i > 0; i--) { - // the length of the current level - long length = skipStream[0].readVLong(); - - // the start pointer of the current level - skipPointer[i] = skipStream[0].getFilePointer(); - if (toBuffer > 0) { - // buffer this level - skipStream[i] = new SkipBuffer(skipStream[0], (int) length); - toBuffer--; - } else { - // clone this stream, it is already at the start of the current level - skipStream[i] = (IndexInput) skipStream[0].clone(); - if (inputIsBuffered && length < BufferedIndexInput.BUFFER_SIZE) { - ((BufferedIndexInput) skipStream[i]).setBufferSize((int) length); - } - - // move base stream beyond the current level - skipStream[0].seek(skipStream[0].getFilePointer() + length); - } - } - - // use base stream for the lowest level - skipPointer[0] = skipStream[0].getFilePointer(); - } - - /** - * Subclasses must implement the actual skip data encoding in this method. - * - * @param level the level skip data shall be read from - * @param skipStream the skip stream to read from - */ - protected abstract int readSkipData(int level, IndexInput skipStream) throws IOException; - - /** Copies the values of the last read skip entry on this level */ - protected void setLastSkipData(int level) { - lastDoc = skipDoc[level]; - lastChildPointer = childPointer[level]; - } - - - /** used to buffer the top skip levels */ - private final static class SkipBuffer extends IndexInput { - private byte[] data; - private long pointer; - private int pos; - - SkipBuffer(IndexInput input, int length) throws IOException { - data = new byte[length]; - pointer = input.getFilePointer(); - input.readBytes(data, 0, length); - } - - @Override - public void close() throws IOException { - data = null; - } - - @Override - public long getFilePointer() { - return pointer + pos; - } - - @Override - public long length() { - return data.length; - } - - @Override - public byte readByte() throws IOException { - return data[pos++]; - } - - @Override - public void readBytes(byte[] b, int offset, int len) throws IOException { - System.arraycopy(data, pos, b, offset, len); - pos += len; - } - - @Override - public void seek(long pos) throws IOException { - this.pos = (int) (pos - pointer); - } - - } -} diff --git a/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java b/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java deleted file mode 100644 index f4f43e251cb..00000000000 --- a/lucene/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java +++ /dev/null @@ -1,151 +0,0 @@ -package org.apache.lucene.index; - -/** - * 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.IndexOutput; -import org.apache.lucene.store.RAMOutputStream; - -/** - * This abstract class writes skip lists with multiple levels. - * - * Example for skipInterval = 3: - * c (skip level 2) - * c c c (skip level 1) - * x x x x x x x x x x (skip level 0) - * d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list) - * 3 6 9 12 15 18 21 24 27 30 (df) - * - * d - document - * x - skip data - * c - skip data with child pointer - * - * Skip level i contains every skipInterval-th entry from skip level i-1. - * Therefore the number of entries on level i is: floor(df / ((skipInterval ^ (i + 1))). - * - * Each skip entry on a level i>0 contains a pointer to the corresponding skip entry in list i-1. - * This guarantees a logarithmic amount of skips to find the target document. - * - * While this class takes care of writing the different skip levels, - * subclasses must define the actual format of the skip data. - * - */ -abstract class MultiLevelSkipListWriter { - // number of levels in this skip list - private int numberOfSkipLevels; - - // the skip interval in the list with level = 0 - private int skipInterval; - - // for every skip level a different buffer is used - private RAMOutputStream[] skipBuffer; - - protected MultiLevelSkipListWriter(int skipInterval, int maxSkipLevels, int df) { - this.skipInterval = skipInterval; - - // calculate the maximum number of skip levels for this document frequency - numberOfSkipLevels = df == 0 ? 0 : (int) Math.floor(Math.log(df) / Math.log(skipInterval)); - - // make sure it does not exceed maxSkipLevels - if (numberOfSkipLevels > maxSkipLevels) { - numberOfSkipLevels = maxSkipLevels; - } - } - - protected void init() { - skipBuffer = new RAMOutputStream[numberOfSkipLevels]; - for (int i = 0; i < numberOfSkipLevels; i++) { - skipBuffer[i] = new RAMOutputStream(); - } - } - - protected void resetSkip() { - // creates new buffers or empties the existing ones - if (skipBuffer == null) { - init(); - } else { - for (int i = 0; i < skipBuffer.length; i++) { - skipBuffer[i].reset(); - } - } - } - - /** - * Subclasses must implement the actual skip data encoding in this method. - * - * @param level the level skip data shall be writing for - * @param skipBuffer the skip buffer to write to - */ - protected abstract void writeSkipData(int level, IndexOutput skipBuffer) throws IOException; - - /** - * Writes the current skip data to the buffers. The current document frequency determines - * the max level is skip data is to be written to. - * - * @param df the current document frequency - * @throws IOException - */ - void bufferSkip(int df) throws IOException { - int numLevels; - - // determine max level - for (numLevels = 0; (df % skipInterval) == 0 && numLevels < numberOfSkipLevels; df /= skipInterval) { - numLevels++; - } - - long childPointer = 0; - - for (int level = 0; level < numLevels; level++) { - writeSkipData(level, skipBuffer[level]); - - long newChildPointer = skipBuffer[level].getFilePointer(); - - if (level != 0) { - // store child pointers for all levels except the lowest - skipBuffer[level].writeVLong(childPointer); - } - - //remember the childPointer for the next level - childPointer = newChildPointer; - } - } - - /** - * Writes the buffered skip lists to the given output. - * - * @param output the IndexOutput the skip lists shall be written to - * @return the pointer the skip list starts - */ - long writeSkip(IndexOutput output) throws IOException { - long skipPointer = output.getFilePointer(); - if (skipBuffer == null || skipBuffer.length == 0) return skipPointer; - - for (int level = numberOfSkipLevels - 1; level > 0; level--) { - long length = skipBuffer[level].getFilePointer(); - if (length > 0) { - output.writeVLong(length); - skipBuffer[level].writeTo(output); - } - } - skipBuffer[0].writeTo(output); - - return skipPointer; - } - -}