mirror of https://github.com/apache/lucene.git
SOLR-2452: merge with trunk up to r1134392
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/solr2452@1134403 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
c98779dd12
|
@ -82,7 +82,7 @@
|
|||
<module name="lucene" />
|
||||
<option name="TEST_OBJECT" value="package" />
|
||||
<option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$/lucene/build" />
|
||||
<option name="VM_PARAMETERS" value="-ea -Dlucene.version=4.0-SNAPSHOT -DtempDir=temp -Dtests.linedocsfile=europarl.lines.txt.gz" />
|
||||
<option name="VM_PARAMETERS" value="-ea -Dlucene.version=4.0-SNAPSHOT -DtempDir=temp" />
|
||||
<option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
|
||||
</configuration>
|
||||
<configuration default="false" name="memory contrib" type="JUnit" factoryName="JUnit">
|
||||
|
|
|
@ -64,6 +64,10 @@
|
|||
<version>${project.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>servlet-api</artifactId>
|
||||
|
|
|
@ -152,10 +152,6 @@
|
|||
<artifactId>servlet-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
|
|
|
@ -404,6 +404,31 @@ New features
|
|||
it's able to handle multi-valued fields and does not hold the term
|
||||
bytes in RAM. (Mike McCandless)
|
||||
|
||||
* LUCENE-3108, LUCENE-2935, LUCENE-2168, LUCENE-1231: Changes from
|
||||
IndexDocValues (ColumnStrideFields):
|
||||
|
||||
- IndexWriter now supports typesafe dense per-document values stored in
|
||||
a column like storage. IndexDocValues are stored on a per-document
|
||||
basis where each documents field can hold exactly one value of a given
|
||||
type. IndexDocValues are provided via Fieldable and can be used in
|
||||
conjunction with stored and indexed values.
|
||||
|
||||
- IndexDocValues provides an entirely RAM resident document id to value
|
||||
mapping per field as well as a DocIdSetIterator based disk-resident
|
||||
sequential access API relying on filesystem-caches.
|
||||
|
||||
- Both APIs are exposed via IndexReader and the Codec / Flex API allowing
|
||||
expert users to integrate customized IndexDocValues reader and writer
|
||||
implementations by extending existing Codecs.
|
||||
|
||||
- IndexDocValues provides implementations for primitive datatypes like int,
|
||||
long, float, double and arrays of byte. Byte based implementations further
|
||||
provide storage variants like straight or dereferenced stored bytes, fixed
|
||||
and variable length bytes as well as index time sorted based on user-provided
|
||||
comparators.
|
||||
|
||||
(Mike McCandless, Simon Willnauer)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms
|
||||
|
@ -424,6 +449,9 @@ Optimizations
|
|||
* LUCENE-2897: Apply deleted terms while flushing a segment. We still
|
||||
buffer deleted terms to later apply to past segments. (Mike McCandless)
|
||||
|
||||
* LUCENE-1736: DateTools.java general improvements.
|
||||
(David Smiley via Steve Rowe)
|
||||
|
||||
Bug fixes
|
||||
|
||||
* LUCENE-2633: PackedInts Packed32 and Packed64 did not support internal
|
||||
|
@ -434,7 +462,7 @@ Bug fixes
|
|||
deletions, provided they share some segments. (yonik)
|
||||
|
||||
* LUCENE-2645: Fix false assertion error when same token was added one
|
||||
after another with 0 posIncr. (Kurosaka Teruhiko via Mike
|
||||
after another with 0 posIncr. (David Smiley, Kurosaka Teruhiko via Mike
|
||||
McCandless)
|
||||
|
||||
======================= Lucene 3.x (not yet released) ================
|
||||
|
@ -465,6 +493,11 @@ Bug fixes
|
|||
* LUCENE-3102: CachingCollector.replay was failing to call setScorer
|
||||
per-segment (Martijn van Groningen via Mike McCandless)
|
||||
|
||||
* LUCENE-3183: Fix rare corner case where seeking to empty term
|
||||
(field="", term="") with terms index interval 1 could hit
|
||||
ArrayIndexOutOfBoundsException (selckin, Robert Muir, Mike
|
||||
McCandless)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-3140: Added experimental FST implementation to Lucene.
|
||||
|
|
|
@ -79,6 +79,12 @@ API Changes
|
|||
First/SecondPassGroupingCollector. (Martijn van Groningen, Mike
|
||||
McCandless)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-3185: Fix bug in NRTCachingDirectory.deleteFile that would
|
||||
always throw exception and sometimes fail to actually delete the
|
||||
file. (Mike McCandless)
|
||||
|
||||
Build
|
||||
|
||||
* LUCENE-3149: Upgrade contrib/icu's ICU jar file to ICU 4.8.
|
||||
|
|
|
@ -165,7 +165,7 @@ public class TokenSources {
|
|||
this.tokens = tokens;
|
||||
termAtt = addAttribute(CharTermAttribute.class);
|
||||
offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
posincAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
|
||||
posincAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.Comparator;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -487,4 +487,9 @@ public class InstantiatedIndexReader extends IndexReader {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.lucene.index.TermPositionVector;
|
|||
import org.apache.lucene.index.TermVectorMapper;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -1278,6 +1279,11 @@ public class MemoryIndex {
|
|||
|
||||
return Collections.unmodifiableSet(fields.keySet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -20,13 +20,19 @@ package org.apache.lucene.index.codecs.appending;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
|
@ -127,15 +133,27 @@ public class AppendingCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files)
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files)
|
||||
throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, codecId, files);
|
||||
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
StandardCodec.getStandardExtensions(extensions);
|
||||
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ public class AppendingTermsDictReader extends BlockTermsReader {
|
|||
public AppendingTermsDictReader(TermsIndexReaderBase indexReader,
|
||||
Directory dir, FieldInfos fieldInfos, String segment,
|
||||
PostingsReaderBase postingsReader, int readBufferSize,
|
||||
int termsCacheSize, String codecId) throws IOException {
|
||||
int termsCacheSize, int codecId) throws IOException {
|
||||
super(indexReader, dir, fieldInfos, segment, postingsReader, readBufferSize,
|
||||
termsCacheSize, codecId);
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.lucene.util.CodecUtil;
|
|||
public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
|
||||
|
||||
public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
|
||||
String segment, int indexDivisor, Comparator<BytesRef> termComp, String codecId)
|
||||
String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
|
||||
throws IOException {
|
||||
super(dir, fieldInfos, segment, indexDivisor, termComp, codecId);
|
||||
}
|
||||
|
|
|
@ -92,7 +92,7 @@ public class NRTCachingDirectory extends Directory {
|
|||
|
||||
/**
|
||||
* We will cache a newly created output if 1) it's a
|
||||
* flush or a merge and the estimated size of the merged segmnt is <=
|
||||
* flush or a merge and the estimated size of the merged segment is <=
|
||||
* maxMergeSizeMB, and 2) the total cached bytes is <=
|
||||
* maxCachedMB */
|
||||
public NRTCachingDirectory(Directory delegate, double maxMergeSizeMB, double maxCachedMB) {
|
||||
|
@ -101,6 +101,36 @@ public class NRTCachingDirectory extends Directory {
|
|||
maxCachedBytes = (long) (maxCachedMB*1024*1024);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LockFactory getLockFactory() {
|
||||
return delegate.getLockFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setLockFactory(LockFactory lf) throws IOException {
|
||||
delegate.setLockFactory(lf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLockID() {
|
||||
return delegate.getLockID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lock makeLock(String name) {
|
||||
return delegate.makeLock(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearLock(String name) throws IOException {
|
||||
delegate.clearLock(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "NRTCachingDirectory(" + delegate + "; maxCacheMB=" + (maxCachedBytes/1024/1024.) + " maxMergeSizeMB=" + (maxMergeSizeBytes/1024/1024.) + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized String[] listAll() throws IOException {
|
||||
final Set<String> files = new HashSet<String>();
|
||||
|
@ -108,7 +138,9 @@ public class NRTCachingDirectory extends Directory {
|
|||
files.add(f);
|
||||
}
|
||||
for(String f : delegate.listAll()) {
|
||||
assert !files.contains(f);
|
||||
// Cannot do this -- if lucene calls createOutput but
|
||||
// file already exists then this falsely trips:
|
||||
//assert !files.contains(f): "file \"" + f + "\" is in both dirs";
|
||||
files.add(f);
|
||||
}
|
||||
return files.toArray(new String[files.size()]);
|
||||
|
@ -136,13 +168,16 @@ public class NRTCachingDirectory extends Directory {
|
|||
|
||||
@Override
|
||||
public synchronized void deleteFile(String name) throws IOException {
|
||||
// Delete from both, in case we are currently uncaching:
|
||||
if (VERBOSE) {
|
||||
System.out.println("nrtdir.deleteFile name=" + name);
|
||||
}
|
||||
if (cache.fileExists(name)) {
|
||||
assert !delegate.fileExists(name);
|
||||
cache.deleteFile(name);
|
||||
} else {
|
||||
delegate.deleteFile(name);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long fileLength(String name) throws IOException {
|
||||
|
@ -207,17 +242,7 @@ public class NRTCachingDirectory extends Directory {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lock makeLock(String name) {
|
||||
return delegate.makeLock(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearLock(String name) throws IOException {
|
||||
delegate.clearLock(name);
|
||||
}
|
||||
|
||||
/** Close thius directory, which flushes any cached files
|
||||
/** Close this directory, which flushes any cached files
|
||||
* to the delegate and then closes the delegate. */
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
@ -277,4 +302,3 @@ public class NRTCachingDirectory extends Directory {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -111,4 +111,12 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
|
|||
conf.setMergeScheduler(cachedFSDir.getMergeScheduler());
|
||||
IndexWriter writer = new IndexWriter(cachedFSDir, conf);
|
||||
}
|
||||
|
||||
public void testDeleteFile() throws Exception {
|
||||
Directory dir = new NRTCachingDirectory(newDirectory(), 2.0, 25.0);
|
||||
dir.createOutput("foo.txt").close();
|
||||
dir.deleteFile("foo.txt");
|
||||
assertEquals(0, dir.listAll().length);
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.lucene.util.Version;
|
|||
* become <code>h?user</code> when using this parser and thus no match would be found (i.e.
|
||||
* using this parser will be no improvement over QueryParser in such cases).
|
||||
*
|
||||
* @version $Revision$, $Date$
|
||||
*/
|
||||
public class AnalyzingQueryParser extends org.apache.lucene.queryParser.QueryParser {
|
||||
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.queryParser.ParseException;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
* @version $Revision$, $Date$
|
||||
*/
|
||||
public class TestAnalyzingQueryParser extends LuceneTestCase {
|
||||
|
||||
|
|
|
@ -1859,7 +1859,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
(if compression is enabled, the algorithm used is ZLIB),
|
||||
only available for indexes until Lucene version 2.9.x</li>
|
||||
|
||||
<li>4th to 6th bits (mask: 0x7<<3) define the type of a
|
||||
<li>4th to 6th bit (mask: 0x7<<3) define the type of a
|
||||
numeric field: <ul>
|
||||
|
||||
<li>all bits in mask are cleared if no numeric field at all</li>
|
||||
|
@ -1868,7 +1868,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
|
||||
<li>2<<3: Value is Long</li>
|
||||
|
||||
<li>3<<3: Value is Int as Float (as of Integer.intBitsToFloat)</li>
|
||||
<li>3<<3: Value is Int as Float (as of Float.intBitsToFloat)</li>
|
||||
|
||||
<li>4<<3: Value is Long as Double (as of Double.longBitsToDouble)</li>
|
||||
|
||||
|
|
Binary file not shown.
|
@ -18,7 +18,9 @@ package org.apache.lucene.document;
|
|||
import org.apache.lucene.search.PhraseQuery; // for javadocs
|
||||
import org.apache.lucene.search.spans.SpanQuery; // for javadocs
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.FieldInvertState; // for javadocs
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.util.StringHelper; // for javadocs
|
||||
|
||||
|
||||
|
@ -47,6 +49,8 @@ public abstract class AbstractField implements Fieldable {
|
|||
// length/offset for all primitive types
|
||||
protected int binaryLength;
|
||||
protected int binaryOffset;
|
||||
protected PerDocFieldValues docValues;
|
||||
|
||||
|
||||
protected AbstractField()
|
||||
{
|
||||
|
@ -289,4 +293,20 @@ public abstract class AbstractField implements Fieldable {
|
|||
result.append('>');
|
||||
return result.toString();
|
||||
}
|
||||
|
||||
public PerDocFieldValues getDocValues() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
public void setDocValues(PerDocFieldValues docValues) {
|
||||
this.docValues = docValues;
|
||||
}
|
||||
|
||||
public boolean hasDocValues() {
|
||||
return docValues != null && docValues.type() != null;
|
||||
}
|
||||
|
||||
public ValueType docValuesType() {
|
||||
return docValues == null? null : docValues.type();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,14 +17,15 @@ package org.apache.lucene.document;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.search.NumericRangeQuery; // for javadocs
|
||||
import org.apache.lucene.util.NumericUtils; // for javadocs
|
||||
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import java.util.Locale;
|
||||
import org.apache.lucene.search.NumericRangeQuery; // for javadocs
|
||||
import org.apache.lucene.util.NumericUtils; // for javadocs
|
||||
import java.util.TimeZone;
|
||||
|
||||
/**
|
||||
* Provides support for converting dates to strings and vice-versa.
|
||||
|
@ -47,35 +48,24 @@ import org.apache.lucene.util.NumericUtils; // for javadocs
|
|||
*/
|
||||
public class DateTools {
|
||||
|
||||
private static final class DateFormats {
|
||||
final static TimeZone GMT = TimeZone.getTimeZone("GMT");
|
||||
|
||||
final SimpleDateFormat YEAR_FORMAT = new SimpleDateFormat("yyyy", Locale.US);
|
||||
final SimpleDateFormat MONTH_FORMAT = new SimpleDateFormat("yyyyMM", Locale.US);
|
||||
final SimpleDateFormat DAY_FORMAT = new SimpleDateFormat("yyyyMMdd", Locale.US);
|
||||
final SimpleDateFormat HOUR_FORMAT = new SimpleDateFormat("yyyyMMddHH", Locale.US);
|
||||
final SimpleDateFormat MINUTE_FORMAT = new SimpleDateFormat("yyyyMMddHHmm", Locale.US);
|
||||
final SimpleDateFormat SECOND_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US);
|
||||
final SimpleDateFormat MILLISECOND_FORMAT = new SimpleDateFormat("yyyyMMddHHmmssSSS", Locale.US);
|
||||
{
|
||||
// times need to be normalized so the value doesn't depend on the
|
||||
// location the index is created/used:
|
||||
YEAR_FORMAT.setTimeZone(GMT);
|
||||
MONTH_FORMAT.setTimeZone(GMT);
|
||||
DAY_FORMAT.setTimeZone(GMT);
|
||||
HOUR_FORMAT.setTimeZone(GMT);
|
||||
MINUTE_FORMAT.setTimeZone(GMT);
|
||||
SECOND_FORMAT.setTimeZone(GMT);
|
||||
MILLISECOND_FORMAT.setTimeZone(GMT);
|
||||
}
|
||||
|
||||
final Calendar calInstance = Calendar.getInstance(GMT, Locale.US);
|
||||
}
|
||||
|
||||
private static final ThreadLocal<DateFormats> FORMATS = new ThreadLocal<DateFormats>() {
|
||||
private static final ThreadLocal<Calendar> TL_CAL = new ThreadLocal<Calendar>() {
|
||||
@Override
|
||||
protected DateFormats initialValue() {
|
||||
return new DateFormats();
|
||||
protected Calendar initialValue() {
|
||||
return Calendar.getInstance(GMT, Locale.US);
|
||||
}
|
||||
};
|
||||
|
||||
//indexed by format length
|
||||
private static final ThreadLocal<SimpleDateFormat[]> TL_FORMATS = new ThreadLocal<SimpleDateFormat[]>() {
|
||||
@Override
|
||||
protected SimpleDateFormat[] initialValue() {
|
||||
SimpleDateFormat[] arr = new SimpleDateFormat[Resolution.MILLISECOND.formatLen+1];
|
||||
for (Resolution resolution : Resolution.values()) {
|
||||
arr[resolution.formatLen] = (SimpleDateFormat)resolution.format.clone();
|
||||
}
|
||||
return arr;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -105,22 +95,8 @@ public class DateTools {
|
|||
* depending on <code>resolution</code>; using GMT as timezone
|
||||
*/
|
||||
public static String timeToString(long time, Resolution resolution) {
|
||||
final DateFormats formats = FORMATS.get();
|
||||
|
||||
formats.calInstance.setTimeInMillis(round(time, resolution));
|
||||
final Date date = formats.calInstance.getTime();
|
||||
|
||||
switch (resolution) {
|
||||
case YEAR: return formats.YEAR_FORMAT.format(date);
|
||||
case MONTH:return formats.MONTH_FORMAT.format(date);
|
||||
case DAY: return formats.DAY_FORMAT.format(date);
|
||||
case HOUR: return formats.HOUR_FORMAT.format(date);
|
||||
case MINUTE: return formats.MINUTE_FORMAT.format(date);
|
||||
case SECOND: return formats.SECOND_FORMAT.format(date);
|
||||
case MILLISECOND: return formats.MILLISECOND_FORMAT.format(date);
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException("unknown resolution " + resolution);
|
||||
final Date date = new Date(round(time, resolution));
|
||||
return TL_FORMATS.get()[resolution.formatLen].format(date);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -148,24 +124,11 @@ public class DateTools {
|
|||
* expected format
|
||||
*/
|
||||
public static Date stringToDate(String dateString) throws ParseException {
|
||||
final DateFormats formats = FORMATS.get();
|
||||
|
||||
if (dateString.length() == 4) {
|
||||
return formats.YEAR_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 6) {
|
||||
return formats.MONTH_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 8) {
|
||||
return formats.DAY_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 10) {
|
||||
return formats.HOUR_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 12) {
|
||||
return formats.MINUTE_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 14) {
|
||||
return formats.SECOND_FORMAT.parse(dateString);
|
||||
} else if (dateString.length() == 17) {
|
||||
return formats.MILLISECOND_FORMAT.parse(dateString);
|
||||
try {
|
||||
return TL_FORMATS.get()[dateString.length()].parse(dateString);
|
||||
} catch (Exception e) {
|
||||
throw new ParseException("Input is not a valid date string: " + dateString, 0);
|
||||
}
|
||||
throw new ParseException("Input is not valid date string: " + dateString, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -191,44 +154,25 @@ public class DateTools {
|
|||
* @return the date with all values more precise than <code>resolution</code>
|
||||
* set to 0 or 1, expressed as milliseconds since January 1, 1970, 00:00:00 GMT
|
||||
*/
|
||||
@SuppressWarnings("fallthrough")
|
||||
public static long round(long time, Resolution resolution) {
|
||||
final Calendar calInstance = FORMATS.get().calInstance;
|
||||
final Calendar calInstance = TL_CAL.get();
|
||||
calInstance.setTimeInMillis(time);
|
||||
|
||||
switch (resolution) {
|
||||
//NOTE: switch statement fall-through is deliberate
|
||||
case YEAR:
|
||||
calInstance.set(Calendar.MONTH, 0);
|
||||
calInstance.set(Calendar.DAY_OF_MONTH, 1);
|
||||
calInstance.set(Calendar.HOUR_OF_DAY, 0);
|
||||
calInstance.set(Calendar.MINUTE, 0);
|
||||
calInstance.set(Calendar.SECOND, 0);
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case MONTH:
|
||||
calInstance.set(Calendar.DAY_OF_MONTH, 1);
|
||||
calInstance.set(Calendar.HOUR_OF_DAY, 0);
|
||||
calInstance.set(Calendar.MINUTE, 0);
|
||||
calInstance.set(Calendar.SECOND, 0);
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case DAY:
|
||||
calInstance.set(Calendar.HOUR_OF_DAY, 0);
|
||||
calInstance.set(Calendar.MINUTE, 0);
|
||||
calInstance.set(Calendar.SECOND, 0);
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case HOUR:
|
||||
calInstance.set(Calendar.MINUTE, 0);
|
||||
calInstance.set(Calendar.SECOND, 0);
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case MINUTE:
|
||||
calInstance.set(Calendar.SECOND, 0);
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case SECOND:
|
||||
calInstance.set(Calendar.MILLISECOND, 0);
|
||||
break;
|
||||
case MILLISECOND:
|
||||
// don't cut off anything
|
||||
break;
|
||||
|
@ -241,7 +185,18 @@ public class DateTools {
|
|||
/** Specifies the time granularity. */
|
||||
public static enum Resolution {
|
||||
|
||||
YEAR, MONTH, DAY, HOUR, MINUTE, SECOND, MILLISECOND;
|
||||
YEAR(4), MONTH(6), DAY(8), HOUR(10), MINUTE(12), SECOND(14), MILLISECOND(17);
|
||||
|
||||
final int formatLen;
|
||||
final SimpleDateFormat format;//should be cloned before use, since it's not threadsafe
|
||||
|
||||
Resolution(int formatLen) {
|
||||
this.formatLen = formatLen;
|
||||
// formatLen 10's place: 11111111
|
||||
// formatLen 1's place: 12345678901234567
|
||||
this.format = new SimpleDateFormat("yyyyMMddHHmmssSSS".substring(0,formatLen),Locale.US);
|
||||
this.format.setTimeZone(GMT);
|
||||
}
|
||||
|
||||
/** this method returns the name of the resolution
|
||||
* in lowercase (for backwards compatibility) */
|
||||
|
|
|
@ -18,6 +18,9 @@ package org.apache.lucene.document;
|
|||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.FieldInvertState; // for javadocs
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.PhraseQuery; // for javadocs
|
||||
import org.apache.lucene.search.spans.SpanQuery; // for javadocs
|
||||
|
||||
|
@ -206,4 +209,29 @@ public interface Fieldable {
|
|||
* fail with an exception.
|
||||
*/
|
||||
void setOmitTermFreqAndPositions(boolean omitTermFreqAndPositions);
|
||||
|
||||
/**
|
||||
* Returns the {@link PerDocFieldValues}
|
||||
*/
|
||||
public PerDocFieldValues getDocValues();
|
||||
|
||||
/**
|
||||
* Sets the {@link PerDocFieldValues} for this field. If
|
||||
* {@link PerDocFieldValues} is set this field will store per-document values
|
||||
*
|
||||
* @see IndexDocValues
|
||||
*/
|
||||
public void setDocValues(PerDocFieldValues docValues);
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff {@link PerDocFieldValues} are set on this
|
||||
* field.
|
||||
*/
|
||||
public boolean hasDocValues();
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of the set {@link PerDocFieldValues} or
|
||||
* <code>null</code> if not set.
|
||||
*/
|
||||
public ValueType docValuesType();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,286 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* 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.Reader;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field.TermVector;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* This class provides a {@link AbstractField} that enables storing of typed
|
||||
* per-document values for scoring, sorting or value retrieval. Here's an
|
||||
* example usage, adding an int value:
|
||||
*
|
||||
* <pre>
|
||||
* document.add(new IndexDocValuesField(name).setInt(value));
|
||||
* </pre>
|
||||
*
|
||||
* For optimal performance, re-use the <code>DocValuesField</code> and
|
||||
* {@link Document} instance for more than one document:
|
||||
*
|
||||
* <pre>
|
||||
* IndexDocValuesField field = new IndexDocValuesField(name);
|
||||
* Document document = new Document();
|
||||
* document.add(field);
|
||||
*
|
||||
* for(all documents) {
|
||||
* ...
|
||||
* field.setInt(value)
|
||||
* writer.addDocument(document);
|
||||
* ...
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* <p>
|
||||
* If doc values are stored in addition to an indexed ({@link Index}) or stored
|
||||
* ({@link Store}) value it's recommended to use the {@link IndexDocValuesField}'s
|
||||
* {@link #set(AbstractField)} API:
|
||||
*
|
||||
* <pre>
|
||||
* IndexDocValuesField field = new IndexDocValuesField(name);
|
||||
* Field indexedField = new Field(name, stringValue, Stored.NO, Indexed.ANALYZED);
|
||||
* Document document = new Document();
|
||||
* document.add(indexedField);
|
||||
* field.set(indexedField);
|
||||
* for(all documents) {
|
||||
* ...
|
||||
* field.setInt(value)
|
||||
* writer.addDocument(document);
|
||||
* ...
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* */
|
||||
public class IndexDocValuesField extends AbstractField implements PerDocFieldValues {
|
||||
|
||||
protected BytesRef bytes;
|
||||
protected double doubleValue;
|
||||
protected long longValue;
|
||||
protected ValueType type;
|
||||
protected Comparator<BytesRef> bytesComparator;
|
||||
|
||||
/**
|
||||
* Creates a new {@link IndexDocValuesField} with the given name.
|
||||
*/
|
||||
public IndexDocValuesField(String name) {
|
||||
super(name, Store.NO, Index.NO, TermVector.NO);
|
||||
setDocValues(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a {@link IndexDocValuesField} prototype
|
||||
*/
|
||||
IndexDocValuesField() {
|
||||
this("");
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>long</code> value and sets the field's {@link ValueType} to
|
||||
* {@link ValueType#INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setType(ValueType)}.
|
||||
*/
|
||||
public void setInt(long value) {
|
||||
if (type == null) {
|
||||
type = ValueType.INTS;
|
||||
}
|
||||
longValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>float</code> value and sets the field's {@link ValueType}
|
||||
* to {@link ValueType#FLOAT_32} unless already set. If you want to
|
||||
* change the type use {@link #setType(ValueType)}.
|
||||
*/
|
||||
public void setFloat(float value) {
|
||||
if (type == null) {
|
||||
type = ValueType.FLOAT_32;
|
||||
}
|
||||
doubleValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>double</code> value and sets the field's {@link ValueType}
|
||||
* to {@link ValueType#FLOAT_64} unless already set. If you want to
|
||||
* change the default type use {@link #setType(ValueType)}.
|
||||
*/
|
||||
public void setFloat(double value) {
|
||||
if (type == null) {
|
||||
type = ValueType.FLOAT_64;
|
||||
}
|
||||
doubleValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value and the field's {@link ValueType}. The
|
||||
* comparator for this field is set to <code>null</code>. If a
|
||||
* <code>null</code> comparator is set the default comparator for the given
|
||||
* {@link ValueType} is used.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type) {
|
||||
setBytes(value, type, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value, the field's {@link ValueType} and the
|
||||
* field's comparator. If the {@link Comparator} is set to <code>null</code>
|
||||
* the default for the given {@link ValueType} is used instead.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if the value or the type are null
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type, Comparator<BytesRef> comp) {
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("value must not be null");
|
||||
}
|
||||
setType(type);
|
||||
if (bytes == null) {
|
||||
bytes = new BytesRef(value);
|
||||
} else {
|
||||
bytes.copy(value);
|
||||
}
|
||||
bytesComparator = comp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} or <code>null</code> if not set.
|
||||
*/
|
||||
public BytesRef getBytes() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} comparator or <code>null</code> if not set
|
||||
*/
|
||||
public Comparator<BytesRef> bytesComparator() {
|
||||
return bytesComparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the set floating point value or <code>0.0d</code> if not set.
|
||||
*/
|
||||
public double getFloat() {
|
||||
return doubleValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the set <code>long</code> value of <code>0</code> if not set.
|
||||
*/
|
||||
public long getInt() {
|
||||
return longValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link BytesRef} comparator for this field. If the field has a
|
||||
* numeric {@link ValueType} the comparator will be ignored.
|
||||
*/
|
||||
public void setBytesComparator(Comparator<BytesRef> comp) {
|
||||
this.bytesComparator = comp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link ValueType} for this field.
|
||||
*/
|
||||
public void setType(ValueType type) {
|
||||
if (type == null) {
|
||||
throw new IllegalArgumentException("Type must not be null");
|
||||
}
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the field's {@link ValueType}
|
||||
*/
|
||||
public ValueType type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns always <code>null</code>
|
||||
*/
|
||||
public Reader readerValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns always <code>null</code>
|
||||
*/
|
||||
public String stringValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns always <code>null</code>
|
||||
*/
|
||||
public TokenStream tokenStreamValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets this {@link IndexDocValuesField} to the given {@link AbstractField} and
|
||||
* returns the given field. Any modifications to this instance will be visible
|
||||
* to the given field.
|
||||
*/
|
||||
public <T extends AbstractField> T set(T field) {
|
||||
field.setDocValues(this);
|
||||
return field;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets a new {@link PerDocFieldValues} instance on the given field with the
|
||||
* given type and returns it.
|
||||
*
|
||||
*/
|
||||
public static <T extends AbstractField> T set(T field, ValueType type) {
|
||||
if (field instanceof IndexDocValuesField)
|
||||
return field;
|
||||
final IndexDocValuesField valField = new IndexDocValuesField();
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
BytesRef ref = field.isBinary() ? new BytesRef(field.getBinaryValue(),
|
||||
field.getBinaryOffset(), field.getBinaryLength()) : new BytesRef(
|
||||
field.stringValue());
|
||||
valField.setBytes(ref, type);
|
||||
break;
|
||||
case INTS:
|
||||
valField.setInt(Long.parseLong(field.stringValue()));
|
||||
break;
|
||||
case FLOAT_32:
|
||||
valField.setFloat(Float.parseFloat(field.stringValue()));
|
||||
break;
|
||||
case FLOAT_64:
|
||||
valField.setFloat(Double.parseDouble(field.stringValue()));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("unknown type: " + type);
|
||||
}
|
||||
return valField.set(field);
|
||||
}
|
||||
|
||||
}
|
|
@ -27,6 +27,9 @@ import org.apache.lucene.document.AbstractField; // for javadocs
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValuesEnum;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -195,6 +198,9 @@ public class CheckIndex {
|
|||
|
||||
/** Status for testing of term vectors (null if term vectors could not be tested). */
|
||||
public TermVectorStatus termVectorStatus;
|
||||
|
||||
/** Status for testing of DocValues (null if DocValues could not be tested). */
|
||||
public DocValuesStatus docValuesStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -254,6 +260,15 @@ public class CheckIndex {
|
|||
/** Exception thrown during term vector test (null on success) */
|
||||
public Throwable error = null;
|
||||
}
|
||||
|
||||
public static final class DocValuesStatus {
|
||||
/** Number of documents tested. */
|
||||
public int docCount;
|
||||
/** Total number of docValues tested. */
|
||||
public long totalValueFields;
|
||||
/** Exception thrown during doc values test (null on success) */
|
||||
public Throwable error = null;
|
||||
}
|
||||
}
|
||||
|
||||
/** Create a new CheckIndex on the directory. */
|
||||
|
@ -500,6 +515,8 @@ public class CheckIndex {
|
|||
// Test Term Vectors
|
||||
segInfoStat.termVectorStatus = testTermVectors(info, reader, nf);
|
||||
|
||||
segInfoStat.docValuesStatus = testDocValues(info, reader);
|
||||
|
||||
// Rethrow the first exception we encountered
|
||||
// This will cause stats for failed segments to be incremented properly
|
||||
if (segInfoStat.fieldNormStatus.error != null) {
|
||||
|
@ -510,6 +527,8 @@ public class CheckIndex {
|
|||
throw new RuntimeException("Stored Field test failed");
|
||||
} else if (segInfoStat.termVectorStatus.error != null) {
|
||||
throw new RuntimeException("Term Vector test failed");
|
||||
} else if (segInfoStat.docValuesStatus.error != null) {
|
||||
throw new RuntimeException("DocValues test failed");
|
||||
}
|
||||
|
||||
msg("");
|
||||
|
@ -921,6 +940,60 @@ public class CheckIndex {
|
|||
return status;
|
||||
}
|
||||
|
||||
private Status.DocValuesStatus testDocValues(SegmentInfo info,
|
||||
SegmentReader reader) {
|
||||
final Status.DocValuesStatus status = new Status.DocValuesStatus();
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
infoStream.print(" test: DocValues........");
|
||||
}
|
||||
final FieldInfos fieldInfos = info.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
status.totalValueFields++;
|
||||
final PerDocValues perDocValues = reader.perDocValues();
|
||||
final IndexDocValues docValues = perDocValues.docValues(fieldInfo.name);
|
||||
if (docValues == null) {
|
||||
continue;
|
||||
}
|
||||
final ValuesEnum values = docValues.getEnum();
|
||||
while (values.nextDoc() != ValuesEnum.NO_MORE_DOCS) {
|
||||
switch (fieldInfo.docValues) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
values.bytes();
|
||||
break;
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
values.getFloat();
|
||||
break;
|
||||
case INTS:
|
||||
values.getInt();
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Field: " + fieldInfo.name
|
||||
+ " - no such DocValues type: " + fieldInfo.docValues);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
msg("OK [" + status.docCount + " total doc Count; Num DocValues Fields "
|
||||
+ status.totalValueFields);
|
||||
} catch (Throwable e) {
|
||||
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
|
||||
status.error = e;
|
||||
if (infoStream != null) {
|
||||
e.printStackTrace(infoStream);
|
||||
}
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test term vectors for a segment.
|
||||
*/
|
||||
|
|
|
@ -158,7 +158,7 @@ public class CompoundFileReader extends Directory {
|
|||
throw new IOException("Stream closed");
|
||||
|
||||
id = IndexFileNames.stripSegmentName(id);
|
||||
FileEntry entry = entries.get(id);
|
||||
final FileEntry entry = entries.get(id);
|
||||
if (entry == null)
|
||||
throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.Lock;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
@ -1022,4 +1023,9 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
throw new UnsupportedOperationException("This IndexCommit does not support deletions");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiPerDocValues#getPerDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,10 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
||||
|
||||
|
@ -80,6 +84,9 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
// FieldInfo.storePayload.
|
||||
final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
|
||||
state.fieldInfos.write(state.directory, fileName);
|
||||
for (DocValuesConsumer consumers : docValues.values()) {
|
||||
consumers.finish(state.numDocs);
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -100,6 +107,14 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
for(PerDocConsumer consumer : perDocConsumers.values()) {
|
||||
try {
|
||||
consumer.close(); // TODO add abort to PerDocConsumer!
|
||||
} catch (IOException e) {
|
||||
// ignore on abort!
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
fieldsWriter.abort();
|
||||
} catch (Throwable t) {
|
||||
|
@ -150,6 +165,15 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
fieldHash = new DocFieldProcessorPerField[2];
|
||||
hashMask = 1;
|
||||
totalFieldCount = 0;
|
||||
for(PerDocConsumer consumer : perDocConsumers.values()) {
|
||||
try {
|
||||
consumer.close();
|
||||
} catch (IOException e) {
|
||||
// ignore and continue closing remaining consumers
|
||||
}
|
||||
}
|
||||
perDocConsumers.clear();
|
||||
docValues.clear();
|
||||
}
|
||||
|
||||
private void rehash() {
|
||||
|
@ -215,7 +239,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
// easily add it
|
||||
FieldInfo fi = fieldInfos.addOrUpdate(fieldName, field.isIndexed(), field.isTermVectorStored(),
|
||||
field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
|
||||
field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
|
||||
field.getOmitNorms(), false, field.getOmitTermFreqAndPositions(), field.docValuesType());
|
||||
|
||||
fp = new DocFieldProcessorPerField(this, fi);
|
||||
fp.next = fieldHash[hashPos];
|
||||
|
@ -227,7 +251,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
} else {
|
||||
fieldInfos.addOrUpdate(fp.fieldInfo.name, field.isIndexed(), field.isTermVectorStored(),
|
||||
field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
|
||||
field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
|
||||
field.getOmitNorms(), false, field.getOmitTermFreqAndPositions(), field.docValuesType());
|
||||
}
|
||||
|
||||
if (thisFieldGen != fp.lastGen) {
|
||||
|
@ -251,6 +275,10 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
if (field.isStored()) {
|
||||
fieldsWriter.addField(field, fp.fieldInfo);
|
||||
}
|
||||
if (field.hasDocValues()) {
|
||||
final DocValuesConsumer docValuesConsumer = docValuesConsumer(docState, fp.fieldInfo);
|
||||
docValuesConsumer.add(docState.docID, field.getDocValues());
|
||||
}
|
||||
}
|
||||
|
||||
// If we are writing vectors then we must visit
|
||||
|
@ -286,4 +314,36 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
final private Map<String, DocValuesConsumer> docValues = new HashMap<String, DocValuesConsumer>();
|
||||
final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
|
||||
|
||||
DocValuesConsumer docValuesConsumer(DocState docState, FieldInfo fieldInfo)
|
||||
throws IOException {
|
||||
DocValuesConsumer docValuesConsumer = docValues.get(fieldInfo.name);
|
||||
if (docValuesConsumer != null) {
|
||||
return docValuesConsumer;
|
||||
}
|
||||
PerDocConsumer perDocConsumer = perDocConsumers.get(fieldInfo.getCodecId());
|
||||
if (perDocConsumer == null) {
|
||||
PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(fieldInfo.getCodecId());
|
||||
SegmentCodecs codecs = perDocWriteState.segmentCodecs;
|
||||
assert codecs.codecs.length > fieldInfo.getCodecId();
|
||||
Codec codec = codecs.codecs[fieldInfo.getCodecId()];
|
||||
perDocConsumer = codec.docsConsumer(perDocWriteState);
|
||||
perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
|
||||
}
|
||||
boolean success = false;
|
||||
try {
|
||||
docValuesConsumer = perDocConsumer.addValuesField(fieldInfo);
|
||||
fieldInfo.commitDocValues();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
fieldInfo.revertUncommitted();
|
||||
}
|
||||
}
|
||||
docValues.put(fieldInfo.name, docValuesConsumer);
|
||||
return docValuesConsumer;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.search.SimilarityProvider;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
public class DocumentsWriterPerThread {
|
||||
|
@ -169,6 +170,7 @@ public class DocumentsWriterPerThread {
|
|||
DocumentsWriterDeleteQueue deleteQueue;
|
||||
DeleteSlice deleteSlice;
|
||||
private final NumberFormat nf = NumberFormat.getInstance();
|
||||
final Allocator byteBlockAllocator;
|
||||
|
||||
|
||||
public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
|
||||
|
@ -181,9 +183,9 @@ public class DocumentsWriterPerThread {
|
|||
this.docState = new DocState(this);
|
||||
this.docState.similarityProvider = parent.indexWriter.getConfig()
|
||||
.getSimilarityProvider();
|
||||
|
||||
consumer = indexingChain.getChain(this);
|
||||
bytesUsed = new AtomicLong(0);
|
||||
byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
|
||||
consumer = indexingChain.getChain(this);
|
||||
pendingDeletes = new BufferedDeletes(false);
|
||||
initialize();
|
||||
}
|
||||
|
@ -538,36 +540,13 @@ public class DocumentsWriterPerThread {
|
|||
bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
|
||||
}
|
||||
|
||||
final Allocator byteBlockAllocator = new DirectTrackingAllocator();
|
||||
|
||||
|
||||
private class DirectTrackingAllocator extends Allocator {
|
||||
public DirectTrackingAllocator() {
|
||||
this(BYTE_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
public DirectTrackingAllocator(int blockSize) {
|
||||
super(blockSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getByteBlock() {
|
||||
bytesUsed.addAndGet(blockSize);
|
||||
return new byte[blockSize];
|
||||
}
|
||||
@Override
|
||||
public void recycleByteBlocks(byte[][] blocks, int start, int end) {
|
||||
bytesUsed.addAndGet(-((end-start)* blockSize));
|
||||
for (int i = start; i < end; i++) {
|
||||
blocks[i] = null;
|
||||
}
|
||||
}
|
||||
|
||||
PerDocWriteState newPerDocWriteState(int codecId) {
|
||||
assert segment != null;
|
||||
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId);
|
||||
}
|
||||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
docState.infoStream = infoStream;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,5 +1,7 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
|
@ -24,6 +26,8 @@ public final class FieldInfo {
|
|||
public final int number;
|
||||
|
||||
public boolean isIndexed;
|
||||
ValueType docValues;
|
||||
|
||||
|
||||
// true if term vector for this field should be stored
|
||||
boolean storeTermVector;
|
||||
|
@ -38,10 +42,11 @@ public final class FieldInfo {
|
|||
|
||||
FieldInfo(String na, boolean tk, int nu, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions, ValueType docValues) {
|
||||
name = na;
|
||||
isIndexed = tk;
|
||||
number = nu;
|
||||
this.docValues = docValues;
|
||||
if (isIndexed) {
|
||||
this.storeTermVector = storeTermVector;
|
||||
this.storeOffsetWithTermVector = storeOffsetWithTermVector;
|
||||
|
@ -72,7 +77,7 @@ public final class FieldInfo {
|
|||
@Override
|
||||
public Object clone() {
|
||||
FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValues);
|
||||
clone.codecId = this.codecId;
|
||||
return clone;
|
||||
}
|
||||
|
@ -107,7 +112,22 @@ public final class FieldInfo {
|
|||
}
|
||||
assert !this.omitTermFreqAndPositions || !this.storePayloads;
|
||||
}
|
||||
void setDocValues(ValueType v) {
|
||||
if (docValues == null) {
|
||||
docValues = v;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean hasDocValues() {
|
||||
return docValues != null;
|
||||
}
|
||||
|
||||
public ValueType getDocValues() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
private boolean vectorsCommitted;
|
||||
private boolean docValuesCommitted;
|
||||
|
||||
/**
|
||||
* Reverts all uncommitted changes on this {@link FieldInfo}
|
||||
|
@ -119,6 +139,10 @@ public final class FieldInfo {
|
|||
storePositionWithTermVector = false;
|
||||
storeTermVector = false;
|
||||
}
|
||||
|
||||
if (docValues != null && !docValuesCommitted) {
|
||||
docValues = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -131,4 +155,9 @@ public final class FieldInfo {
|
|||
assert storeTermVector;
|
||||
vectorsCommitted = true;
|
||||
}
|
||||
|
||||
void commitDocValues() {
|
||||
assert hasDocValues();
|
||||
docValuesCommitted = true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.Map.Entry;
|
|||
import org.apache.lucene.index.SegmentCodecs; // Required for Java 1.5 javadocs
|
||||
import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -202,6 +203,9 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
public static final int FORMAT_START = -2;
|
||||
public static final int FORMAT_PER_FIELD_CODEC = -3;
|
||||
|
||||
// Records index values for this field
|
||||
public static final int FORMAT_INDEX_VALUES = -3;
|
||||
|
||||
// whenever you add a new format, make it 1 smaller (negative version logic)!
|
||||
static final int FORMAT_CURRENT = FORMAT_PER_FIELD_CODEC;
|
||||
|
||||
|
@ -410,7 +414,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, boolean omitNorms) {
|
||||
addOrUpdate(name, isIndexed, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, false, false);
|
||||
storeOffsetWithTermVector, omitNorms, false, false, null);
|
||||
}
|
||||
|
||||
/** If the field is not yet known, adds it. If it is known, checks to make
|
||||
|
@ -429,14 +433,14 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
synchronized public FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions, ValueType docValues) {
|
||||
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValues);
|
||||
}
|
||||
|
||||
synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
|
||||
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions, ValueType docValues) {
|
||||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
|
||||
}
|
||||
|
@ -444,11 +448,12 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldInfo fi = fieldInfo(name);
|
||||
if (fi == null) {
|
||||
final int fieldNumber = nextFieldNumber(name, preferredFieldNumber);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValues);
|
||||
} else {
|
||||
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
fi.setDocValues(docValues);
|
||||
}
|
||||
if (fi.isIndexed && fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
|
||||
if ((fi.isIndexed || fi.hasDocValues()) && fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
|
||||
segmentCodecsBuilder.tryAddAndSet(fi);
|
||||
}
|
||||
version++;
|
||||
|
@ -460,7 +465,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed, fi.storeTermVector,
|
||||
fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
|
||||
fi.omitNorms, fi.storePayloads,
|
||||
fi.omitTermFreqAndPositions);
|
||||
fi.omitTermFreqAndPositions, fi.docValues);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -468,15 +473,14 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions, ValueType docValuesType) {
|
||||
// don't check modifiable here since we use that to initially build up FIs
|
||||
name = StringHelper.intern(name);
|
||||
if (globalFieldNumbers != null) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
}
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValuesType);
|
||||
putInternal(fi);
|
||||
return fi;
|
||||
}
|
||||
|
@ -600,6 +604,45 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
output.writeInt(fi.number);
|
||||
output.writeInt(fi.getCodecId());
|
||||
output.writeByte(bits);
|
||||
|
||||
final byte b;
|
||||
|
||||
if (fi.docValues == null) {
|
||||
b = 0;
|
||||
} else {
|
||||
switch(fi.docValues) {
|
||||
case INTS:
|
||||
b = 1;
|
||||
break;
|
||||
case FLOAT_32:
|
||||
b = 2;
|
||||
break;
|
||||
case FLOAT_64:
|
||||
b = 3;
|
||||
break;
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
b = 4;
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
b = 5;
|
||||
break;
|
||||
case BYTES_FIXED_SORTED:
|
||||
b = 6;
|
||||
break;
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
b = 7;
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
b = 8;
|
||||
break;
|
||||
case BYTES_VAR_SORTED:
|
||||
b = 9;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("unhandled indexValues type " + fi.docValues);
|
||||
}
|
||||
}
|
||||
output.writeByte(b);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -637,7 +680,45 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
hasVectors |= storeTermVector;
|
||||
hasProx |= isIndexed && !omitTermFreqAndPositions;
|
||||
final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
|
||||
ValueType docValuesType = null;
|
||||
if (format <= FORMAT_INDEX_VALUES) {
|
||||
final byte b = input.readByte();
|
||||
switch(b) {
|
||||
case 0:
|
||||
docValuesType = null;
|
||||
break;
|
||||
case 1:
|
||||
docValuesType = ValueType.INTS;
|
||||
break;
|
||||
case 2:
|
||||
docValuesType = ValueType.FLOAT_32;
|
||||
break;
|
||||
case 3:
|
||||
docValuesType = ValueType.FLOAT_64;
|
||||
break;
|
||||
case 4:
|
||||
docValuesType = ValueType.BYTES_FIXED_STRAIGHT;
|
||||
break;
|
||||
case 5:
|
||||
docValuesType = ValueType.BYTES_FIXED_DEREF;
|
||||
break;
|
||||
case 6:
|
||||
docValuesType = ValueType.BYTES_FIXED_SORTED;
|
||||
break;
|
||||
case 7:
|
||||
docValuesType = ValueType.BYTES_VAR_STRAIGHT;
|
||||
break;
|
||||
case 8:
|
||||
docValuesType = ValueType.BYTES_VAR_DEREF;
|
||||
break;
|
||||
case 9:
|
||||
docValuesType = ValueType.BYTES_VAR_SORTED;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("unhandled indexValues type " + b);
|
||||
}
|
||||
}
|
||||
final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValuesType);
|
||||
addInternal.setCodecId(codecId);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -463,4 +463,9 @@ public class FilterIndexReader extends IndexReader {
|
|||
super.removeReaderFinishedListener(listener);
|
||||
in.removeReaderFinishedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return in.perDocValues();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.FilenameFilter;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
|
|
|
@ -84,7 +84,6 @@ public final class IndexFileNames {
|
|||
/** Extension of global field numbers */
|
||||
public static final String GLOBAL_FIELD_NUM_MAP_EXTENSION = "fnx";
|
||||
|
||||
|
||||
/**
|
||||
* This array contains all filename extensions used by
|
||||
* Lucene's index files, with one exception, namely the
|
||||
|
@ -209,6 +208,11 @@ public final class IndexFileNames {
|
|||
}
|
||||
}
|
||||
|
||||
/** Sugar for passing "" + name instead */
|
||||
public static String segmentFileName(String segmentName, int name, String ext) {
|
||||
return segmentFileName(segmentName, ""+name, ext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the given filename ends with the given extension. One
|
||||
* should provide a <i>pure</i> extension, without '.'.
|
||||
|
|
|
@ -23,6 +23,8 @@ import org.apache.lucene.search.FieldCache; // javadocs
|
|||
import org.apache.lucene.search.Similarity;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -174,6 +176,9 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
public static final FieldOption TERMVECTOR_WITH_OFFSET = new FieldOption ("TERMVECTOR_WITH_OFFSET");
|
||||
/** All fields with termvectors with offset values and position values enabled */
|
||||
public static final FieldOption TERMVECTOR_WITH_POSITION_OFFSET = new FieldOption ("TERMVECTOR_WITH_POSITION_OFFSET");
|
||||
/** All fields holding doc values */
|
||||
public static final FieldOption DOC_VALUES = new FieldOption ("DOC_VALUES");
|
||||
|
||||
}
|
||||
|
||||
private boolean closed;
|
||||
|
@ -1051,7 +1056,8 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
protected abstract void doSetNorm(int doc, String field, byte value)
|
||||
throws CorruptIndexException, IOException;
|
||||
|
||||
/** Flex API: returns {@link Fields} for this reader.
|
||||
/**
|
||||
* Returns {@link Fields} for this reader.
|
||||
* This method may return null if the reader has no
|
||||
* postings.
|
||||
*
|
||||
|
@ -1065,6 +1071,21 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* through them yourself. */
|
||||
public abstract Fields fields() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns {@link PerDocValues} for this reader.
|
||||
* This method may return null if the reader has no per-document
|
||||
* values stored.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this is a multi reader ({@link
|
||||
* #getSequentialSubReaders} is not null) then this
|
||||
* method will throw UnsupportedOperationException. If
|
||||
* you really need {@link PerDocValues} for such a reader,
|
||||
* use {@link MultiPerDocValues#getPerDocs(IndexReader)}. However, for
|
||||
* performance reasons, it's best to get all sub-readers
|
||||
* using {@link ReaderUtil#gatherSubReaders} and iterate
|
||||
* through them yourself. */
|
||||
public abstract PerDocValues perDocValues() throws IOException;
|
||||
|
||||
public int docFreq(Term term) throws IOException {
|
||||
return docFreq(term.field(), term.bytes());
|
||||
}
|
||||
|
@ -1566,6 +1587,13 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
throw new UnsupportedOperationException("This reader does not support this method.");
|
||||
}
|
||||
|
||||
public final IndexDocValues docValues(String field) throws IOException {
|
||||
final PerDocValues perDoc = perDocValues();
|
||||
if (perDoc == null) {
|
||||
return null;
|
||||
}
|
||||
return perDoc.docValues(field);
|
||||
}
|
||||
|
||||
private volatile Fields fields;
|
||||
|
||||
|
@ -1579,6 +1607,19 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
return fields;
|
||||
}
|
||||
|
||||
private volatile PerDocValues perDocValues;
|
||||
|
||||
/** @lucene.internal */
|
||||
void storePerDoc(PerDocValues perDocValues) {
|
||||
this.perDocValues = perDocValues;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
PerDocValues retrievePerDoc() {
|
||||
return perDocValues;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A struct like class that represents a hierarchical relationship between
|
||||
* {@link IndexReader} instances.
|
||||
|
|
|
@ -1999,6 +1999,9 @@ public class IndexWriter implements Closeable {
|
|||
// will always write to a new generation ("write
|
||||
// once").
|
||||
segmentInfos.rollbackSegmentInfos(rollbackSegments);
|
||||
if (infoStream != null ) {
|
||||
message("rollback: infos=" + segString(segmentInfos));
|
||||
}
|
||||
|
||||
docWriter.abort();
|
||||
|
||||
|
@ -2439,6 +2442,8 @@ public class IndexWriter implements Closeable {
|
|||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
// TODO: somehow we should fix this merge so it's
|
||||
// abortable so that IW.close(false) is able to stop it
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
|
||||
mergedName, null, payloadProcessorProvider,
|
||||
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
|
||||
|
@ -2456,6 +2461,11 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
boolean useCompoundFile;
|
||||
synchronized(this) { // Guard segmentInfos
|
||||
if (stopMerges) {
|
||||
deleter.deleteNewFiles(info.files());
|
||||
return;
|
||||
}
|
||||
ensureOpen();
|
||||
useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, info);
|
||||
}
|
||||
|
||||
|
@ -2471,6 +2481,11 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
// Register the new segment
|
||||
synchronized(this) {
|
||||
if (stopMerges) {
|
||||
deleter.deleteNewFiles(info.files());
|
||||
return;
|
||||
}
|
||||
ensureOpen();
|
||||
segmentInfos.add(info);
|
||||
checkpoint();
|
||||
}
|
||||
|
@ -3076,6 +3091,7 @@ public class IndexWriter implements Closeable {
|
|||
boolean success = false;
|
||||
|
||||
final long t0 = System.currentTimeMillis();
|
||||
//System.out.println(Thread.currentThread().getName() + ": merge start: size=" + (merge.estimatedMergeBytes/1024./1024.) + " MB\n merge=" + merge.segString(directory) + "\n idx=" + segString());
|
||||
|
||||
try {
|
||||
try {
|
||||
|
@ -3116,6 +3132,7 @@ public class IndexWriter implements Closeable {
|
|||
if (infoStream != null && merge.info != null) {
|
||||
message("merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": merge end");
|
||||
}
|
||||
|
||||
/** Hook that's called when the specified merge is complete. */
|
||||
|
@ -3734,6 +3751,8 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
assert testPoint("midStartCommit");
|
||||
|
||||
boolean pendingCommitSet = false;
|
||||
|
||||
try {
|
||||
// This call can take a long time -- 10s of seconds
|
||||
// or more. We do it without sync:
|
||||
|
@ -3753,6 +3772,7 @@ public class IndexWriter implements Closeable {
|
|||
toSync.prepareCommit(directory);
|
||||
|
||||
pendingCommit = toSync;
|
||||
pendingCommitSet = true;
|
||||
pendingCommitChangeCount = myChangeCount;
|
||||
}
|
||||
|
||||
|
@ -3770,7 +3790,7 @@ public class IndexWriter implements Closeable {
|
|||
// double-write a segments_N file.
|
||||
segmentInfos.updateGeneration(toSync);
|
||||
|
||||
if (pendingCommit == null) {
|
||||
if (!pendingCommitSet) {
|
||||
if (infoStream != null) {
|
||||
message("hit exception committing segments file");
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
|
@ -38,10 +39,14 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
// Holds sub-readers containing field we are currently
|
||||
// on, popped from queue.
|
||||
private final FieldsEnumWithSlice[] top;
|
||||
private final FieldsEnumWithSlice[] enumWithSlices;
|
||||
|
||||
private int numTop;
|
||||
|
||||
// Re-used TermsEnum
|
||||
private final MultiTermsEnum terms;
|
||||
private final MultiIndexDocValues docValues;
|
||||
|
||||
|
||||
private String currentField;
|
||||
|
||||
|
@ -50,7 +55,9 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
public MultiFieldsEnum(FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
|
||||
terms = new MultiTermsEnum(subSlices);
|
||||
queue = new FieldMergeQueue(subs.length);
|
||||
docValues = new MultiIndexDocValues();
|
||||
top = new FieldsEnumWithSlice[subs.length];
|
||||
List<FieldsEnumWithSlice> enumWithSlices = new ArrayList<FieldsEnumWithSlice>();
|
||||
|
||||
// Init q
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
|
@ -59,10 +66,13 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
if (field != null) {
|
||||
// this FieldsEnum has at least one field
|
||||
final FieldsEnumWithSlice sub = new FieldsEnumWithSlice(subs[i], subSlices[i], i);
|
||||
enumWithSlices.add(sub);
|
||||
sub.current = field;
|
||||
queue.add(sub);
|
||||
}
|
||||
}
|
||||
this.enumWithSlices = enumWithSlices.toArray(FieldsEnumWithSlice.EMPTY_ARRAY);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -114,6 +124,7 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
}
|
||||
|
||||
public final static class FieldsEnumWithSlice {
|
||||
public static final FieldsEnumWithSlice[] EMPTY_ARRAY = new FieldsEnumWithSlice[0];
|
||||
final FieldsEnum fields;
|
||||
final ReaderUtil.Slice slice;
|
||||
final int index;
|
||||
|
|
|
@ -0,0 +1,174 @@
|
|||
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.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues.DocValuesIndex;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.ReaderUtil.Gather;
|
||||
|
||||
/**
|
||||
* Exposes per-document values, merged from per-document values API of
|
||||
* sub-segments. This is useful when you're interacting with an {@link IndexReader}
|
||||
* implementation that consists of sequential sub-readers (eg DirectoryReader
|
||||
* or {@link MultiReader}).
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: for multi readers, you'll get better performance by gathering
|
||||
* the sub readers using {@link ReaderUtil#gatherSubReaders} and then operate
|
||||
* per-reader, instead of using this class.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class MultiPerDocValues extends PerDocValues {
|
||||
private final PerDocValues[] subs;
|
||||
private final ReaderUtil.Slice[] subSlices;
|
||||
private final Map<String, IndexDocValues> docValues = new ConcurrentHashMap<String, IndexDocValues>();
|
||||
private final TreeSet<String> fields;
|
||||
|
||||
public MultiPerDocValues(PerDocValues[] subs, ReaderUtil.Slice[] subSlices) {
|
||||
this.subs = subs;
|
||||
this.subSlices = subSlices;
|
||||
fields = new TreeSet<String>();
|
||||
for (PerDocValues sub : subs) {
|
||||
fields.addAll(sub.fields());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a single {@link PerDocValues} instance for this reader, merging
|
||||
* their values on the fly. This method will not return <code>null</code>.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: this is a slow way to access postings. It's better to get the
|
||||
* sub-readers (using {@link Gather}) and iterate through them yourself.
|
||||
*/
|
||||
public static PerDocValues getPerDocs(IndexReader r) throws IOException {
|
||||
final IndexReader[] subs = r.getSequentialSubReaders();
|
||||
if (subs == null) {
|
||||
// already an atomic reader
|
||||
return r.perDocValues();
|
||||
} else if (subs.length == 0) {
|
||||
// no fields
|
||||
return null;
|
||||
} else if (subs.length == 1) {
|
||||
return getPerDocs(subs[0]);
|
||||
}
|
||||
PerDocValues perDocValues = r.retrievePerDoc();
|
||||
if (perDocValues == null) {
|
||||
|
||||
final List<PerDocValues> producer = new ArrayList<PerDocValues>();
|
||||
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
|
||||
|
||||
new ReaderUtil.Gather(r) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) throws IOException {
|
||||
final PerDocValues f = r.perDocValues();
|
||||
if (f != null) {
|
||||
producer.add(f);
|
||||
slices
|
||||
.add(new ReaderUtil.Slice(base, r.maxDoc(), producer.size() - 1));
|
||||
}
|
||||
}
|
||||
}.run();
|
||||
|
||||
if (producer.size() == 0) {
|
||||
return null;
|
||||
} else if (producer.size() == 1) {
|
||||
perDocValues = producer.get(0);
|
||||
} else {
|
||||
perDocValues = new MultiPerDocValues(
|
||||
producer.toArray(PerDocValues.EMPTY_ARRAY),
|
||||
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
|
||||
}
|
||||
r.storePerDoc(perDocValues);
|
||||
}
|
||||
return perDocValues;
|
||||
}
|
||||
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
IndexDocValues result = docValues.get(field);
|
||||
if (result == null) {
|
||||
// Lazy init: first time this field is requested, we
|
||||
// create & add to docValues:
|
||||
final List<MultiIndexDocValues.DocValuesIndex> docValuesIndex = new ArrayList<MultiIndexDocValues.DocValuesIndex>();
|
||||
int docsUpto = 0;
|
||||
ValueType type = null;
|
||||
// Gather all sub-readers that share this field
|
||||
for (int i = 0; i < subs.length; i++) {
|
||||
IndexDocValues values = subs[i].docValues(field);
|
||||
final int start = subSlices[i].start;
|
||||
final int length = subSlices[i].length;
|
||||
if (values != null) {
|
||||
if (docsUpto != start) {
|
||||
type = values.type();
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(
|
||||
new MultiIndexDocValues.DummyDocValues(start, type), docsUpto, start
|
||||
- docsUpto));
|
||||
}
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(values, start,
|
||||
length));
|
||||
docsUpto = start + length;
|
||||
|
||||
} else if (i + 1 == subs.length && !docValuesIndex.isEmpty()) {
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(
|
||||
new MultiIndexDocValues.DummyDocValues(start, type), docsUpto, start
|
||||
- docsUpto));
|
||||
}
|
||||
}
|
||||
if (docValuesIndex.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
result = new MultiIndexDocValues(
|
||||
docValuesIndex.toArray(DocValuesIndex.EMPTY_ARRAY));
|
||||
docValues.put(field, result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
final PerDocValues[] perDocValues = this.subs;
|
||||
IOException ex = null;
|
||||
for (PerDocValues values : perDocValues) {
|
||||
try {
|
||||
values.close();
|
||||
} catch (IOException e) {
|
||||
if (ex == null) {
|
||||
ex = e;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (ex != null) {
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return fields;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
@ -403,4 +404,9 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
sub.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiPerDocValues#getPerDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.document.FieldSelectorResult;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
@ -60,7 +62,8 @@ public class ParallelReader extends IndexReader {
|
|||
private int numDocs;
|
||||
private boolean hasDeletions;
|
||||
|
||||
private ParallelFields fields = new ParallelFields();
|
||||
private final ParallelFields fields = new ParallelFields();
|
||||
private final ParallelPerDocs perDocs = new ParallelPerDocs();
|
||||
|
||||
/** Construct a ParallelReader.
|
||||
* <p>Note that all subreaders are closed if this ParallelReader is closed.</p>
|
||||
|
@ -134,6 +137,7 @@ public class ParallelReader extends IndexReader {
|
|||
fieldToReader.put(field, reader);
|
||||
}
|
||||
this.fields.addField(field, reader);
|
||||
this.perDocs.addField(field, reader);
|
||||
}
|
||||
|
||||
if (!ignoreStoredFields)
|
||||
|
@ -180,6 +184,7 @@ public class ParallelReader extends IndexReader {
|
|||
return TermsEnum.EMPTY;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Single instance of this, per ParallelReader instance
|
||||
|
@ -187,7 +192,8 @@ public class ParallelReader extends IndexReader {
|
|||
final HashMap<String,Terms> fields = new HashMap<String,Terms>();
|
||||
|
||||
public void addField(String field, IndexReader r) throws IOException {
|
||||
fields.put(field, MultiFields.getFields(r).terms(field));
|
||||
Fields multiFields = MultiFields.getFields(r);
|
||||
fields.put(field, multiFields.terms(field));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -563,6 +569,36 @@ public class ParallelReader extends IndexReader {
|
|||
reader.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return perDocs;
|
||||
}
|
||||
|
||||
// Single instance of this, per ParallelReader instance
|
||||
private static final class ParallelPerDocs extends PerDocValues {
|
||||
final TreeMap<String,IndexDocValues> fields = new TreeMap<String,IndexDocValues>();
|
||||
|
||||
void addField(String field, IndexReader r) throws IOException {
|
||||
PerDocValues perDocs = MultiPerDocValues.getPerDocs(r);
|
||||
fields.put(field, perDocs.docValues(field));
|
||||
}
|
||||
|
||||
//@Override -- not until Java 1.6
|
||||
public void close() throws IOException {
|
||||
// nothing to do here
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return fields.keySet();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,70 @@
|
|||
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.PrintStream;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Encapsulates all necessary state to initiate a {@link PerDocConsumer} and
|
||||
* create all necessary files in order to consume and merge per-document values.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class PerDocWriteState {
|
||||
public final PrintStream infoStream;
|
||||
public final Directory directory;
|
||||
public final String segmentName;
|
||||
public final FieldInfos fieldInfos;
|
||||
public final AtomicLong bytesUsed;
|
||||
public final SegmentCodecs segmentCodecs;
|
||||
public final int codecId;
|
||||
|
||||
PerDocWriteState(PrintStream infoStream, Directory directory,
|
||||
String segmentName, FieldInfos fieldInfos, AtomicLong bytesUsed,
|
||||
int codecId) {
|
||||
this.infoStream = infoStream;
|
||||
this.directory = directory;
|
||||
this.segmentName = segmentName;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.segmentCodecs = fieldInfos.buildSegmentCodecs(false);
|
||||
this.codecId = codecId;
|
||||
this.bytesUsed = bytesUsed;
|
||||
}
|
||||
|
||||
PerDocWriteState(SegmentWriteState state) {
|
||||
infoStream = state.infoStream;
|
||||
directory = state.directory;
|
||||
segmentCodecs = state.segmentCodecs;
|
||||
segmentName = state.segmentName;
|
||||
fieldInfos = state.fieldInfos;
|
||||
codecId = state.codecId;
|
||||
bytesUsed = new AtomicLong(0);
|
||||
}
|
||||
|
||||
PerDocWriteState(PerDocWriteState state, int codecId) {
|
||||
this.infoStream = state.infoStream;
|
||||
this.directory = state.directory;
|
||||
this.segmentName = state.segmentName;
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
this.segmentCodecs = state.segmentCodecs;
|
||||
this.codecId = codecId;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
}
|
||||
}
|
|
@ -19,16 +19,22 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -64,7 +70,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
for (int i = 0; i < codecs.length; i++) {
|
||||
boolean success = false;
|
||||
try {
|
||||
consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, "" + i)));
|
||||
consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, i)));
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -99,13 +105,13 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.isIndexed) { // TODO this does not work for non-indexed fields
|
||||
if (fi.isIndexed) {
|
||||
fields.add(fi.name);
|
||||
assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
|
||||
if (!producers.containsKey(codec)) {
|
||||
producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
|
||||
si, fieldInfos, readBufferSize, indexDivisor, ""+fi.getCodecId())));
|
||||
si, fieldInfos, readBufferSize, indexDivisor, fi.getCodecId())));
|
||||
}
|
||||
codecs.put(fi.name, producers.get(codec));
|
||||
}
|
||||
|
@ -121,6 +127,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private final class FieldsIterator extends FieldsEnum {
|
||||
private final Iterator<String> it;
|
||||
private String current;
|
||||
|
@ -184,7 +191,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, String codecId, Set<String> files)
|
||||
public void files(Directory dir, SegmentInfo info, int codecId, Set<String> files)
|
||||
throws IOException {
|
||||
// ignore codecid since segmentCodec will assign it per codec
|
||||
segmentCodecs.files(dir, info, files);
|
||||
|
@ -196,4 +203,91 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
codec.getExtensions(extensions);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new PerDocConsumers(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new PerDocProducers(state.dir, state.fieldInfos, state.segmentInfo,
|
||||
state.readBufferSize, state.termsIndexDivisor);
|
||||
}
|
||||
|
||||
private final class PerDocProducers extends PerDocValues {
|
||||
private final TreeMap<String, PerDocValues> codecs = new TreeMap<String, PerDocValues>();
|
||||
|
||||
public PerDocProducers(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
|
||||
int readBufferSize, int indexDivisor) throws IOException {
|
||||
final Map<Codec, PerDocValues> producers = new HashMap<Codec, PerDocValues>();
|
||||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
if (fi.hasDocValues()) {
|
||||
assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
Codec codec = segmentCodecs.codecs[fi.getCodecId()];
|
||||
if (!producers.containsKey(codec)) {
|
||||
producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
|
||||
si, fieldInfos, readBufferSize, indexDivisor, fi.getCodecId())));
|
||||
}
|
||||
codecs.put(fi.name, producers.get(codec));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, producers.values());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return codecs.keySet();
|
||||
}
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
final PerDocValues perDocProducer = codecs.get(field);
|
||||
if (perDocProducer == null) {
|
||||
return null;
|
||||
}
|
||||
return perDocProducer.docValues(field);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.closeSafely(false, codecs.values());
|
||||
}
|
||||
}
|
||||
|
||||
private final class PerDocConsumers extends PerDocConsumer {
|
||||
private final PerDocConsumer[] consumers;
|
||||
private final Codec[] codecs;
|
||||
private final PerDocWriteState state;
|
||||
|
||||
public PerDocConsumers(PerDocWriteState state) throws IOException {
|
||||
assert segmentCodecs == state.segmentCodecs;
|
||||
this.state = state;
|
||||
codecs = segmentCodecs.codecs;
|
||||
consumers = new PerDocConsumer[codecs.length];
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.closeSafely(false, consumers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
|
||||
final int codecId = field.getCodecId();
|
||||
assert codecId != FieldInfo.UNASSIGNED_CODEC_ID;
|
||||
PerDocConsumer perDoc = consumers[codecId];
|
||||
if (perDoc == null) {
|
||||
perDoc = codecs[codecId].docsConsumer(new PerDocWriteState(state, codecId));
|
||||
assert perDoc != null;
|
||||
consumers[codecId] = perDoc;
|
||||
}
|
||||
return perDoc.addValuesField(field);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -102,7 +102,7 @@ final class SegmentCodecs implements Cloneable {
|
|||
throws IOException {
|
||||
final Codec[] codecArray = codecs;
|
||||
for (int i = 0; i < codecArray.length; i++) {
|
||||
codecArray[i].files(dir, info, ""+i, files);
|
||||
codecArray[i].files(dir, info, i, files);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -148,13 +148,6 @@ final class SegmentCodecs implements Cloneable {
|
|||
return this;
|
||||
}
|
||||
|
||||
SegmentCodecsBuilder addAll(FieldInfos infos) {
|
||||
for (FieldInfo fieldInfo : infos) {
|
||||
tryAddAndSet(fieldInfo);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
SegmentCodecs build() {
|
||||
return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY));
|
||||
}
|
||||
|
|
|
@ -20,7 +20,9 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** Holds core readers that are shared (unchanged) when
|
||||
|
@ -39,6 +41,7 @@ final class SegmentCoreReaders {
|
|||
final FieldInfos fieldInfos;
|
||||
|
||||
final FieldsProducer fields;
|
||||
final PerDocValues perDocProducer;
|
||||
|
||||
final Directory dir;
|
||||
final Directory cfsDir;
|
||||
|
@ -52,6 +55,8 @@ final class SegmentCoreReaders {
|
|||
CompoundFileReader cfsReader;
|
||||
CompoundFileReader storeCFSReader;
|
||||
|
||||
|
||||
|
||||
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor) throws IOException {
|
||||
|
||||
if (termsIndexDivisor == 0) {
|
||||
|
@ -76,11 +81,12 @@ final class SegmentCoreReaders {
|
|||
fieldInfos = si.getFieldInfos();
|
||||
|
||||
this.termsIndexDivisor = termsIndexDivisor;
|
||||
|
||||
final Codec codec = segmentCodecs.codec();
|
||||
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor);
|
||||
// Ask codec for its Fields
|
||||
fields = segmentCodecs.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
|
||||
fields = codec.fieldsProducer(segmentReadState);
|
||||
assert fields != null;
|
||||
|
||||
perDocProducer = codec.docsProducer(segmentReadState);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -119,6 +125,10 @@ final class SegmentCoreReaders {
|
|||
fields.close();
|
||||
}
|
||||
|
||||
if (perDocProducer != null) {
|
||||
perDocProducer.close();
|
||||
}
|
||||
|
||||
if (termVectorsReaderOrig != null) {
|
||||
termVectorsReaderOrig.close();
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
|
|
@ -29,6 +29,8 @@ import org.apache.lucene.index.MergePolicy.MergeAbortedException;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.MergeState;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -116,7 +118,6 @@ final class SegmentMerger {
|
|||
|
||||
if (fieldInfos.hasVectors())
|
||||
mergeVectors();
|
||||
|
||||
return mergedDocs;
|
||||
}
|
||||
|
||||
|
@ -154,7 +155,7 @@ final class SegmentMerger {
|
|||
for (String field : names) {
|
||||
fInfos.addOrUpdate(field, true, storeTermVectors,
|
||||
storePositionWithTermVector, storeOffsetWithTermVector, !reader
|
||||
.hasNorms(field), storePayloads, omitTFAndPositions);
|
||||
.hasNorms(field), storePayloads, omitTFAndPositions, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -222,6 +223,7 @@ final class SegmentMerger {
|
|||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, false);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, false);
|
||||
fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
|
||||
fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
|
||||
}
|
||||
}
|
||||
final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
|
||||
|
@ -477,10 +479,17 @@ final class SegmentMerger {
|
|||
int docBase = 0;
|
||||
|
||||
final List<Fields> fields = new ArrayList<Fields>();
|
||||
|
||||
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
|
||||
final List<Bits> bits = new ArrayList<Bits>();
|
||||
final List<Integer> bitsStarts = new ArrayList<Integer>();
|
||||
|
||||
// TODO: move this into its own method - this merges currently only docvalues
|
||||
final List<PerDocValues> perDocProducers = new ArrayList<PerDocValues>();
|
||||
final List<ReaderUtil.Slice> perDocSlices = new ArrayList<ReaderUtil.Slice>();
|
||||
final List<Bits> perDocBits = new ArrayList<Bits>();
|
||||
final List<Integer> perDocBitsStarts = new ArrayList<Integer>();
|
||||
|
||||
for(IndexReader r : readers) {
|
||||
final Fields f = r.fields();
|
||||
final int maxDoc = r.maxDoc();
|
||||
|
@ -490,10 +499,18 @@ final class SegmentMerger {
|
|||
bits.add(r.getDeletedDocs());
|
||||
bitsStarts.add(docBase);
|
||||
}
|
||||
final PerDocValues producer = r.perDocValues();
|
||||
if (producer != null) {
|
||||
perDocSlices.add(new ReaderUtil.Slice(docBase, maxDoc, fields.size()));
|
||||
perDocProducers.add(producer);
|
||||
perDocBits.add(r.getDeletedDocs());
|
||||
perDocBitsStarts.add(docBase);
|
||||
}
|
||||
docBase += maxDoc;
|
||||
}
|
||||
|
||||
bitsStarts.add(docBase);
|
||||
perDocBitsStarts.add(docBase);
|
||||
|
||||
// we may gather more readers than mergeState.readerCount
|
||||
mergeState = new MergeState();
|
||||
|
@ -559,6 +576,20 @@ final class SegmentMerger {
|
|||
} finally {
|
||||
consumer.close();
|
||||
}
|
||||
if (!perDocSlices.isEmpty()) {
|
||||
mergeState.multiDeletedDocs = new MultiBits(perDocBits, perDocBitsStarts);
|
||||
final PerDocConsumer docsConsumer = codec
|
||||
.docsConsumer(new PerDocWriteState(segmentWriteState));
|
||||
try {
|
||||
final MultiPerDocValues multiPerDocValues = new MultiPerDocValues(perDocProducers
|
||||
.toArray(PerDocValues.EMPTY_ARRAY), perDocSlices
|
||||
.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
|
||||
docsConsumer.merge(mergeState, multiPerDocValues);
|
||||
} finally {
|
||||
docsConsumer.close();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private MergeState mergeState;
|
||||
|
|
|
@ -34,11 +34,11 @@ public class SegmentReadState {
|
|||
// that must do so), then it should negate this value to
|
||||
// get the app's terms divisor:
|
||||
public int termsIndexDivisor;
|
||||
public final String codecId;
|
||||
public final int codecId;
|
||||
|
||||
public SegmentReadState(Directory dir, SegmentInfo info,
|
||||
FieldInfos fieldInfos, int readBufferSize, int termsIndexDivisor) {
|
||||
this(dir, info, fieldInfos, readBufferSize, termsIndexDivisor, "");
|
||||
this(dir, info, fieldInfos, readBufferSize, termsIndexDivisor, -1);
|
||||
}
|
||||
|
||||
public SegmentReadState(Directory dir,
|
||||
|
@ -46,7 +46,7 @@ public class SegmentReadState {
|
|||
FieldInfos fieldInfos,
|
||||
int readBufferSize,
|
||||
int termsIndexDivisor,
|
||||
String codecId) {
|
||||
int codecId) {
|
||||
this.dir = dir;
|
||||
this.segmentInfo = info;
|
||||
this.fieldInfos = fieldInfos;
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.BufferedIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -839,4 +841,9 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
// longer used (all SegmentReaders sharing it have been
|
||||
// closed).
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return core.perDocProducer;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ public class SegmentWriteState {
|
|||
public BitVector deletedDocs;
|
||||
|
||||
final SegmentCodecs segmentCodecs;
|
||||
public final String codecId;
|
||||
public final int codecId;
|
||||
|
||||
/** Expert: The fraction of terms in the "dictionary" which should be stored
|
||||
* in RAM. Smaller values use more memory, but make searching slightly
|
||||
|
@ -62,13 +62,13 @@ public class SegmentWriteState {
|
|||
this.numDocs = numDocs;
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
this.segmentCodecs = segmentCodecs;
|
||||
codecId = "";
|
||||
codecId = -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a shallow {@link SegmentWriteState} copy final a codec ID
|
||||
*/
|
||||
SegmentWriteState(SegmentWriteState state, String codecId) {
|
||||
SegmentWriteState(SegmentWriteState state, int codecId) {
|
||||
infoStream = state.infoStream;
|
||||
directory = state.directory;
|
||||
segmentName = state.segmentName;
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.util.ReaderUtil; // javadoc
|
|||
|
||||
import org.apache.lucene.index.DirectoryReader; // javadoc
|
||||
import org.apache.lucene.index.MultiReader; // javadoc
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
|
||||
/**
|
||||
* This class forces a composite reader (eg a {@link
|
||||
|
@ -64,6 +65,11 @@ public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
|||
return MultiFields.getFields(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return MultiPerDocValues.getPerDocs(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDeletedDocs() {
|
||||
return MultiFields.getDeletedDocs(in);
|
||||
|
|
|
@ -108,7 +108,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
//private String segment;
|
||||
|
||||
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
|
||||
int termsCacheSize, String codecId)
|
||||
int termsCacheSize, int codecId)
|
||||
throws IOException {
|
||||
|
||||
this.postingsReader = postingsReader;
|
||||
|
@ -196,7 +196,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, BlockTermsWriter.TERMS_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -51,6 +52,10 @@ public abstract class Codec {
|
|||
* use; else, those files may be deleted. */
|
||||
public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
|
||||
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
|
||||
public abstract PerDocValues docsProducer(SegmentReadState state) throws IOException;
|
||||
|
||||
/**
|
||||
* Gathers files associated with this segment
|
||||
*
|
||||
|
@ -59,7 +64,7 @@ public abstract class Codec {
|
|||
* @param id the codec id within this segment
|
||||
* @param files the of files to add the codec files to.
|
||||
*/
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException;
|
||||
|
||||
/** Records all file extensions this codec uses */
|
||||
public abstract void getExtensions(Set<String> extensions);
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
/** Holds a set of codecs, keyed by name. You subclass
|
||||
* this, instantiate it, and register your codecs, then
|
||||
|
|
|
@ -0,0 +1,104 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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 java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
public class DefaultDocValuesConsumer extends PerDocConsumer {
|
||||
private final String segmentName;
|
||||
private final int codecId;
|
||||
private final Directory directory;
|
||||
private final AtomicLong bytesUsed;
|
||||
private final Comparator<BytesRef> comparator;
|
||||
|
||||
public DefaultDocValuesConsumer(PerDocWriteState state, Comparator<BytesRef> comparator) {
|
||||
this.segmentName = state.segmentName;
|
||||
this.codecId = state.codecId;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
this.directory = state.directory;
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
|
||||
return Writer.create(field.getDocValues(),
|
||||
docValuesId(segmentName, codecId, field.number),
|
||||
// TODO can we have a compound file per segment and codec for
|
||||
// docvalues?
|
||||
directory, comparator, bytesUsed);
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
|
||||
Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
|
||||
String filename = docValuesId(segmentInfo.name, codecId,
|
||||
fieldInfo.number);
|
||||
switch (fieldInfo.getDocValues()) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
files.add(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.INDEX_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.INDEX_EXTENSION));
|
||||
// until here all types use an index
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
case INTS:
|
||||
files.add(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
break;
|
||||
default:
|
||||
assert false;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static String docValuesId(String segmentsName, int codecID, int fieldId) {
|
||||
return segmentsName + "_" + codecID + "-" + fieldId;
|
||||
}
|
||||
|
||||
public static void getDocValuesExtensions(Set<String> extensions) {
|
||||
extensions.add(Writer.DATA_EXTENSION);
|
||||
extensions.add(Writer.INDEX_EXTENSION);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,170 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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.Collection;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.values.Bytes;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.Floats;
|
||||
import org.apache.lucene.index.values.Ints;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Abstract base class for FieldsProducer implementations supporting
|
||||
* {@link IndexDocValues}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class DefaultDocValuesProducer extends PerDocValues {
|
||||
|
||||
protected final TreeMap<String, IndexDocValues> docValues;
|
||||
|
||||
/**
|
||||
* Creates a new {@link DefaultDocValuesProducer} instance and loads all
|
||||
* {@link IndexDocValues} instances for this segment and codec.
|
||||
*
|
||||
* @param si
|
||||
* the segment info to load the {@link IndexDocValues} for.
|
||||
* @param dir
|
||||
* the directory to load the {@link IndexDocValues} from.
|
||||
* @param fieldInfo
|
||||
* the {@link FieldInfos}
|
||||
* @param codecId
|
||||
* the codec ID
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public DefaultDocValuesProducer(SegmentInfo si, Directory dir,
|
||||
FieldInfos fieldInfo, int codecId) throws IOException {
|
||||
docValues = load(fieldInfo, si.name, si.docCount, dir, codecId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link IndexDocValues} instance for the given field name or
|
||||
* <code>null</code> if this field has no {@link IndexDocValues}.
|
||||
*/
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
return docValues.get(field);
|
||||
}
|
||||
|
||||
// Only opens files... doesn't actually load any values
|
||||
protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
|
||||
String segment, int docCount, Directory dir, int codecId)
|
||||
throws IOException {
|
||||
TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) {
|
||||
final String field = fieldInfo.name;
|
||||
// TODO can we have a compound file per segment and codec for
|
||||
// docvalues?
|
||||
final String id = DefaultDocValuesConsumer.docValuesId(segment,
|
||||
codecId, fieldInfo.number);
|
||||
values.put(field,
|
||||
loadDocValues(docCount, dir, id, fieldInfo.getDocValues()));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// if we fail we must close all opened resources if there are any
|
||||
closeDocValues(values.values());
|
||||
}
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Loads a {@link IndexDocValues} instance depending on the given {@link ValueType}.
|
||||
* Codecs that use different implementations for a certain {@link ValueType} can
|
||||
* simply override this method and return their custom implementations.
|
||||
*
|
||||
* @param docCount
|
||||
* number of documents in the segment
|
||||
* @param dir
|
||||
* the {@link Directory} to load the {@link IndexDocValues} from
|
||||
* @param id
|
||||
* the unique file ID within the segment
|
||||
* @param type
|
||||
* the type to load
|
||||
* @return a {@link IndexDocValues} instance for the given type
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
* @throws IllegalArgumentException
|
||||
* if the given {@link ValueType} is not supported
|
||||
*/
|
||||
protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
ValueType type) throws IOException {
|
||||
switch (type) {
|
||||
case INTS:
|
||||
return Ints.getValues(dir, id, false);
|
||||
case FLOAT_32:
|
||||
return Floats.getValues(dir, id, docCount);
|
||||
case FLOAT_64:
|
||||
return Floats.getValues(dir, id, docCount);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount);
|
||||
case BYTES_FIXED_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount);
|
||||
case BYTES_VAR_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount);
|
||||
case BYTES_VAR_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount);
|
||||
default:
|
||||
throw new IllegalStateException("unrecognized index values mode " + type);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
closeDocValues(docValues.values());
|
||||
}
|
||||
|
||||
private void closeDocValues(final Collection<IndexDocValues> values)
|
||||
throws IOException {
|
||||
IOException ex = null;
|
||||
for (IndexDocValues docValues : values) {
|
||||
try {
|
||||
docValues.close();
|
||||
} catch (IOException e) {
|
||||
ex = e;
|
||||
}
|
||||
}
|
||||
if (ex != null) {
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return docValues.keySet();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,166 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* 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.Collection;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes {@link PerDocFieldValues}.
|
||||
* {@link DocValuesConsumer} are always associated with a specific field and
|
||||
* segments. Concrete implementations of this API write the given
|
||||
* {@link PerDocFieldValues} into a implementation specific format depending on
|
||||
* the fields meta-data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValuesConsumer {
|
||||
|
||||
protected final AtomicLong bytesUsed;
|
||||
|
||||
/**
|
||||
* Creates a new {@link DocValuesConsumer}.
|
||||
*
|
||||
* @param bytesUsed
|
||||
* bytes-usage tracking reference used by implementation to track
|
||||
* internally allocated memory. All tracked bytes must be released
|
||||
* once {@link #finish(int)} has been called.
|
||||
*/
|
||||
protected DocValuesConsumer(AtomicLong bytesUsed) {
|
||||
this.bytesUsed = bytesUsed == null ? new AtomicLong(0) : bytesUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the given {@link PerDocFieldValues} instance to this
|
||||
* {@link DocValuesConsumer}
|
||||
*
|
||||
* @param docID
|
||||
* the document ID to add the value for. The docID must always
|
||||
* increase or be <tt>0</tt> if it is the first call to this method.
|
||||
* @param docValues
|
||||
* the values to add
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract void add(int docID, PerDocFieldValues docValues)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Called when the consumer of this API is doc with adding
|
||||
* {@link PerDocFieldValues} to this {@link DocValuesConsumer}
|
||||
*
|
||||
* @param docCount
|
||||
* the total number of documents in this {@link DocValuesConsumer}.
|
||||
* Must be greater than or equal the last given docID to
|
||||
* {@link #add(int, PerDocFieldValues)}.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
|
||||
/**
|
||||
* Gathers files associated with this {@link DocValuesConsumer}
|
||||
*
|
||||
* @param files
|
||||
* the of files to add the consumers files to.
|
||||
*/
|
||||
public abstract void files(Collection<String> files) throws IOException;
|
||||
|
||||
/**
|
||||
* Merges the given {@link org.apache.lucene.index.codecs.MergeState} into
|
||||
* this {@link DocValuesConsumer}.
|
||||
*
|
||||
* @param mergeState
|
||||
* the state to merge
|
||||
* @param values
|
||||
* the docValues to merge in
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public void merge(org.apache.lucene.index.codecs.MergeState mergeState,
|
||||
IndexDocValues values) throws IOException {
|
||||
assert mergeState != null;
|
||||
// TODO we need some kind of compatibility notation for values such
|
||||
// that two slightly different segments can be merged eg. fixed vs.
|
||||
// variable byte len or float32 vs. float64
|
||||
int docBase = 0;
|
||||
boolean merged = false;
|
||||
/*
|
||||
* We ignore the given DocValues here and merge from the subReaders directly
|
||||
* to support bulk copies on the DocValues Writer level. if this gets merged
|
||||
* with MultiDocValues the writer can not optimize for bulk-copyable data
|
||||
*/
|
||||
for (final IndexReader reader : mergeState.readers) {
|
||||
final IndexDocValues r = reader.docValues(mergeState.fieldInfo.name);
|
||||
if (r != null) {
|
||||
merged = true;
|
||||
merge(new Writer.MergeState(r, docBase, reader.maxDoc(), reader
|
||||
.getDeletedDocs()));
|
||||
}
|
||||
docBase += reader.numDocs();
|
||||
}
|
||||
if (merged) {
|
||||
finish(mergeState.mergedDocCount);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the given {@link MergeState} into this {@link DocValuesConsumer}.
|
||||
* {@link MergeState#docBase} must always be increasing. Merging segments out
|
||||
* of order is not supported.
|
||||
*
|
||||
* @param mergeState
|
||||
* the {@link MergeState} to merge
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
protected abstract void merge(MergeState mergeState) throws IOException;
|
||||
|
||||
/**
|
||||
* Specialized auxiliary MergeState is necessary since we don't want to
|
||||
* exploit internals up to the codecs consumer. An instance of this class is
|
||||
* created for each merged low level {@link IndexReader} we are merging to
|
||||
* support low level bulk copies.
|
||||
*/
|
||||
public static class MergeState {
|
||||
/**
|
||||
* the source reader for this MergeState - merged values should be read from
|
||||
* this instance
|
||||
*/
|
||||
public final IndexDocValues reader;
|
||||
/** the absolute docBase for this MergeState within the resulting segment */
|
||||
public final int docBase;
|
||||
/** the number of documents in this MergeState */
|
||||
public final int docCount;
|
||||
/** the deleted bits for this MergeState */
|
||||
public final Bits bits;
|
||||
|
||||
public MergeState(IndexDocValues reader, int docBase, int docCount, Bits bits) {
|
||||
assert reader != null;
|
||||
this.reader = reader;
|
||||
this.docBase = docBase;
|
||||
this.docCount = docCount;
|
||||
this.bits = bits;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Closeable;
|
||||
|
@ -45,8 +46,12 @@ public abstract class FieldsConsumer implements Closeable {
|
|||
String field;
|
||||
while((field = fieldsEnum.next()) != null) {
|
||||
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
|
||||
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field;
|
||||
TermsEnum terms = fieldsEnum.terms();
|
||||
if (terms != null) {
|
||||
final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
|
||||
termsConsumer.merge(mergeState, fieldsEnum.terms());
|
||||
termsConsumer.merge(mergeState, terms);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,10 +17,12 @@ package org.apache.lucene.index.codecs;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.Fields;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
||||
/** Abstract API that consumes terms, doc, freq, prox and
|
||||
* payloads postings. Concrete implementations of this
|
||||
|
@ -33,4 +35,28 @@ import java.io.Closeable;
|
|||
public abstract class FieldsProducer extends Fields implements Closeable {
|
||||
public abstract void close() throws IOException;
|
||||
public abstract void loadTermsIndex(int indexDivisor) throws IOException;
|
||||
|
||||
public static final FieldsProducer EMPTY = new FieldsProducer() {
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsEnum iterator() throws IOException {
|
||||
return FieldsEnum.EMPTY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void loadTermsIndex(int indexDivisor) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -68,7 +68,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
// start of the field info data
|
||||
protected long dirOffset;
|
||||
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String codecId)
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
|
||||
throws IOException {
|
||||
|
||||
this.termComp = termComp;
|
||||
|
@ -406,7 +406,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
/**
|
||||
* 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.values.IndexDocValues;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes per document values. Concrete implementations of
|
||||
* this convert field values into a Codec specific format during indexing.
|
||||
* <p>
|
||||
* The {@link PerDocConsumer} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class PerDocConsumer implements Closeable{
|
||||
/** Adds a new DocValuesField */
|
||||
public abstract DocValuesConsumer addValuesField(FieldInfo field)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Consumes and merges the given {@link PerDocValues} producer
|
||||
* into this consumers format.
|
||||
*/
|
||||
public void merge(MergeState mergeState, PerDocValues producer)
|
||||
throws IOException {
|
||||
Iterable<String> fields = producer.fields();
|
||||
for (String field : fields) {
|
||||
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
|
||||
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "
|
||||
+ field;
|
||||
if (mergeState.fieldInfo.hasDocValues()) {
|
||||
final IndexDocValues docValues = producer.docValues(field);
|
||||
if (docValues == null) {
|
||||
/*
|
||||
* It is actually possible that a fieldInfo has a values type but no
|
||||
* values are actually available. this can happen if there are already
|
||||
* segments without values around.
|
||||
*/
|
||||
continue;
|
||||
}
|
||||
final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);
|
||||
assert docValuesConsumer != null;
|
||||
docValuesConsumer.merge(mergeState, docValues);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
/**
|
||||
* 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 java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
|
||||
/**
|
||||
* Abstract API that provides access to one or more per-document storage
|
||||
* features. The concrete implementations provide access to the underlying
|
||||
* storage on a per-document basis corresponding to their actual
|
||||
* {@link PerDocConsumer} counterpart.
|
||||
* <p>
|
||||
* The {@link PerDocValues} API is accessible through the
|
||||
* {@link Codec} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class PerDocValues implements Closeable {
|
||||
/**
|
||||
* Returns {@link IndexDocValues} for the current field.
|
||||
*
|
||||
* @param field
|
||||
* the field name
|
||||
* @return the {@link IndexDocValues} for this field or <code>null</code> if not
|
||||
* applicable.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract IndexDocValues docValues(String field) throws IOException;
|
||||
|
||||
public static final PerDocValues[] EMPTY_ARRAY = new PerDocValues[0];
|
||||
|
||||
/**
|
||||
* Returns all fields this {@link PerDocValues} contains values for.
|
||||
*/
|
||||
public abstract Collection<String> fields();
|
||||
|
||||
}
|
|
@ -57,8 +57,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
protected long dirOffset;
|
||||
|
||||
final String segment;
|
||||
|
||||
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String codecId)
|
||||
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId)
|
||||
throws IOException {
|
||||
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
|
@ -159,15 +158,11 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
|
||||
private final class FieldIndexData {
|
||||
|
||||
private final FieldInfo fieldInfo;
|
||||
private final long indexStart;
|
||||
|
||||
// Set only if terms index is loaded:
|
||||
private volatile FST<Long> fst;
|
||||
|
||||
public FieldIndexData(FieldInfo fieldInfo, long indexStart) throws IOException {
|
||||
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.indexStart = indexStart;
|
||||
|
||||
if (indexDivisor > 0) {
|
||||
|
@ -241,7 +236,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, String id, Collection<String> files) {
|
||||
public static void files(Directory dir, SegmentInfo info, int id, Collection<String> files) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, id, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
|
|
|
@ -22,11 +22,14 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
|
||||
/** Codec that reads the pre-flex-indexing postings
|
||||
* format. It does not provide a writer because newly
|
||||
|
@ -66,7 +69,7 @@ public class PreFlexCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, String id, Set<String> files) throws IOException {
|
||||
public void files(Directory dir, SegmentInfo info, int id, Set<String> files) throws IOException {
|
||||
// preflex fields have no codec ID - we ignore it here
|
||||
PreFlexFields.files(dir, info, files);
|
||||
}
|
||||
|
@ -78,4 +81,14 @@ public class PreFlexCodec extends Codec {
|
|||
extensions.add(TERMS_EXTENSION);
|
||||
extensions.add(TERMS_INDEX_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
throw new UnsupportedOperationException("PerDocConsumer is not supported by Preflex codec");
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
throw new UnsupportedOperationException("PerDocValues is not supported by Preflex codec");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -153,8 +153,12 @@ public final class SegmentTermEnum implements Cloneable {
|
|||
return true;
|
||||
}
|
||||
|
||||
/** Optimized scan, without allocating new terms.
|
||||
* Return number of invocations to next(). */
|
||||
/* 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;
|
||||
|
|
|
@ -57,6 +57,7 @@ public final class TermInfosReader {
|
|||
final long termOrd;
|
||||
public TermInfoAndOrd(TermInfo ti, long termOrd) {
|
||||
super(ti);
|
||||
assert termOrd >= 0;
|
||||
this.termOrd = termOrd;
|
||||
}
|
||||
}
|
||||
|
@ -306,8 +307,14 @@ public final class TermInfosReader {
|
|||
ti = enumerator.termInfo;
|
||||
if (tiOrd == null) {
|
||||
if (useCache) {
|
||||
// LUCENE-3183: it's possible, if term is Term("",
|
||||
// ""), for the STE to be incorrectly un-positioned
|
||||
// after scan-to; work around this by not caching in
|
||||
// this case:
|
||||
if (enumerator.position >= 0) {
|
||||
termsCache.put(new CloneableTerm(term), new TermInfoAndOrd(ti, enumerator.position));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
assert sameTermInfo(ti, tiOrd, enumerator);
|
||||
assert enumerator.position == tiOrd.termOrd;
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.pulsing;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -28,8 +29,12 @@ import org.apache.lucene.index.codecs.PostingsWriterBase;
|
|||
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
|
||||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
|
@ -38,6 +43,7 @@ import org.apache.lucene.index.codecs.TermsIndexReaderBase;
|
|||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** This codec "inlines" the postings for terms that have
|
||||
|
@ -147,14 +153,26 @@ public class PulsingCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, id, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
StandardCodec.getStandardExtensions(extensions);
|
||||
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,8 +58,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
|
|||
int maxSkipLevels;
|
||||
int skipMinimum;
|
||||
|
||||
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, String codecId) throws IOException {
|
||||
|
||||
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, int codecId) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
|
@ -85,7 +84,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo segmentInfo, String codecId, Collection<String> files) throws IOException {
|
||||
public static void files(SegmentInfo segmentInfo, int codecId, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION));
|
||||
|
||||
|
|
|
@ -20,14 +20,20 @@ package org.apache.lucene.index.codecs.simpletext;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** For debugging, curiosity, transparency only!! Do not
|
||||
* use this codec in production.
|
||||
|
@ -56,17 +62,30 @@ public class SimpleTextCodec extends Codec {
|
|||
/** Extension of freq postings file */
|
||||
static final String POSTINGS_EXTENSION = "pst";
|
||||
|
||||
static String getPostingsFileName(String segment, String id) {
|
||||
static String getPostingsFileName(String segment, int id) {
|
||||
return IndexFileNames.segmentFileName(segment, id, POSTINGS_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
files.add(getPostingsFileName(segmentInfo.name, id));
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
extensions.add(POSTINGS_EXTENSION);
|
||||
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
|
||||
}
|
||||
|
||||
// TODO: would be great if these used a plain text impl
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,7 +58,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
|
||||
|
||||
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
|
||||
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, ""+state.codecId));
|
||||
in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId));
|
||||
|
||||
fieldInfos = state.fieldInfos;
|
||||
}
|
||||
|
|
|
@ -20,12 +20,16 @@ package org.apache.lucene.index.codecs.standard;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
|
||||
|
@ -34,7 +38,9 @@ import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
|
|||
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
|
||||
import org.apache.lucene.index.codecs.BlockTermsWriter;
|
||||
import org.apache.lucene.index.codecs.BlockTermsReader;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Default codec.
|
||||
* @lucene.experimental */
|
||||
|
@ -130,15 +136,17 @@ public class StandardCodec extends Codec {
|
|||
static final String PROX_EXTENSION = "prx";
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
StandardPostingsReader.files(dir, segmentInfo, id, files);
|
||||
BlockTermsReader.files(dir, segmentInfo, id, files);
|
||||
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
getStandardExtensions(extensions);
|
||||
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
|
||||
}
|
||||
|
||||
public static void getStandardExtensions(Set<String> extensions) {
|
||||
|
@ -147,4 +155,14 @@ public class StandardCodec extends Codec {
|
|||
BlockTermsReader.getExtensions(extensions);
|
||||
VariableGapTermsIndexReader.getIndexExtensions(extensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
|
||||
//private String segment;
|
||||
|
||||
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, String codecId) throws IOException {
|
||||
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, int codecId) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
|
||||
readBufferSize);
|
||||
//this.segment = segmentInfo.name;
|
||||
|
@ -71,7 +71,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) throws IOException {
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
|
||||
if (segmentInfo.getHasProx()) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
|
||||
|
|
|
@ -0,0 +1,491 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/** Base class for specific Bytes Reader/Writer implementations */
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
|
||||
/**
|
||||
* Provides concrete Writer/Reader implementations for <tt>byte[]</tt> value per
|
||||
* document. There are 6 package-private default implementations of this, for
|
||||
* all combinations of {@link Mode#DEREF}/{@link Mode#STRAIGHT}/
|
||||
* {@link Mode#SORTED} x fixed-length/variable-length.
|
||||
*
|
||||
* <p>
|
||||
* NOTE: Currently the total amount of byte[] data stored (across a single
|
||||
* segment) cannot exceed 2GB.
|
||||
* </p>
|
||||
* <p>
|
||||
* NOTE: Each byte[] must be <= 32768 bytes in length
|
||||
* </p>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Bytes {
|
||||
// TODO - add bulk copy where possible
|
||||
private Bytes() { /* don't instantiate! */
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines the {@link Writer}s store mode. The writer will either store the
|
||||
* bytes sequentially ({@link #STRAIGHT}, dereferenced ({@link #DEREF}) or
|
||||
* sorted ({@link #SORTED})
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static enum Mode {
|
||||
/**
|
||||
* Mode for sequentially stored bytes
|
||||
*/
|
||||
STRAIGHT,
|
||||
/**
|
||||
* Mode for dereferenced stored bytes
|
||||
*/
|
||||
DEREF,
|
||||
/**
|
||||
* Mode for sorted stored bytes
|
||||
*/
|
||||
SORTED
|
||||
};
|
||||
|
||||
/**
|
||||
* Creates a new <tt>byte[]</tt> {@link Writer} instances for the given
|
||||
* directory.
|
||||
*
|
||||
* @param dir
|
||||
* the directory to write the values to
|
||||
* @param id
|
||||
* the id used to create a unique file name. Usually composed out of
|
||||
* the segment name and a unique id per segment.
|
||||
* @param mode
|
||||
* the writers store mode
|
||||
* @param comp
|
||||
* a {@link BytesRef} comparator - only used with {@link Mode#SORTED}
|
||||
* @param fixedSize
|
||||
* <code>true</code> if all bytes subsequently passed to the
|
||||
* {@link Writer} will have the same length
|
||||
* @param bytesUsed
|
||||
* an {@link AtomicLong} instance to track the used bytes within the
|
||||
* {@link Writer}. A call to {@link Writer#finish(int)} will release
|
||||
* all internally used resources and frees the memeory tracking
|
||||
* reference.
|
||||
* @return a new {@link Writer} instance
|
||||
* @throws IOException
|
||||
* if the files for the writer can not be created.
|
||||
*/
|
||||
public static Writer getWriter(Directory dir, String id, Mode mode,
|
||||
Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
// TODO -- i shouldn't have to specify fixed? can
|
||||
// track itself & do the write thing at write time?
|
||||
if (comp == null) {
|
||||
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
if (fixedSize) {
|
||||
if (mode == Mode.STRAIGHT) {
|
||||
return new FixedStraightBytesImpl.Writer(dir, id);
|
||||
} else if (mode == Mode.DEREF) {
|
||||
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
|
||||
}
|
||||
} else {
|
||||
if (mode == Mode.STRAIGHT) {
|
||||
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed);
|
||||
} else if (mode == Mode.DEREF) {
|
||||
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException("");
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link IndexDocValues} instance that provides either memory
|
||||
* resident or iterative access to a per-document stored <tt>byte[]</tt>
|
||||
* value. The returned {@link IndexDocValues} instance will be initialized without
|
||||
* consuming a significant amount of memory.
|
||||
*
|
||||
* @param dir
|
||||
* the directory to load the {@link IndexDocValues} from.
|
||||
* @param id
|
||||
* the file ID in the {@link Directory} to load the values from.
|
||||
* @param mode
|
||||
* the mode used to store the values
|
||||
* @param fixedSize
|
||||
* <code>true</code> iff the values are stored with fixed-size,
|
||||
* otherwise <code>false</code>
|
||||
* @param maxDoc
|
||||
* the number of document values stored for the given ID
|
||||
* @return an initialized {@link IndexDocValues} instance.
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public static IndexDocValues getValues(Directory dir, String id, Mode mode,
|
||||
boolean fixedSize, int maxDoc) throws IOException {
|
||||
// TODO -- I can peek @ header to determing fixed/mode?
|
||||
if (fixedSize) {
|
||||
if (mode == Mode.STRAIGHT) {
|
||||
return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
|
||||
} else if (mode == Mode.DEREF) {
|
||||
return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
|
||||
}
|
||||
} else {
|
||||
if (mode == Mode.STRAIGHT) {
|
||||
return new VarStraightBytesImpl.Reader(dir, id, maxDoc);
|
||||
} else if (mode == Mode.DEREF) {
|
||||
return new VarDerefBytesImpl.Reader(dir, id, maxDoc);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new VarSortedBytesImpl.Reader(dir, id, maxDoc);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException("Illegal Mode: " + mode);
|
||||
}
|
||||
|
||||
// TODO open up this API?
|
||||
static abstract class BytesBaseSource extends Source {
|
||||
private final PagedBytes pagedBytes;
|
||||
protected final IndexInput datIn;
|
||||
protected final IndexInput idxIn;
|
||||
protected final static int PAGED_BYTES_BITS = 15;
|
||||
protected final PagedBytes.Reader data;
|
||||
protected final long totalLengthInBytes;
|
||||
|
||||
protected BytesBaseSource(IndexInput datIn, IndexInput idxIn,
|
||||
PagedBytes pagedBytes, long bytesToRead) throws IOException {
|
||||
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
|
||||
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
|
||||
this.datIn = datIn;
|
||||
this.totalLengthInBytes = bytesToRead;
|
||||
this.pagedBytes = pagedBytes;
|
||||
this.pagedBytes.copy(datIn, bytesToRead);
|
||||
data = pagedBytes.freeze(true);
|
||||
this.idxIn = idxIn;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
data.close(); // close data
|
||||
} finally {
|
||||
try {
|
||||
if (datIn != null) {
|
||||
datIn.close();
|
||||
}
|
||||
} finally {
|
||||
if (idxIn != null) {// if straight - no index needed
|
||||
idxIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns one greater than the largest possible document number.
|
||||
*/
|
||||
protected abstract int maxDoc();
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, maxDoc()) {
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
while (source.getBytes(target, bytesRef).length == 0) {
|
||||
if (++target >= numDocs) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
}
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static abstract class BytesBaseSortedSource extends SortedSource {
|
||||
protected final IndexInput datIn;
|
||||
protected final IndexInput idxIn;
|
||||
protected final BytesRef defaultValue = new BytesRef();
|
||||
protected final static int PAGED_BYTES_BITS = 15;
|
||||
private final PagedBytes pagedBytes;
|
||||
protected final PagedBytes.Reader data;
|
||||
private final Comparator<BytesRef> comp;
|
||||
|
||||
protected BytesBaseSortedSource(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead)
|
||||
throws IOException {
|
||||
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
|
||||
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
|
||||
this.datIn = datIn;
|
||||
this.pagedBytes = pagedBytes;
|
||||
this.pagedBytes.copy(datIn, bytesToRead);
|
||||
data = pagedBytes.freeze(true);
|
||||
this.idxIn = idxIn;
|
||||
this.comp = comp == null ? BytesRef.getUTF8SortedAsUnicodeComparator()
|
||||
: comp;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
|
||||
assert ord >= 0;
|
||||
return deref(ord, bytesRef);
|
||||
}
|
||||
|
||||
protected void closeIndexInput() throws IOException {
|
||||
try {
|
||||
if (datIn != null) {
|
||||
datIn.close();
|
||||
}
|
||||
} finally {
|
||||
if (idxIn != null) {// if straight
|
||||
idxIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the largest doc id + 1 in this doc values source
|
||||
*/
|
||||
protected abstract int maxDoc();
|
||||
|
||||
/**
|
||||
* Copies the value for the given ord to the given {@link BytesRef} and
|
||||
* returns it.
|
||||
*/
|
||||
protected abstract BytesRef deref(int ord, BytesRef bytesRef);
|
||||
|
||||
protected int binarySearch(BytesRef b, BytesRef bytesRef, int low,
|
||||
int high) {
|
||||
int mid = 0;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
deref(mid, bytesRef);
|
||||
final int cmp = comp.compare(bytesRef, b);
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return mid;
|
||||
}
|
||||
}
|
||||
assert comp.compare(bytesRef, b) != 0;
|
||||
return -(low + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, maxDoc()) {
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
while (source.getBytes(target, bytesRef).length == 0) {
|
||||
if (++target >= numDocs) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
}
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: open up this API?!
|
||||
static abstract class BytesWriterBase extends Writer {
|
||||
private final String id;
|
||||
protected IndexOutput idxOut;
|
||||
protected IndexOutput datOut;
|
||||
protected BytesRef bytesRef;
|
||||
protected final ByteBlockPool pool;
|
||||
|
||||
protected BytesWriterBase(Directory dir, String id, String codecName,
|
||||
int version, boolean initIndex, ByteBlockPool pool,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
super(bytesUsed);
|
||||
this.id = id;
|
||||
this.pool = pool;
|
||||
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
|
||||
DATA_EXTENSION));
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeHeader(datOut, codecName, version);
|
||||
if (initIndex) {
|
||||
idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
|
||||
INDEX_EXTENSION));
|
||||
CodecUtil.writeHeader(idxOut, codecName, version);
|
||||
} else {
|
||||
idxOut = null;
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, datOut, idxOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Must be called only with increasing docIDs. It's OK for some docIDs to be
|
||||
* skipped; they will be filled with 0 bytes.
|
||||
*/
|
||||
@Override
|
||||
public abstract void add(int docID, BytesRef bytes) throws IOException;
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
IOUtils.closeSafely(false, datOut, idxOut);
|
||||
} finally {
|
||||
if (pool != null) {
|
||||
pool.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID) throws IOException {
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
final BytesRef ref;
|
||||
if ((ref = docValues.getBytes()) != null) {
|
||||
add(docID, ref);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setNextEnum(ValuesEnum valuesEnum) {
|
||||
bytesRef = valuesEnum.bytes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Collection<String> files) throws IOException {
|
||||
assert datOut != null;
|
||||
files.add(IndexFileNames.segmentFileName(id, "", DATA_EXTENSION));
|
||||
if (idxOut != null) { // called after flush - so this must be initialized
|
||||
// if needed or present
|
||||
final String idxFile = IndexFileNames.segmentFileName(id, "",
|
||||
INDEX_EXTENSION);
|
||||
files.add(idxFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens all necessary files, but does not read any data in until you call
|
||||
* {@link #load}.
|
||||
*/
|
||||
static abstract class BytesReaderBase extends IndexDocValues {
|
||||
protected final IndexInput idxIn;
|
||||
protected final IndexInput datIn;
|
||||
protected final int version;
|
||||
protected final String id;
|
||||
|
||||
protected BytesReaderBase(Directory dir, String id, String codecName,
|
||||
int maxVersion, boolean doIndex) throws IOException {
|
||||
this.id = id;
|
||||
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
boolean success = false;
|
||||
try {
|
||||
version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
|
||||
if (doIndex) {
|
||||
idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
|
||||
Writer.INDEX_EXTENSION));
|
||||
final int version2 = CodecUtil.checkHeader(idxIn, codecName,
|
||||
maxVersion, maxVersion);
|
||||
assert version == version2;
|
||||
} else {
|
||||
idxIn = null;
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
closeInternal();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* clones and returns the data {@link IndexInput}
|
||||
*/
|
||||
protected final IndexInput cloneData() {
|
||||
assert datIn != null;
|
||||
return (IndexInput) datIn.clone();
|
||||
}
|
||||
|
||||
/**
|
||||
* clones and returns the indexing {@link IndexInput}
|
||||
*/
|
||||
protected final IndexInput cloneIndex() {
|
||||
assert idxIn != null;
|
||||
return (IndexInput) idxIn.clone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
super.close();
|
||||
} finally {
|
||||
closeInternal();
|
||||
}
|
||||
}
|
||||
|
||||
private void closeInternal() throws IOException {
|
||||
try {
|
||||
datIn.close();
|
||||
} finally {
|
||||
if (idxIn != null) {
|
||||
idxIn.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,279 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// Stores fixed-length byte[] by deref, ie when two docs
|
||||
// have the same value, they store only 1 byte[]
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class FixedDerefBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "FixedDerefBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class Writer extends BytesWriterBase {
|
||||
private int size = -1;
|
||||
private int[] docToID;
|
||||
private final BytesRefHash hash = new BytesRefHash(pool,
|
||||
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
|
||||
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
|
||||
public Writer(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
|
||||
bytesUsed);
|
||||
}
|
||||
|
||||
public Writer(Directory dir, String id, Allocator allocator,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
|
||||
new ByteBlockPool(allocator), bytesUsed);
|
||||
docToID = new int[1];
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
|
||||
// uses bytes too!
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0) // default value - skip it
|
||||
return;
|
||||
if (size == -1) {
|
||||
size = bytes.length;
|
||||
datOut.writeInt(size);
|
||||
} else if (bytes.length != size) {
|
||||
throw new IllegalArgumentException("expected bytes size=" + size
|
||||
+ " but got " + bytes.length);
|
||||
}
|
||||
int ord = hash.add(bytes);
|
||||
|
||||
if (ord >= 0) {
|
||||
// new added entry
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
} else {
|
||||
ord = (-ord) - 1;
|
||||
}
|
||||
|
||||
if (docID >= docToID.length) {
|
||||
final int size = docToID.length;
|
||||
docToID = ArrayUtil.grow(docToID, 1 + docID);
|
||||
bytesUsed.addAndGet((docToID.length - size)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
docToID[docID] = 1 + ord;
|
||||
}
|
||||
|
||||
// Important that we get docCount, in case there were
|
||||
// some last docs that we didn't see
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (size == -1) {
|
||||
datOut.writeInt(size);
|
||||
}
|
||||
final int count = 1 + hash.size();
|
||||
idxOut.writeInt(count - 1);
|
||||
// write index
|
||||
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
|
||||
PackedInts.bitsRequired(count - 1));
|
||||
final int limit = docCount > docToID.length ? docToID.length : docCount;
|
||||
for (int i = 0; i < limit; i++) {
|
||||
w.add(docToID[i]);
|
||||
}
|
||||
// fill up remaining doc with zeros
|
||||
for (int i = limit; i < docCount; i++) {
|
||||
w.add(0);
|
||||
}
|
||||
w.finish();
|
||||
} finally {
|
||||
hash.close();
|
||||
super.finish(docCount);
|
||||
bytesUsed
|
||||
.addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
|
||||
docToID = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
private final int size;
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true);
|
||||
size = datIn.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
final IndexInput index = cloneIndex();
|
||||
return new Source(cloneData(), index, size, index.readInt());
|
||||
}
|
||||
|
||||
private static class Source extends BytesBaseSource {
|
||||
private final PackedInts.Reader index;
|
||||
private final int size;
|
||||
private final int numValues;
|
||||
|
||||
protected Source(IndexInput datIn, IndexInput idxIn, int size,
|
||||
int numValues) throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues);
|
||||
this.size = size;
|
||||
this.numValues = numValues;
|
||||
index = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int id = (int) index.get(docID);
|
||||
if (id == 0) {
|
||||
bytesRef.length = 0;
|
||||
return bytesRef;
|
||||
}
|
||||
return data.fillSlice(bytesRef, ((id - 1) * size), size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return numValues;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_DEREF;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return index.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
|
||||
}
|
||||
|
||||
static class DerefBytesEnum extends ValuesEnum {
|
||||
protected final IndexInput datIn;
|
||||
private final PackedInts.ReaderIterator idx;
|
||||
protected final long fp;
|
||||
private final int size;
|
||||
private final int valueCount;
|
||||
private int pos = -1;
|
||||
|
||||
public DerefBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
IndexInput idxIn, int size) throws IOException {
|
||||
this(source, datIn, idxIn, size, ValueType.BYTES_FIXED_DEREF);
|
||||
}
|
||||
|
||||
protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
IndexInput idxIn, int size, ValueType enumType) throws IOException {
|
||||
super(source, enumType);
|
||||
this.datIn = datIn;
|
||||
this.size = size;
|
||||
idxIn.readInt();// read valueCount
|
||||
idx = PackedInts.getReaderIterator(idxIn);
|
||||
fp = datIn.getFilePointer();
|
||||
bytesRef.grow(this.size);
|
||||
bytesRef.length = this.size;
|
||||
bytesRef.offset = 0;
|
||||
valueCount = idx.size();
|
||||
}
|
||||
|
||||
protected void copyFrom(ValuesEnum valuesEnum) {
|
||||
bytesRef = valuesEnum.bytesRef;
|
||||
if (bytesRef.bytes.length < size) {
|
||||
bytesRef.grow(size);
|
||||
}
|
||||
bytesRef.length = size;
|
||||
bytesRef.offset = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target < valueCount) {
|
||||
long address;
|
||||
while ((address = idx.advance(target)) == 0) {
|
||||
if (++target >= valueCount) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
}
|
||||
pos = idx.ord();
|
||||
fill(address, bytesRef);
|
||||
return pos;
|
||||
}
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= valueCount) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
datIn.close();
|
||||
} finally {
|
||||
idx.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected void fill(long address, BytesRef ref) throws IOException {
|
||||
datIn.seek(fp + ((address - 1) * size));
|
||||
datIn.readBytes(ref.bytes, 0, size);
|
||||
ref.length = size;
|
||||
ref.offset = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_DEREF;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,242 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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 java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// Stores fixed-length byte[] by deref, ie when two docs
|
||||
// have the same value, they store only 1 byte[]
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class FixedSortedBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "FixedSortedBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class Writer extends BytesWriterBase {
|
||||
private int size = -1;
|
||||
private int[] docToEntry;
|
||||
private final Comparator<BytesRef> comp;
|
||||
|
||||
private final BytesRefHash hash = new BytesRefHash(pool,
|
||||
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
|
||||
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
|
||||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
|
||||
bytesUsed);
|
||||
}
|
||||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
Allocator allocator, AtomicLong bytesUsed) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
|
||||
new ByteBlockPool(allocator), bytesUsed);
|
||||
docToEntry = new int[1];
|
||||
// docToEntry[0] = -1;
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
|
||||
this.comp = comp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0)
|
||||
return; // default - skip it
|
||||
if (size == -1) {
|
||||
size = bytes.length;
|
||||
datOut.writeInt(size);
|
||||
} else if (bytes.length != size) {
|
||||
throw new IllegalArgumentException("expected bytes size=" + size
|
||||
+ " but got " + bytes.length);
|
||||
}
|
||||
if (docID >= docToEntry.length) {
|
||||
final int[] newArray = new int[ArrayUtil.oversize(1 + docID,
|
||||
RamUsageEstimator.NUM_BYTES_INT)];
|
||||
System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
|
||||
bytesUsed.addAndGet((newArray.length - docToEntry.length)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
docToEntry = newArray;
|
||||
}
|
||||
int e = hash.add(bytes);
|
||||
docToEntry[docID] = 1 + (e < 0 ? (-e) - 1 : e);
|
||||
}
|
||||
|
||||
// Important that we get docCount, in case there were
|
||||
// some last docs that we didn't see
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (size == -1) {// no data added
|
||||
datOut.writeInt(size);
|
||||
}
|
||||
final int[] sortedEntries = hash.sort(comp);
|
||||
final int count = hash.size();
|
||||
int[] address = new int[count];
|
||||
// first dump bytes data, recording address as we go
|
||||
for (int i = 0; i < count; i++) {
|
||||
final int e = sortedEntries[i];
|
||||
final BytesRef bytes = hash.get(e, new BytesRef());
|
||||
assert bytes.length == size;
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
address[e] = 1 + i;
|
||||
}
|
||||
|
||||
idxOut.writeInt(count);
|
||||
|
||||
// next write index
|
||||
PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
|
||||
PackedInts.bitsRequired(count));
|
||||
final int limit;
|
||||
if (docCount > docToEntry.length) {
|
||||
limit = docToEntry.length;
|
||||
} else {
|
||||
limit = docCount;
|
||||
}
|
||||
for (int i = 0; i < limit; i++) {
|
||||
final int e = docToEntry[i];
|
||||
if (e == 0) {
|
||||
// null is encoded as zero
|
||||
w.add(0);
|
||||
} else {
|
||||
assert e > 0 && e <= count : "index must 0 > && <= " + count
|
||||
+ " was: " + e;
|
||||
w.add(address[e - 1]);
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = limit; i < docCount; i++) {
|
||||
w.add(0);
|
||||
}
|
||||
w.finish();
|
||||
} finally {
|
||||
super.finish(docCount);
|
||||
bytesUsed.addAndGet((-docToEntry.length)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
docToEntry = null;
|
||||
hash.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
private final int size;
|
||||
|
||||
public Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true);
|
||||
size = datIn.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.lucene.index.values.IndexDocValues.Source load()
|
||||
throws IOException {
|
||||
return loadSorted(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSource loadSorted(Comparator<BytesRef> comp)
|
||||
throws IOException {
|
||||
final IndexInput idxInput = cloneIndex();
|
||||
final IndexInput datInput = cloneData();
|
||||
datInput.seek(CodecUtil.headerLength(CODEC_NAME) + 4);
|
||||
idxInput.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
return new Source(datInput, idxInput, size, idxInput.readInt(), comp);
|
||||
}
|
||||
|
||||
private static class Source extends BytesBaseSortedSource {
|
||||
|
||||
private final PackedInts.Reader index;
|
||||
private final int numValue;
|
||||
private final int size;
|
||||
|
||||
public Source(IndexInput datIn, IndexInput idxIn, int size,
|
||||
int numValues, Comparator<BytesRef> comp) throws IOException {
|
||||
super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size
|
||||
* numValues);
|
||||
this.size = size;
|
||||
this.numValue = numValues;
|
||||
index = PackedInts.getReader(idxIn);
|
||||
closeIndexInput();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int ord(int docID) {
|
||||
return (int) index.get(docID) -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByValue(BytesRef bytes, BytesRef tmpRef) {
|
||||
return binarySearch(bytes, tmpRef, 0, numValue - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return numValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BytesRef deref(int ord, BytesRef bytesRef) {
|
||||
return data.fillSlice(bytesRef, (ord * size), size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_SORTED;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return index.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
// do unsorted
|
||||
return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_SORTED;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,249 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.values.Bytes.BytesBaseSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
|
||||
// Simplest storage: stores fixed length byte[] per
|
||||
// document, with no dedup and no sorting.
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class FixedStraightBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "FixedStraightBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class Writer extends BytesWriterBase {
|
||||
private int size = -1;
|
||||
// start at -1 if the first added value is > 0
|
||||
private int lastDocID = -1;
|
||||
private byte[] oneRecord;
|
||||
|
||||
protected Writer(Directory dir, String id) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null);
|
||||
}
|
||||
|
||||
// TODO - impl bulk copy here!
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (size == -1) {
|
||||
size = bytes.length;
|
||||
datOut.writeInt(size);
|
||||
oneRecord = new byte[size];
|
||||
} else if (bytes.length != size) {
|
||||
throw new IllegalArgumentException("expected bytes size=" + size
|
||||
+ " but got " + bytes.length);
|
||||
}
|
||||
fill(docID);
|
||||
assert bytes.bytes.length >= bytes.length;
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
}
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
*
|
||||
* @see
|
||||
* org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values
|
||||
* .Writer.MergeState)
|
||||
*/
|
||||
@Override
|
||||
protected void merge(MergeState state) throws IOException {
|
||||
if (state.bits == null && state.reader instanceof Reader) {
|
||||
Reader reader = (Reader) state.reader;
|
||||
final int maxDocs = reader.maxDoc;
|
||||
if (maxDocs == 0) {
|
||||
return;
|
||||
}
|
||||
if (size == -1) {
|
||||
size = reader.size;
|
||||
datOut.writeInt(size);
|
||||
oneRecord = new byte[size];
|
||||
}
|
||||
fill(state.docBase);
|
||||
// TODO should we add a transfer to API to each reader?
|
||||
final IndexInput cloneData = reader.cloneData();
|
||||
try {
|
||||
datOut.copyBytes(cloneData, size * maxDocs);
|
||||
} finally {
|
||||
cloneData.close();
|
||||
}
|
||||
|
||||
lastDocID += maxDocs - 1;
|
||||
} else
|
||||
super.merge(state);
|
||||
}
|
||||
|
||||
// Fills up to but not including this docID
|
||||
private void fill(int docID) throws IOException {
|
||||
assert size >= 0;
|
||||
for (int i = lastDocID + 1; i < docID; i++) {
|
||||
datOut.writeBytes(oneRecord, size);
|
||||
}
|
||||
lastDocID = docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (size == -1) {// no data added
|
||||
datOut.writeInt(0);
|
||||
} else {
|
||||
fill(docCount);
|
||||
}
|
||||
} finally {
|
||||
super.finish(docCount);
|
||||
}
|
||||
}
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return oneRecord == null ? 0 : oneRecord.length;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
private final int size;
|
||||
private final int maxDoc;
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, false);
|
||||
size = datIn.readInt();
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new Source(cloneData(), size, maxDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
datIn.close();
|
||||
}
|
||||
|
||||
private static class Source extends BytesBaseSource {
|
||||
private final int size;
|
||||
private final int maxDoc;
|
||||
|
||||
public Source(IndexInput datIn, int size, int maxDoc)
|
||||
throws IOException {
|
||||
super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
|
||||
this.size = size;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
return data.fillSlice(bytesRef, docID * size, size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return maxDoc;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
|
||||
}
|
||||
|
||||
private static final class FixedStraightBytesEnum extends ValuesEnum {
|
||||
private final IndexInput datIn;
|
||||
private final int size;
|
||||
private final int maxDoc;
|
||||
private int pos = -1;
|
||||
private final long fp;
|
||||
|
||||
public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
int size, int maxDoc) throws IOException {
|
||||
super(source, ValueType.BYTES_FIXED_STRAIGHT);
|
||||
this.datIn = datIn;
|
||||
this.size = size;
|
||||
this.maxDoc = maxDoc;
|
||||
bytesRef.grow(size);
|
||||
bytesRef.length = size;
|
||||
bytesRef.offset = 0;
|
||||
fp = datIn.getFilePointer();
|
||||
}
|
||||
|
||||
protected void copyFrom(ValuesEnum valuesEnum) {
|
||||
bytesRef = valuesEnum.bytesRef;
|
||||
if (bytesRef.bytes.length < size) {
|
||||
bytesRef.grow(size);
|
||||
}
|
||||
bytesRef.length = size;
|
||||
bytesRef.offset = 0;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
datIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= maxDoc || size == 0) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
if ((target - 1) != pos) // pos inc == 1
|
||||
datIn.seek(fp + target * size);
|
||||
datIn.readBytes(bytesRef.bytes, 0, size);
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,467 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.Collection;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.FloatsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Exposes {@link Writer} and reader ({@link Source}) for 32 bit and 64 bit
|
||||
* floating point values.
|
||||
* <p>
|
||||
* Current implementations store either 4 byte or 8 byte floating points with
|
||||
* full precision without any compression.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Floats {
|
||||
// TODO - add bulk copy where possible
|
||||
private static final String CODEC_NAME = "SimpleFloats";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
private static final int INT_DEFAULT = Float
|
||||
.floatToRawIntBits(0.0f);
|
||||
private static final long LONG_DEFAULT = Double
|
||||
.doubleToRawLongBits(0.0d);
|
||||
|
||||
|
||||
public static Writer getWriter(Directory dir, String id, int precisionBytes,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
if (precisionBytes != 4 && precisionBytes != 8) {
|
||||
throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
|
||||
+ precisionBytes);
|
||||
}
|
||||
if (precisionBytes == 4) {
|
||||
return new Float4Writer(dir, id, bytesUsed);
|
||||
} else {
|
||||
return new Float8Writer(dir, id, bytesUsed);
|
||||
}
|
||||
}
|
||||
|
||||
public static IndexDocValues getValues(Directory dir, String id, int maxDoc)
|
||||
throws IOException {
|
||||
return new FloatsReader(dir, id, maxDoc);
|
||||
}
|
||||
|
||||
abstract static class FloatsWriter extends Writer {
|
||||
private final String id;
|
||||
private FloatsRef floatsRef;
|
||||
protected int lastDocId = -1;
|
||||
protected IndexOutput datOut;
|
||||
private final byte precision;
|
||||
|
||||
protected FloatsWriter(Directory dir, String id, int precision,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
super(bytesUsed);
|
||||
this.id = id;
|
||||
this.precision = (byte) precision;
|
||||
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
|
||||
assert datOut.getFilePointer() == CodecUtil.headerLength(CODEC_NAME);
|
||||
datOut.writeByte(this.precision);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, datOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID) throws IOException {
|
||||
add(docID, floatsRef.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
add(docID, docValues.getFloat());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setNextEnum(ValuesEnum valuesEnum) {
|
||||
floatsRef = valuesEnum.getFloat();
|
||||
}
|
||||
|
||||
protected abstract int fillDefault(int num) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void merge(MergeState state) throws IOException {
|
||||
if (state.bits == null && state.reader instanceof FloatsReader) {
|
||||
// no deletes - bulk copy
|
||||
// TODO: should be do bulks with deletes too?
|
||||
final FloatsReader reader = (FloatsReader) state.reader;
|
||||
assert reader.precisionBytes == (int) precision;
|
||||
if (reader.maxDoc == 0)
|
||||
return;
|
||||
final int docBase = state.docBase;
|
||||
if (docBase - lastDocId > 1) {
|
||||
// fill with default values
|
||||
lastDocId += fillDefault(docBase - lastDocId - 1);
|
||||
}
|
||||
lastDocId += reader.transferTo(datOut);
|
||||
} else
|
||||
super.merge(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(id, "", Writer.DATA_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
// Writes 4 bytes (float) per value
|
||||
static class Float4Writer extends FloatsWriter {
|
||||
|
||||
protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
super(dir, id, 4, bytesUsed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(final int docID, final double v)
|
||||
throws IOException {
|
||||
assert docID > lastDocId : "docID: " + docID
|
||||
+ " must be greater than the last added doc id: " + lastDocId;
|
||||
if (docID - lastDocId > 1) {
|
||||
// fill with default values
|
||||
lastDocId += fillDefault(docID - lastDocId - 1);
|
||||
}
|
||||
assert datOut != null;
|
||||
datOut.writeInt(Float.floatToRawIntBits((float) v));
|
||||
++lastDocId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (docCount > lastDocId + 1)
|
||||
for (int i = lastDocId; i < docCount; i++) {
|
||||
datOut.writeInt(INT_DEFAULT); // default value
|
||||
}
|
||||
} finally {
|
||||
datOut.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int fillDefault(int numValues) throws IOException {
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
datOut.writeInt(INT_DEFAULT);
|
||||
}
|
||||
return numValues;
|
||||
}
|
||||
}
|
||||
|
||||
// Writes 8 bytes (double) per value
|
||||
static class Float8Writer extends FloatsWriter {
|
||||
|
||||
protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
super(dir, id, 8, bytesUsed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, double v) throws IOException {
|
||||
assert docID > lastDocId : "docID: " + docID
|
||||
+ " must be greater than the last added doc id: " + lastDocId;
|
||||
if (docID - lastDocId > 1) {
|
||||
// fill with default values
|
||||
lastDocId += fillDefault(docID - lastDocId - 1);
|
||||
}
|
||||
assert datOut != null;
|
||||
datOut.writeLong(Double.doubleToRawLongBits(v));
|
||||
++lastDocId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (docCount > lastDocId + 1)
|
||||
for (int i = lastDocId; i < docCount; i++) {
|
||||
datOut.writeLong(LONG_DEFAULT); // default value
|
||||
}
|
||||
} finally {
|
||||
datOut.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int fillDefault(int numValues) throws IOException {
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
datOut.writeLong(LONG_DEFAULT);
|
||||
}
|
||||
return numValues;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens all necessary files, but does not read any data in until you call
|
||||
* {@link #load}.
|
||||
*/
|
||||
static class FloatsReader extends IndexDocValues {
|
||||
|
||||
private final IndexInput datIn;
|
||||
private final int precisionBytes;
|
||||
// TODO(simonw) is ByteBuffer the way to go here?
|
||||
private final int maxDoc;
|
||||
|
||||
protected FloatsReader(Directory dir, String id, int maxDoc)
|
||||
throws IOException {
|
||||
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
|
||||
precisionBytes = datIn.readByte();
|
||||
assert precisionBytes == 4 || precisionBytes == 8;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
int transferTo(IndexOutput out) throws IOException {
|
||||
IndexInput indexInput = (IndexInput) datIn.clone();
|
||||
try {
|
||||
indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
// skip precision:
|
||||
indexInput.readByte();
|
||||
out.copyBytes(indexInput, precisionBytes * maxDoc);
|
||||
} finally {
|
||||
indexInput.close();
|
||||
}
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the actual values. You may call this more than once, eg if you
|
||||
* already previously loaded but then discarded the Source.
|
||||
*/
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
/* we always read BIG_ENDIAN here since the writer uses
|
||||
* DataOutput#writeInt() / writeLong() we can simply read the ints / longs
|
||||
* back in using readInt / readLong */
|
||||
final IndexInput indexInput = (IndexInput) datIn.clone();
|
||||
indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
// skip precision:
|
||||
indexInput.readByte();
|
||||
if (precisionBytes == 4) {
|
||||
final float[] values = new float[(4 * maxDoc) >> 2];
|
||||
assert values.length == maxDoc;
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Float.intBitsToFloat(indexInput.readInt());
|
||||
}
|
||||
return new Source4(values);
|
||||
} else {
|
||||
final double[] values = new double[(8 * maxDoc) >> 3];
|
||||
assert values.length == maxDoc;
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Double.longBitsToDouble(indexInput.readLong());
|
||||
}
|
||||
return new Source8(values);
|
||||
}
|
||||
}
|
||||
|
||||
private class Source4 extends Source {
|
||||
private final float[] values;
|
||||
|
||||
Source4(final float[] values ) throws IOException {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException {
|
||||
return new SourceEnum(attrSource, ValueType.FLOAT_32, this, maxDoc) {
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs)
|
||||
return pos = NO_MORE_DOCS;
|
||||
floatsRef.floats[floatsRef.offset] = source.getFloat(target);
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.FLOAT_32;
|
||||
}
|
||||
}
|
||||
|
||||
private class Source8 extends Source {
|
||||
private final double[] values;
|
||||
|
||||
Source8(final double[] values) throws IOException {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, maxDoc) {
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs)
|
||||
return pos = NO_MORE_DOCS;
|
||||
floatsRef.floats[floatsRef.offset] = source.getFloat(target);
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.FLOAT_64;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
datIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
IndexInput indexInput = (IndexInput) datIn.clone();
|
||||
indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
// skip precision:
|
||||
indexInput.readByte();
|
||||
return precisionBytes == 4 ? new Floats4Enum(source, indexInput, maxDoc)
|
||||
: new Floats8EnumImpl(source, indexInput, maxDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return precisionBytes == 4 ? ValueType.FLOAT_32
|
||||
: ValueType.FLOAT_64;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Floats4Enum extends FloatsEnumImpl {
|
||||
|
||||
Floats4Enum(AttributeSource source, IndexInput dataIn, int maxDoc)
|
||||
throws IOException {
|
||||
super(source, dataIn, 4, maxDoc, ValueType.FLOAT_32);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= maxDoc)
|
||||
return pos = NO_MORE_DOCS;
|
||||
dataIn.seek(fp + (target * precision));
|
||||
final int intBits = dataIn.readInt();
|
||||
floatsRef.floats[0] = Float.intBitsToFloat(intBits);
|
||||
floatsRef.offset = 0;
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class Floats8EnumImpl extends FloatsEnumImpl {
|
||||
|
||||
Floats8EnumImpl(AttributeSource source, IndexInput dataIn, int maxDoc)
|
||||
throws IOException {
|
||||
super(source, dataIn, 8, maxDoc, ValueType.FLOAT_64);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
dataIn.seek(fp + (target * precision));
|
||||
final long value = dataIn.readLong();
|
||||
floatsRef.floats[floatsRef.offset] = Double.longBitsToDouble(value);
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
static abstract class FloatsEnumImpl extends ValuesEnum {
|
||||
protected final IndexInput dataIn;
|
||||
protected int pos = -1;
|
||||
protected final int precision;
|
||||
protected final int maxDoc;
|
||||
protected final long fp;
|
||||
|
||||
FloatsEnumImpl(AttributeSource source, IndexInput dataIn, int precision,
|
||||
int maxDoc, ValueType type) throws IOException {
|
||||
super(source, precision == 4 ? ValueType.FLOAT_32
|
||||
: ValueType.FLOAT_64);
|
||||
this.dataIn = dataIn;
|
||||
this.precision = precision;
|
||||
this.maxDoc = maxDoc;
|
||||
fp = dataIn.getFilePointer();
|
||||
floatsRef.offset = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
dataIn.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,364 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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 java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* {@link IndexDocValues} provides a dense per-document typed storage for fast
|
||||
* value access based on the lucene internal document id. {@link IndexDocValues}
|
||||
* exposes two distinct APIs:
|
||||
* <ul>
|
||||
* <li>via {@link Source} an entirely RAM resident API for random access</li>
|
||||
* <li>via {@link ValuesEnum} a disk resident API for sequential access</li>
|
||||
* </ul> {@link IndexDocValues} are exposed via
|
||||
* {@link IndexReader#perDocValues()} on a per-segment basis. For best
|
||||
* performance {@link IndexDocValues} should be consumed per-segment just like
|
||||
* IndexReader.
|
||||
* <p>
|
||||
* {@link IndexDocValues} are fully integrated into the {@link Codec} API.
|
||||
* Custom implementations can be exposed on a per field basis via
|
||||
* {@link CodecProvider}.
|
||||
*
|
||||
* @see ValueType for limitations and default implementation documentation
|
||||
* @see IndexDocValuesField for adding values to the index
|
||||
* @see Codec#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
|
||||
* customization
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class IndexDocValues implements Closeable {
|
||||
/*
|
||||
* TODO: it might be useful to add another Random Access enum for some
|
||||
* implementations like packed ints and only return such a random access enum
|
||||
* if the impl supports random access. For super large segments it might be
|
||||
* useful or even required in certain environements to have disc based random
|
||||
* access
|
||||
*/
|
||||
public static final IndexDocValues[] EMPTY_ARRAY = new IndexDocValues[0];
|
||||
|
||||
private SourceCache cache = new SourceCache.DirectSourceCache();
|
||||
|
||||
/**
|
||||
* Returns an iterator that steps through all documents values for this
|
||||
* {@link IndexDocValues} field instance. {@link ValuesEnum} will skip document
|
||||
* without a value if applicable.
|
||||
*/
|
||||
public ValuesEnum getEnum() throws IOException {
|
||||
return getEnum(null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an iterator that steps through all documents values for this
|
||||
* {@link IndexDocValues} field instance. {@link ValuesEnum} will skip document
|
||||
* without a value if applicable.
|
||||
* <p>
|
||||
* If an {@link AttributeSource} is supplied to this method the
|
||||
* {@link ValuesEnum} will use the given source to access implementation
|
||||
* related attributes.
|
||||
*/
|
||||
public abstract ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Loads a new {@link Source} instance for this {@link IndexDocValues} field
|
||||
* instance. Source instances returned from this method are not cached. It is
|
||||
* the callers responsibility to maintain the instance and release its
|
||||
* resources once the source is not needed anymore.
|
||||
* <p>
|
||||
* This method will return null iff this {@link IndexDocValues} represent a
|
||||
* {@link SortedSource}.
|
||||
* <p>
|
||||
* For managed {@link Source} instances see {@link #getSource()}.
|
||||
*
|
||||
* @see #getSource()
|
||||
* @see #setCache(SourceCache)
|
||||
*/
|
||||
public abstract Source load() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Source} instance through the current {@link SourceCache}.
|
||||
* Iff no {@link Source} has been loaded into the cache so far the source will
|
||||
* be loaded through {@link #load()} and passed to the {@link SourceCache}.
|
||||
* The caller of this method should not close the obtained {@link Source}
|
||||
* instance unless it is not needed for the rest of its life time.
|
||||
* <p>
|
||||
* {@link Source} instances obtained from this method are closed / released
|
||||
* from the cache once this {@link IndexDocValues} instance is closed by the
|
||||
* {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
|
||||
* {@link IndexDocValues} was created from.
|
||||
* <p>
|
||||
* This method will return null iff this {@link IndexDocValues} represent a
|
||||
* {@link SortedSource}.
|
||||
*/
|
||||
public Source getSource() throws IOException {
|
||||
return cache.load(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link SortedSource} instance for this {@link IndexDocValues} field
|
||||
* instance like {@link #getSource()}.
|
||||
* <p>
|
||||
* This method will return null iff this {@link IndexDocValues} represent a
|
||||
* {@link Source} instead of a {@link SortedSource}.
|
||||
*/
|
||||
public SortedSource getSortedSorted(Comparator<BytesRef> comparator)
|
||||
throws IOException {
|
||||
return cache.loadSorted(this, comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads and returns a {@link SortedSource} instance for this
|
||||
* {@link IndexDocValues} field instance like {@link #load()}.
|
||||
* <p>
|
||||
* This method will return null iff this {@link IndexDocValues} represent a
|
||||
* {@link Source} instead of a {@link SortedSource}.
|
||||
*/
|
||||
public SortedSource loadSorted(Comparator<BytesRef> comparator)
|
||||
throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of this {@link IndexDocValues} instance
|
||||
*/
|
||||
public abstract ValueType type();
|
||||
|
||||
/**
|
||||
* Closes this {@link IndexDocValues} instance. This method should only be called
|
||||
* by the creator of this {@link IndexDocValues} instance. API users should not
|
||||
* close {@link IndexDocValues} instances.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
cache.close(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link SourceCache} used by this {@link IndexDocValues} instance. This
|
||||
* method should be called before {@link #load()} or
|
||||
* {@link #loadSorted(Comparator)} is called. All {@link Source} or
|
||||
* {@link SortedSource} instances in the currently used cache will be closed
|
||||
* before the new cache is installed.
|
||||
* <p>
|
||||
* Note: All instances previously obtained from {@link #load()} or
|
||||
* {@link #loadSorted(Comparator)} will be closed.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if the given cache is <code>null</code>
|
||||
*
|
||||
*/
|
||||
public void setCache(SourceCache cache) {
|
||||
if (cache == null)
|
||||
throw new IllegalArgumentException("cache must not be null");
|
||||
synchronized (this.cache) {
|
||||
this.cache.close(this);
|
||||
this.cache = cache;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Source of per document values like long, double or {@link BytesRef}
|
||||
* depending on the {@link IndexDocValues} fields {@link ValueType}. Source
|
||||
* implementations provide random access semantics similar to array lookups
|
||||
* and typically are entirely memory resident.
|
||||
* <p>
|
||||
* {@link Source} defines 3 {@link ValueType} //TODO finish this
|
||||
*/
|
||||
public static abstract class Source {
|
||||
|
||||
/**
|
||||
* Returns a <tt>long</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>long</tt> values.
|
||||
*/
|
||||
public long getInt(int docID) {
|
||||
throw new UnsupportedOperationException("ints are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a <tt>double</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>double</tt> values.
|
||||
*/
|
||||
public double getFloat(int docID) {
|
||||
throw new UnsupportedOperationException("floats are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link BytesRef} for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>byte[]</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>byte[]</tt> values.
|
||||
*/
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
throw new UnsupportedOperationException("bytes are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns number of unique values. Some implementations may throw
|
||||
* UnsupportedOperationException.
|
||||
*/
|
||||
public int getValueCount() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link ValuesEnum} for this source.
|
||||
*/
|
||||
public ValuesEnum getEnum() throws IOException {
|
||||
return getEnum(null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of this source.
|
||||
*
|
||||
* @return the {@link ValueType} of this source.
|
||||
*/
|
||||
public abstract ValueType type();
|
||||
|
||||
/**
|
||||
* Returns a {@link ValuesEnum} for this source which uses the given
|
||||
* {@link AttributeSource}.
|
||||
*/
|
||||
public abstract ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link ValuesEnum} utility for {@link Source} implemenations.
|
||||
*
|
||||
*/
|
||||
public abstract static class SourceEnum extends ValuesEnum {
|
||||
protected final Source source;
|
||||
protected final int numDocs;
|
||||
protected int pos = -1;
|
||||
|
||||
/**
|
||||
* Creates a new {@link SourceEnum}
|
||||
*
|
||||
* @param attrs
|
||||
* the {@link AttributeSource} for this enum
|
||||
* @param type
|
||||
* the enums {@link ValueType}
|
||||
* @param source
|
||||
* the source this enum operates on
|
||||
* @param numDocs
|
||||
* the number of documents within the source
|
||||
*/
|
||||
protected SourceEnum(AttributeSource attrs, ValueType type, Source source,
|
||||
int numDocs) {
|
||||
super(attrs, type);
|
||||
this.source = source;
|
||||
this.numDocs = numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos == NO_MORE_DOCS)
|
||||
return NO_MORE_DOCS;
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A sorted variant of {@link Source} for <tt>byte[]</tt> values per document.
|
||||
* <p>
|
||||
* Note: {@link ValuesEnum} obtained from a {@link SortedSource} will
|
||||
* enumerate values in document order and not in sorted order.
|
||||
*/
|
||||
public static abstract class SortedSource extends Source {
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int ord = ord(docID);
|
||||
if (ord < 0) {
|
||||
bytesRef.length = 0;
|
||||
} else {
|
||||
getByOrd(ord , bytesRef);
|
||||
}
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns ord for specified docID. If this docID had not been added to the
|
||||
* Writer, the ord is 0. Ord is dense, ie, starts at 0, then increments by 1
|
||||
* for the next (as defined by {@link Comparator} value.
|
||||
*/
|
||||
public abstract int ord(int docID);
|
||||
|
||||
/** Returns value for specified ord. */
|
||||
public abstract BytesRef getByOrd(int ord, BytesRef bytesRef);
|
||||
|
||||
|
||||
/**
|
||||
* Finds the ordinal whose value is greater or equal to the given value.
|
||||
*
|
||||
* @return the given values ordinal if found or otherwise
|
||||
* <code>(-(ord)-1)</code>, defined as the ordinal of the first
|
||||
* element that is greater than the given value. This guarantees
|
||||
* that the return value will always be >= 0 if the given value
|
||||
* is found.
|
||||
*
|
||||
*/
|
||||
public final int getByValue(BytesRef value) {
|
||||
return getByValue(value, new BytesRef());
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a lookup by value.
|
||||
*
|
||||
* @param value
|
||||
* the value to look up
|
||||
* @param tmpRef
|
||||
* a temporary {@link BytesRef} instance used to compare internal
|
||||
* values to the given value. Must not be <code>null</code>
|
||||
* @return the given values ordinal if found or otherwise
|
||||
* <code>(-(ord)-1)</code>, defined as the ordinal of the first
|
||||
* element that is greater than the given value. This guarantees
|
||||
* that the return value will always be >= 0 if the given value
|
||||
* is found.
|
||||
*/
|
||||
public abstract int getByValue(BytesRef value, BytesRef tmpRef);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.IntsImpl.IntsReader;
|
||||
import org.apache.lucene.index.values.IntsImpl.IntsWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Ints {
|
||||
// TODO - add bulk copy where possible
|
||||
|
||||
private Ints() {
|
||||
}
|
||||
|
||||
public static Writer getWriter(Directory dir, String id,
|
||||
boolean useFixedArray, AtomicLong bytesUsed) throws IOException {
|
||||
// TODO - implement fixed?!
|
||||
return new IntsWriter(dir, id, bytesUsed);
|
||||
}
|
||||
|
||||
public static IndexDocValues getValues(Directory dir, String id,
|
||||
boolean useFixedArray) throws IOException {
|
||||
return new IntsReader(dir, id);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,429 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.Collection;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LongsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Stores ints packed with fixed-bit precision.
|
||||
*
|
||||
* @lucene.experimental
|
||||
* */
|
||||
class IntsImpl {
|
||||
|
||||
private static final String CODEC_NAME = "Ints";
|
||||
private static final byte PACKED = 0x00;
|
||||
private static final byte FIXED = 0x01;
|
||||
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class IntsWriter extends Writer {
|
||||
|
||||
// TODO: can we bulkcopy this on a merge?
|
||||
private LongsRef intsRef;
|
||||
private long[] docToValue;
|
||||
private long minValue;
|
||||
private long maxValue;
|
||||
private boolean started;
|
||||
private final String id;
|
||||
private int lastDocId = -1;
|
||||
private IndexOutput datOut;
|
||||
|
||||
protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
super(bytesUsed);
|
||||
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
|
||||
DATA_EXTENSION));
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
|
||||
this.id = id;
|
||||
docToValue = new long[1];
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG); // TODO the
|
||||
// bitset
|
||||
// needs memory
|
||||
// too
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
datOut.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, long v) throws IOException {
|
||||
assert lastDocId < docID;
|
||||
if (!started) {
|
||||
started = true;
|
||||
minValue = maxValue = v;
|
||||
} else {
|
||||
if (v < minValue) {
|
||||
minValue = v;
|
||||
} else if (v > maxValue) {
|
||||
maxValue = v;
|
||||
}
|
||||
}
|
||||
lastDocId = docID;
|
||||
|
||||
if (docID >= docToValue.length) {
|
||||
final long len = docToValue.length;
|
||||
docToValue = ArrayUtil.grow(docToValue, 1 + docID);
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG
|
||||
* ((docToValue.length) - len));
|
||||
}
|
||||
docToValue[docID] = v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (!started) {
|
||||
minValue = maxValue = 0;
|
||||
}
|
||||
// if we exceed the range of positive longs we must switch to fixed ints
|
||||
if ((maxValue - minValue) < (((long)1) << 63) && (maxValue - minValue) >= 0) {
|
||||
writePackedInts(docCount);
|
||||
} else {
|
||||
writeFixedInts(docCount);
|
||||
}
|
||||
|
||||
} finally {
|
||||
datOut.close();
|
||||
bytesUsed
|
||||
.addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG * docToValue.length));
|
||||
docToValue = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void writeFixedInts(int docCount) throws IOException {
|
||||
datOut.writeByte(FIXED);
|
||||
datOut.writeInt(docCount);
|
||||
for (int i = 0; i < docToValue.length; i++) {
|
||||
datOut.writeLong(docToValue[i]); // write full array - we use 0 as default
|
||||
}
|
||||
for (int i = docToValue.length; i < docCount; i++) {
|
||||
datOut.writeLong(0); // fill with defaults values
|
||||
}
|
||||
}
|
||||
|
||||
private void writePackedInts(int docCount) throws IOException {
|
||||
datOut.writeByte(PACKED);
|
||||
datOut.writeLong(minValue);
|
||||
// write a default value to recognize docs without a value for that
|
||||
// field
|
||||
final long defaultValue = maxValue>= 0 && minValue <=0 ? 0-minValue : ++maxValue-minValue;
|
||||
datOut.writeLong(defaultValue);
|
||||
PackedInts.Writer w = PackedInts.getWriter(datOut, docCount,
|
||||
PackedInts.bitsRequired(maxValue-minValue));
|
||||
final int limit = docToValue.length > docCount ? docCount : docToValue.length;
|
||||
for (int i = 0; i < limit; i++) {
|
||||
w.add(docToValue[i] == 0 ? defaultValue : docToValue[i] - minValue);
|
||||
}
|
||||
for (int i = limit; i < docCount; i++) {
|
||||
w.add(defaultValue);
|
||||
}
|
||||
|
||||
w.finish();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID) throws IOException {
|
||||
add(docID, intsRef.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setNextEnum(ValuesEnum valuesEnum) {
|
||||
intsRef = valuesEnum.getInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
add(docID, docValues.getInt());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Collection<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(id, "", DATA_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens all necessary files, but does not read any data in until you call
|
||||
* {@link #load}.
|
||||
*/
|
||||
static class IntsReader extends IndexDocValues {
|
||||
private final IndexInput datIn;
|
||||
private final boolean packed;
|
||||
|
||||
protected IntsReader(Directory dir, String id) throws IOException {
|
||||
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
|
||||
packed = PACKED == datIn.readByte();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, datIn);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the actual values. You may call this more than once, eg if you
|
||||
* already previously loaded but then discarded the Source.
|
||||
*/
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
final IndexInput input = (IndexInput) datIn.clone();
|
||||
boolean success = false;
|
||||
try {
|
||||
final Source source = packed ? new PackedIntsSource(input)
|
||||
: new FixedIntsSource(input);
|
||||
success = true;
|
||||
return source;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, datIn);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class FixedIntsSource extends Source {
|
||||
private final long[] values;
|
||||
public FixedIntsSource(IndexInput dataIn) throws IOException {
|
||||
dataIn.seek(CodecUtil.headerLength(CODEC_NAME) + 1);
|
||||
final int numDocs = dataIn.readInt();
|
||||
values = new long[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = dataIn.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.INTS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, values.length) {
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs)
|
||||
return pos = NO_MORE_DOCS;
|
||||
intsRef.ints[intsRef.offset] = values[target];
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class PackedIntsSource extends Source {
|
||||
private final long minValue;
|
||||
private final long defaultValue;
|
||||
private final PackedInts.Reader values;
|
||||
|
||||
public PackedIntsSource(IndexInput dataIn) throws IOException {
|
||||
dataIn.seek(CodecUtil.headerLength(CODEC_NAME) + 1);
|
||||
minValue = dataIn.readLong();
|
||||
defaultValue = dataIn.readLong();
|
||||
values = PackedInts.getReader(dataIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
// TODO -- can we somehow avoid 2X method calls
|
||||
// on each get? must push minValue down, and make
|
||||
// PackedInts implement Ints.Source
|
||||
assert docID >= 0;
|
||||
final long value = values.get(docID);
|
||||
return value == defaultValue ? 0 : minValue + value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource)
|
||||
throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, values.size()) {
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs)
|
||||
return pos = NO_MORE_DOCS;
|
||||
intsRef.ints[intsRef.offset] = source.getInt(target);
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.INTS;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
datIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
final IndexInput input = (IndexInput) datIn.clone();
|
||||
boolean success = false;
|
||||
try {
|
||||
ValuesEnum inst = packed ? new PackedIntsEnumImpl(source, input)
|
||||
: new FixedIntsEnumImpl(source, input);
|
||||
success = true;
|
||||
return inst;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeSafely(true, input);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.INTS;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final class PackedIntsEnumImpl extends ValuesEnum {
|
||||
private final PackedInts.ReaderIterator ints;
|
||||
private long minValue;
|
||||
private final IndexInput dataIn;
|
||||
private final long defaultValue;
|
||||
private final int maxDoc;
|
||||
private int pos = -1;
|
||||
|
||||
private PackedIntsEnumImpl(AttributeSource source, IndexInput dataIn)
|
||||
throws IOException {
|
||||
super(source, ValueType.INTS);
|
||||
intsRef.offset = 0;
|
||||
this.dataIn = dataIn;
|
||||
dataIn.seek(CodecUtil.headerLength(CODEC_NAME) + 1);
|
||||
minValue = dataIn.readLong();
|
||||
defaultValue = dataIn.readLong();
|
||||
this.ints = PackedInts.getReaderIterator(dataIn);
|
||||
maxDoc = ints.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
ints.close();
|
||||
dataIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
final long val = ints.advance(target);
|
||||
intsRef.ints[intsRef.offset] = val == defaultValue ? 0 : minValue + val;
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class FixedIntsEnumImpl extends ValuesEnum {
|
||||
private final IndexInput dataIn;
|
||||
private final int maxDoc;
|
||||
private int pos = -1;
|
||||
|
||||
private FixedIntsEnumImpl(AttributeSource source, IndexInput dataIn)
|
||||
throws IOException {
|
||||
super(source, ValueType.INTS);
|
||||
intsRef.offset = 0;
|
||||
this.dataIn = dataIn;
|
||||
dataIn.seek(CodecUtil.headerLength(CODEC_NAME) + 1);
|
||||
maxDoc = dataIn.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
dataIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
assert target > pos;
|
||||
if (target > pos+1) {
|
||||
dataIn.seek(dataIn.getFilePointer() + ((target - pos - 1) * 8));
|
||||
}
|
||||
intsRef.ints[intsRef.offset] = dataIn.readLong();
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,269 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
/**
|
||||
* A wrapper for compound IndexReader providing access to per segment
|
||||
* {@link IndexDocValues}
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class MultiIndexDocValues extends IndexDocValues {
|
||||
|
||||
public static class DocValuesIndex {
|
||||
public final static DocValuesIndex[] EMPTY_ARRAY = new DocValuesIndex[0];
|
||||
final int start;
|
||||
final int length;
|
||||
final IndexDocValues docValues;
|
||||
|
||||
public DocValuesIndex(IndexDocValues docValues, int start, int length) {
|
||||
this.docValues = docValues;
|
||||
this.start = start;
|
||||
this.length = length;
|
||||
}
|
||||
}
|
||||
|
||||
private DocValuesIndex[] docValuesIdx;
|
||||
private int[] starts;
|
||||
|
||||
public MultiIndexDocValues() {
|
||||
starts = new int[0];
|
||||
docValuesIdx = new DocValuesIndex[0];
|
||||
}
|
||||
|
||||
public MultiIndexDocValues(DocValuesIndex[] docValuesIdx) {
|
||||
reset(docValuesIdx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new MultiValuesEnum(docValuesIdx, starts);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new MultiSource(docValuesIdx, starts);
|
||||
}
|
||||
|
||||
public IndexDocValues reset(DocValuesIndex[] docValuesIdx) {
|
||||
int[] start = new int[docValuesIdx.length];
|
||||
for (int i = 0; i < docValuesIdx.length; i++) {
|
||||
start[i] = docValuesIdx[i].start;
|
||||
}
|
||||
this.starts = start;
|
||||
this.docValuesIdx = docValuesIdx;
|
||||
return this;
|
||||
}
|
||||
|
||||
public static class DummyDocValues extends IndexDocValues {
|
||||
final int maxDoc;
|
||||
final Source emptySoruce;
|
||||
|
||||
public DummyDocValues(int maxDoc, ValueType type) {
|
||||
this.maxDoc = maxDoc;
|
||||
this.emptySoruce = new EmptySource(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
return emptySoruce.getEnum(attrSource);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return emptySoruce;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return emptySoruce.type();
|
||||
}
|
||||
}
|
||||
|
||||
private static class MultiValuesEnum extends ValuesEnum {
|
||||
private DocValuesIndex[] docValuesIdx;
|
||||
private final int maxDoc;
|
||||
private int currentStart;
|
||||
private int currentMax;
|
||||
private int currentDoc = -1;
|
||||
private ValuesEnum currentEnum;
|
||||
private final int[] starts;
|
||||
|
||||
public MultiValuesEnum(DocValuesIndex[] docValuesIdx, int[] starts)
|
||||
throws IOException {
|
||||
super(docValuesIdx[0].docValues.type());
|
||||
this.docValuesIdx = docValuesIdx;
|
||||
final DocValuesIndex last = docValuesIdx[docValuesIdx.length - 1];
|
||||
maxDoc = last.start + last.length;
|
||||
final DocValuesIndex idx = docValuesIdx[0];
|
||||
currentEnum = idx.docValues.getEnum(this.attributes());
|
||||
currentEnum.copyFrom(this);
|
||||
intsRef = currentEnum.intsRef;
|
||||
currentMax = idx.length;
|
||||
currentStart = 0;
|
||||
this.starts = starts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
currentEnum.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
assert target > currentDoc : "target " + target
|
||||
+ " must be > than the current doc " + currentDoc;
|
||||
int relativeDoc = target - currentStart;
|
||||
do {
|
||||
if (target >= maxDoc) {// we are beyond max doc
|
||||
return currentDoc = NO_MORE_DOCS;
|
||||
}
|
||||
if (target >= currentMax) {
|
||||
final int idx = ReaderUtil.subIndex(target, starts);
|
||||
currentEnum.close();
|
||||
currentEnum = docValuesIdx[idx].docValues.getEnum();
|
||||
currentEnum.copyFrom(this);
|
||||
currentStart = docValuesIdx[idx].start;
|
||||
currentMax = currentStart + docValuesIdx[idx].length;
|
||||
relativeDoc = target - currentStart;
|
||||
}
|
||||
target = currentMax; // make sure that we advance to the next enum if the current is exhausted
|
||||
|
||||
} while ((relativeDoc = currentEnum.advance(relativeDoc)) == NO_MORE_DOCS);
|
||||
return currentDoc = currentStart + relativeDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return advance(currentDoc + 1);
|
||||
}
|
||||
}
|
||||
|
||||
private static class MultiSource extends Source {
|
||||
private int numDocs = 0;
|
||||
private int start = 0;
|
||||
private Source current;
|
||||
private final int[] starts;
|
||||
private final DocValuesIndex[] docValuesIdx;
|
||||
|
||||
public MultiSource(DocValuesIndex[] docValuesIdx, int[] starts) {
|
||||
this.docValuesIdx = docValuesIdx;
|
||||
this.starts = starts;
|
||||
assert docValuesIdx.length != 0;
|
||||
|
||||
}
|
||||
|
||||
public long getInt(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getInt(doc);
|
||||
}
|
||||
|
||||
private final int ensureSource(int docID) {
|
||||
if (docID >= start && docID < start+numDocs) {
|
||||
return docID - start;
|
||||
} else {
|
||||
final int idx = ReaderUtil.subIndex(docID, starts);
|
||||
assert idx >= 0 && idx < docValuesIdx.length : "idx was " + idx
|
||||
+ " for doc id: " + docID + " slices : " + Arrays.toString(starts);
|
||||
assert docValuesIdx[idx] != null;
|
||||
try {
|
||||
current = docValuesIdx[idx].docValues.getSource();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("load failed", e); // TODO how should we
|
||||
// handle this
|
||||
}
|
||||
|
||||
start = docValuesIdx[idx].start;
|
||||
numDocs = docValuesIdx[idx].length;
|
||||
return docID - start;
|
||||
}
|
||||
}
|
||||
|
||||
public double getFloat(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getFloat(doc);
|
||||
}
|
||||
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getBytes(doc, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
throw new UnsupportedOperationException(); // TODO
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return docValuesIdx[0].docValues.type();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class EmptySource extends Source {
|
||||
private final ValueType type;
|
||||
|
||||
public EmptySource(ValueType type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return 0d;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
return ValuesEnum.emptyEnum(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return this.docValuesIdx[0].docValues.type();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.Comparator;
|
||||
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Per document and field values consumed by {@link DocValuesConsumer}.
|
||||
* @see IndexDocValuesField
|
||||
* @see Fieldable#setDocValues(PerDocFieldValues)
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public interface PerDocFieldValues {
|
||||
|
||||
/**
|
||||
* Sets the given <code>long</code> value.
|
||||
*/
|
||||
public void setInt(long value);
|
||||
|
||||
/**
|
||||
* Sets the given <code>float</code> value.
|
||||
*/
|
||||
public void setFloat(float value);
|
||||
|
||||
/**
|
||||
* Sets the given <code>double</code> value.
|
||||
*/
|
||||
public void setFloat(double value);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value and the field's {@link ValueType}. The
|
||||
* comparator for this field is set to <code>null</code>. If a
|
||||
* <code>null</code> comparator is set the default comparator for the given
|
||||
* {@link ValueType} is used.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value, the field's {@link ValueType} and the
|
||||
* field's comparator. If the {@link Comparator} is set to <code>null</code>
|
||||
* the default for the given {@link ValueType} is used instead.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type, Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} or <code>null</code> if not set.
|
||||
*/
|
||||
public BytesRef getBytes();
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} comparator or <code>null</code> if not set
|
||||
*/
|
||||
public Comparator<BytesRef> bytesComparator();
|
||||
|
||||
/**
|
||||
* Returns the set floating point value or <code>0.0d</code> if not set.
|
||||
*/
|
||||
public double getFloat();
|
||||
|
||||
/**
|
||||
* Returns the set <code>long</code> value of <code>0</code> if not set.
|
||||
*/
|
||||
public long getInt();
|
||||
|
||||
/**
|
||||
* Sets the {@link BytesRef} comparator for this field. If the field has a
|
||||
* numeric {@link ValueType} the comparator will be ignored.
|
||||
*/
|
||||
public void setBytesComparator(Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Sets the {@link ValueType}
|
||||
*/
|
||||
public void setType(ValueType type);
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType}
|
||||
*/
|
||||
public ValueType type();
|
||||
|
||||
}
|
|
@ -0,0 +1,120 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Abstract base class for {@link IndexDocValues} {@link Source} /
|
||||
* {@link SortedSource} cache.
|
||||
* <p>
|
||||
* {@link Source} and {@link SortedSource} instances loaded via
|
||||
* {@link IndexDocValues#load()} and {@link IndexDocValues#loadSorted(Comparator)} are
|
||||
* entirely memory resident and need to be maintained by the caller. Each call
|
||||
* to {@link IndexDocValues#load()} or {@link IndexDocValues#loadSorted(Comparator)} will
|
||||
* cause an entire reload of the underlying data. Source and
|
||||
* {@link SortedSource} instances obtained from {@link IndexDocValues#getSource()}
|
||||
* and {@link IndexDocValues#getSource()} respectively are maintained by a
|
||||
* {@link SourceCache} that is closed ({@link #close(IndexDocValues)}) once the
|
||||
* {@link IndexReader} that created the {@link IndexDocValues} instance is closed.
|
||||
* <p>
|
||||
* Unless {@link Source} and {@link SortedSource} instances are managed by
|
||||
* another entity it is recommended to use the cached variants to obtain a
|
||||
* source instance.
|
||||
* <p>
|
||||
* Implementation of this API must be thread-safe.
|
||||
*
|
||||
* @see IndexDocValues#setCache(SourceCache)
|
||||
* @see IndexDocValues#getSource()
|
||||
* @see IndexDocValues#getSortedSorted(Comparator)
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class SourceCache {
|
||||
|
||||
/**
|
||||
* Atomically loads a {@link Source} into the cache from the given
|
||||
* {@link IndexDocValues} and returns it iff no other {@link Source} has already
|
||||
* been cached. Otherwise the cached source is returned.
|
||||
* <p>
|
||||
* This method will not return <code>null</code>
|
||||
*/
|
||||
public abstract Source load(IndexDocValues values) throws IOException;
|
||||
|
||||
/**
|
||||
* Atomically loads a {@link SortedSource} into the cache from the given
|
||||
* {@link IndexDocValues} and returns it iff no other {@link SortedSource} has
|
||||
* already been cached. Otherwise the cached source is returned.
|
||||
* <p>
|
||||
* This method will not return <code>null</code>
|
||||
*/
|
||||
public abstract SortedSource loadSorted(IndexDocValues values,
|
||||
Comparator<BytesRef> comp) throws IOException;
|
||||
|
||||
/**
|
||||
* Atomically invalidates the cached {@link Source} and {@link SortedSource}
|
||||
* instances if any and empties the cache.
|
||||
*/
|
||||
public abstract void invalidate(IndexDocValues values);
|
||||
|
||||
/**
|
||||
* Atomically closes the cache and frees all resources.
|
||||
*/
|
||||
public synchronized void close(IndexDocValues values) {
|
||||
invalidate(values);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple per {@link IndexDocValues} instance cache implementation that holds a
|
||||
* {@link Source} and {@link SortedSource} reference as a member variable.
|
||||
* <p>
|
||||
* If a {@link DirectSourceCache} instance is closed or invalidated the cached
|
||||
* reference are simply set to <code>null</code>
|
||||
*/
|
||||
public static final class DirectSourceCache extends SourceCache {
|
||||
private Source ref;
|
||||
private SortedSource sortedRef;
|
||||
|
||||
public synchronized Source load(IndexDocValues values) throws IOException {
|
||||
if (ref == null) {
|
||||
ref = values.load();
|
||||
}
|
||||
return ref;
|
||||
}
|
||||
|
||||
public synchronized SortedSource loadSorted(IndexDocValues values,
|
||||
Comparator<BytesRef> comp) throws IOException {
|
||||
if (sortedRef == null) {
|
||||
sortedRef = values.loadSorted(comp);
|
||||
}
|
||||
return sortedRef;
|
||||
}
|
||||
|
||||
public synchronized void invalidate(IndexDocValues values) {
|
||||
ref = null;
|
||||
sortedRef = null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,184 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* <code>ValueType</code> specifies the {@link IndexDocValues} type for a
|
||||
* certain field. A <code>ValueType</code> only defines the data type for a field
|
||||
* while the actual implementation used to encode and decode the values depends
|
||||
* on the the {@link Codec#docsConsumer} and {@link Codec#docsProducer} methods.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public enum ValueType {
|
||||
/*
|
||||
* TODO: Add INT_32 INT_64 INT_16 & INT_8?!
|
||||
*/
|
||||
/**
|
||||
* A 64 bit integer value. By default this type uses
|
||||
* {@link PackedInts} to compress the values, as an offset
|
||||
* from the minimum value, as long as the value range
|
||||
* fits into 2<sup>63</sup>-1. Otherwise,
|
||||
* the default implementation falls back to fixed size 64bit
|
||||
* integers.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead. In turn,
|
||||
* {@link ValuesEnum} instances will not skip documents without an explicit
|
||||
* value assigned. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
INTS,
|
||||
|
||||
/**
|
||||
* A 32 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 32bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0f</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0f</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0f</tt> instead. In
|
||||
* turn, {@link ValuesEnum} instances will not skip documents without an
|
||||
* explicit value assigned. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_32,
|
||||
/**
|
||||
* A 64 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 64bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0d</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0d</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0d</tt> instead. In
|
||||
* turn, {@link ValuesEnum} instances will not skip documents without an
|
||||
* explicit value assigned. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_64,
|
||||
|
||||
// TODO(simonw): -- shouldn't lucene decide/detect straight vs
|
||||
// deref, as well fixed vs var?
|
||||
/**
|
||||
* A fixed length straight byte[]. All values added to
|
||||
* such a field must be of the same length. All bytes are stored sequentially
|
||||
* for fast offset access.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0 byte</tt> filled byte[] based on the length of the first seen
|
||||
* value as the default value without any distinction between explicitly
|
||||
* provided values during indexing. All documents without an explicit value
|
||||
* will use the default instead. In turn, {@link ValuesEnum} instances will
|
||||
* not skip documents without an explicit value assigned. Custom default
|
||||
* values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A fixed length dereferenced byte[] variant. Fields with
|
||||
* this type only store distinct byte values and store an additional offset
|
||||
* pointer per document to dereference the shared byte[].
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. In turn, {@link ValuesEnum} instances will skip over documents
|
||||
* without an explicit value assigned. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_DEREF,
|
||||
|
||||
/**
|
||||
* A fixed length pre-sorted byte[] variant. Fields with this type only
|
||||
* store distinct byte values and store an additional offset pointer per
|
||||
* document to dereference the shared byte[]. The stored
|
||||
* byte[] is presorted, by default by unsigned byte order,
|
||||
* and allows access via document id, ordinal and by-value.
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. In turn, {@link ValuesEnum} instances will skip over documents
|
||||
* without an explicit value assigned. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_FIXED_SORTED,
|
||||
|
||||
/**
|
||||
* Variable length straight stored byte[] variant. All bytes are
|
||||
* stored sequentially for compactness. Usage of this type via the
|
||||
* disk-resident API might yield performance degradation since no additional
|
||||
* index is used to advance by more than one document value at a time.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without an
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* byte[] reference. In contrast to dereferenced variants, {@link ValuesEnum}
|
||||
* instances will <b>not</b> skip over documents without an explicit value
|
||||
* assigned. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A variable length dereferenced byte[]. Just like
|
||||
* {@link #BYTES_FIXED_DEREF}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. In turn, {@link ValuesEnum} instances will skip over documents
|
||||
* without an explicit value assigned. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_DEREF,
|
||||
|
||||
/**
|
||||
* A variable length pre-sorted byte[] variant. Just like
|
||||
* {@link #BYTES_FIXED_SORTED}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. In turn, {@link ValuesEnum} instances will skip over documents
|
||||
* without an explicit value assigned. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_VAR_SORTED
|
||||
}
|
|
@ -0,0 +1,173 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FloatsRef;
|
||||
import org.apache.lucene.util.LongsRef;
|
||||
|
||||
/**
|
||||
* {@link ValuesEnum} is a {@link DocIdSetIterator} iterating <tt>byte[]</tt>
|
||||
* , <tt>long</tt> and <tt>double</tt> stored per document. Depending on the
|
||||
* enum's {@link ValueType} ({@link #type()}) the enum might skip over documents that
|
||||
* have no value stored. Types like {@link ValueType#BYTES_VAR_STRAIGHT} might not
|
||||
* skip over documents even if there is no value associated with a document. The
|
||||
* value for document without values again depends on the types implementation
|
||||
* although a reference for a {@link ValueType} returned from a accessor method
|
||||
* {@link #getFloat()}, {@link #getInt()} or {@link #bytes()} will never be
|
||||
* <code>null</code> even if a document has no value.
|
||||
* <p>
|
||||
* Note: Only the reference for the enum's type are initialized to non
|
||||
* <code>null</code> ie. {@link #getInt()} will always return <code>null</code>
|
||||
* if the enum's Type is {@link ValueType#FLOAT_32}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class ValuesEnum extends DocIdSetIterator {
|
||||
private AttributeSource source;
|
||||
private final ValueType enumType;
|
||||
protected BytesRef bytesRef;
|
||||
protected FloatsRef floatsRef;
|
||||
protected LongsRef intsRef;
|
||||
|
||||
/**
|
||||
* Creates a new {@link ValuesEnum} for the given type. The
|
||||
* {@link AttributeSource} for this enum is set to <code>null</code>
|
||||
*/
|
||||
protected ValuesEnum(ValueType enumType) {
|
||||
this(null, enumType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link ValuesEnum} for the given type.
|
||||
*/
|
||||
protected ValuesEnum(AttributeSource source, ValueType enumType) {
|
||||
this.source = source;
|
||||
this.enumType = enumType;
|
||||
switch (enumType) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
bytesRef = new BytesRef();
|
||||
break;
|
||||
case INTS:
|
||||
intsRef = new LongsRef(1);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
floatsRef = new FloatsRef(1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the type of this enum
|
||||
*/
|
||||
public ValueType type() {
|
||||
return enumType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link BytesRef} or <code>null</code> if this enum doesn't
|
||||
* enumerate byte[] values
|
||||
*/
|
||||
public BytesRef bytes() {
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link FloatsRef} or <code>null</code> if this enum doesn't
|
||||
* enumerate floating point values
|
||||
*/
|
||||
public FloatsRef getFloat() {
|
||||
return floatsRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link LongsRef} or <code>null</code> if this enum doesn't
|
||||
* enumerate integer values.
|
||||
*/
|
||||
public LongsRef getInt() {
|
||||
return intsRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the internal state from the given enum
|
||||
*/
|
||||
protected void copyFrom(ValuesEnum valuesEnum) {
|
||||
intsRef = valuesEnum.intsRef;
|
||||
floatsRef = valuesEnum.floatsRef;
|
||||
bytesRef = valuesEnum.bytesRef;
|
||||
source = valuesEnum.source;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link AttributeSource} associated with this enum.
|
||||
* <p>
|
||||
* Note: this method might create a new AttribueSource if no
|
||||
* {@link AttributeSource} has been provided during enum creation.
|
||||
*/
|
||||
public AttributeSource attributes() {
|
||||
if (source == null) {
|
||||
source = new AttributeSource();
|
||||
}
|
||||
return source;
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the enum
|
||||
*
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract void close() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns an empty {@link ValuesEnum} for the given {@link ValueType}.
|
||||
*/
|
||||
public static ValuesEnum emptyEnum(ValueType type) {
|
||||
return new ValuesEnum(type) {
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,287 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// Stores variable-length byte[] by deref, ie when two docs
|
||||
// have the same value, they store only 1 byte[] and both
|
||||
// docs reference that single source
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class VarDerefBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "VarDerefBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
private static final class AddressByteStartArray extends
|
||||
TrackingDirectBytesStartArray {
|
||||
int[] address;
|
||||
|
||||
AddressByteStartArray(int size, AtomicLong bytesUsed) {
|
||||
super(size, bytesUsed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AtomicLong bytesUsed() {
|
||||
return bytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] clear() {
|
||||
if (address != null) {
|
||||
bytesUsed.addAndGet(-address.length * RamUsageEstimator.NUM_BYTES_INT);
|
||||
address = null;
|
||||
}
|
||||
return super.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] grow() {
|
||||
assert address != null;
|
||||
final int oldSize = address.length;
|
||||
final int[] retVal = super.grow();
|
||||
address = ArrayUtil.grow(address, retVal.length);
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
|
||||
* (address.length - oldSize));
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] init() {
|
||||
if (address == null) {
|
||||
address = new int[ArrayUtil.oversize(initSize,
|
||||
RamUsageEstimator.NUM_BYTES_INT)];
|
||||
bytesUsed.addAndGet((address.length) * RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
return super.init();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/*
|
||||
* TODO: if impls like this are merged we are bound to the amount of memory we
|
||||
* can store into a BytesRefHash and therefore how much memory a ByteBlockPool
|
||||
* can address. This is currently limited to 2GB. While we could extend that
|
||||
* and use 64bit for addressing this still limits us to the existing main
|
||||
* memory as all distinct bytes will be loaded up into main memory. We could
|
||||
* move the byte[] writing to #finish(int) and store the bytes in sorted
|
||||
* order and merge them in a streamed fashion.
|
||||
*/
|
||||
static class Writer extends BytesWriterBase {
|
||||
private int[] docToAddress;
|
||||
private int address = 1;
|
||||
|
||||
private final AddressByteStartArray array = new AddressByteStartArray(1,
|
||||
bytesUsed);
|
||||
private final BytesRefHash hash = new BytesRefHash(pool, 16, array);
|
||||
|
||||
public Writer(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
|
||||
bytesUsed);
|
||||
}
|
||||
|
||||
public Writer(Directory dir, String id, Allocator allocator,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
|
||||
new ByteBlockPool(allocator), bytesUsed);
|
||||
docToAddress = new int[1];
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0)
|
||||
return; // default
|
||||
final int e = hash.add(bytes);
|
||||
|
||||
if (docID >= docToAddress.length) {
|
||||
final int oldSize = docToAddress.length;
|
||||
docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
|
||||
* (docToAddress.length - oldSize));
|
||||
}
|
||||
final int docAddress;
|
||||
if (e >= 0) {
|
||||
docAddress = array.address[e] = address;
|
||||
address += writePrefixLength(datOut, bytes);
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
address += bytes.length;
|
||||
} else {
|
||||
docAddress = array.address[(-e) - 1];
|
||||
}
|
||||
docToAddress[docID] = docAddress;
|
||||
}
|
||||
|
||||
private static int writePrefixLength(DataOutput datOut, BytesRef bytes)
|
||||
throws IOException {
|
||||
if (bytes.length < 128) {
|
||||
datOut.writeByte((byte) bytes.length);
|
||||
return 1;
|
||||
} else {
|
||||
datOut.writeByte((byte) (0x80 | (bytes.length >> 8)));
|
||||
datOut.writeByte((byte) (bytes.length & 0xff));
|
||||
return 2;
|
||||
}
|
||||
}
|
||||
|
||||
// Important that we get docCount, in case there were
|
||||
// some last docs that we didn't see
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
idxOut.writeInt(address - 1);
|
||||
// write index
|
||||
// TODO(simonw): -- allow forcing fixed array (not -1)
|
||||
// TODO(simonw): check the address calculation / make it more intuitive
|
||||
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
|
||||
PackedInts.bitsRequired(address - 1));
|
||||
final int limit;
|
||||
if (docCount > docToAddress.length) {
|
||||
limit = docToAddress.length;
|
||||
} else {
|
||||
limit = docCount;
|
||||
}
|
||||
for (int i = 0; i < limit; i++) {
|
||||
w.add(docToAddress[i]);
|
||||
}
|
||||
for (int i = limit; i < docCount; i++) {
|
||||
w.add(0);
|
||||
}
|
||||
w.finish();
|
||||
} finally {
|
||||
hash.close();
|
||||
super.finish(docCount);
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
|
||||
* (-docToAddress.length));
|
||||
docToAddress = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
final IndexInput data = cloneData();
|
||||
final IndexInput index = cloneIndex();
|
||||
data.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
index.seek(CodecUtil.headerLength(CODEC_NAME));
|
||||
final long totalBytes = index.readInt(); // should be long
|
||||
return new Source(data, index, totalBytes);
|
||||
}
|
||||
|
||||
private static class Source extends BytesBaseSource {
|
||||
private final PackedInts.Reader index;
|
||||
|
||||
public Source(IndexInput datIn, IndexInput idxIn, long totalBytes)
|
||||
throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), totalBytes);
|
||||
index = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
long address = index.get(docID);
|
||||
bytesRef.length = 0;
|
||||
return address == 0 ? bytesRef : data.fillSliceWithPrefix(bytesRef,
|
||||
--address);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_DEREF;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return index.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new VarDerefBytesEnum(source, cloneData(), cloneIndex());
|
||||
}
|
||||
|
||||
static class VarDerefBytesEnum extends DerefBytesEnum {
|
||||
|
||||
public VarDerefBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
IndexInput idxIn) throws IOException {
|
||||
super(source, datIn, idxIn, -1, ValueType.BYTES_VAR_DEREF);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void fill(long address, BytesRef ref) throws IOException {
|
||||
datIn.seek(fp + --address);
|
||||
final byte sizeByte = datIn.readByte();
|
||||
final int size;
|
||||
if ((sizeByte & 128) == 0) {
|
||||
// length is 1 byte
|
||||
size = sizeByte;
|
||||
} else {
|
||||
size = ((sizeByte & 0x7f) << 8) | ((datIn.readByte() & 0xff));
|
||||
}
|
||||
if (ref.bytes.length < size)
|
||||
ref.grow(size);
|
||||
ref.length = size;
|
||||
ref.offset = 0;
|
||||
datIn.readBytes(ref.bytes, 0, size);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_DEREF;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,315 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// Stores variable-length byte[] by deref, ie when two docs
|
||||
// have the same value, they store only 1 byte[] and both
|
||||
// docs reference that single source
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class VarSortedBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "VarDerefBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class Writer extends BytesWriterBase {
|
||||
private int[] docToEntry;
|
||||
private final Comparator<BytesRef> comp;
|
||||
|
||||
private final BytesRefHash hash = new BytesRefHash(pool,
|
||||
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
|
||||
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
|
||||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
AtomicLong bytesUsed) throws IOException {
|
||||
this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
|
||||
bytesUsed);
|
||||
}
|
||||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
Allocator allocator, AtomicLong bytesUsed) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
|
||||
new ByteBlockPool(allocator), bytesUsed);
|
||||
this.comp = comp;
|
||||
docToEntry = new int[1];
|
||||
docToEntry[0] = -1;
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0)
|
||||
return;// default
|
||||
if (docID >= docToEntry.length) {
|
||||
int[] newArray = new int[ArrayUtil.oversize(1 + docID,
|
||||
RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
|
||||
System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
|
||||
Arrays.fill(newArray, docToEntry.length, newArray.length, -1);
|
||||
bytesUsed.addAndGet((newArray.length - docToEntry.length)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
docToEntry = newArray;
|
||||
}
|
||||
final int e = hash.add(bytes);
|
||||
docToEntry[docID] = e < 0 ? (-e) - 1 : e;
|
||||
}
|
||||
|
||||
// Important that we get docCount, in case there were
|
||||
// some last docs that we didn't see
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
final int count = hash.size();
|
||||
try {
|
||||
final int[] sortedEntries = hash.sort(comp);
|
||||
// first dump bytes data, recording index & offset as
|
||||
// we go
|
||||
long offset = 0;
|
||||
long lastOffset = 0;
|
||||
final int[] index = new int[count];
|
||||
final long[] offsets = new long[count];
|
||||
for (int i = 0; i < count; i++) {
|
||||
final int e = sortedEntries[i];
|
||||
offsets[i] = offset;
|
||||
index[e] = 1 + i;
|
||||
|
||||
final BytesRef bytes = hash.get(e, new BytesRef());
|
||||
// TODO: we could prefix code...
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
lastOffset = offset;
|
||||
offset += bytes.length;
|
||||
}
|
||||
|
||||
// total bytes of data
|
||||
idxOut.writeLong(offset);
|
||||
|
||||
// write index -- first doc -> 1+ord
|
||||
// TODO(simonw): allow not -1:
|
||||
final PackedInts.Writer indexWriter = PackedInts.getWriter(idxOut,
|
||||
docCount, PackedInts.bitsRequired(count));
|
||||
final int limit = docCount > docToEntry.length ? docToEntry.length
|
||||
: docCount;
|
||||
for (int i = 0; i < limit; i++) {
|
||||
final int e = docToEntry[i];
|
||||
indexWriter.add(e == -1 ? 0 : index[e]);
|
||||
}
|
||||
for (int i = limit; i < docCount; i++) {
|
||||
indexWriter.add(0);
|
||||
}
|
||||
indexWriter.finish();
|
||||
|
||||
// next ord (0-based) -> offset
|
||||
// TODO(simonw): -- allow not -1:
|
||||
PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count,
|
||||
PackedInts.bitsRequired(lastOffset));
|
||||
for (int i = 0; i < count; i++) {
|
||||
offsetWriter.add(offsets[i]);
|
||||
}
|
||||
offsetWriter.finish();
|
||||
} finally {
|
||||
super.finish(docCount);
|
||||
bytesUsed.addAndGet((-docToEntry.length)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
hash.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.lucene.index.values.IndexDocValues.Source load()
|
||||
throws IOException {
|
||||
return loadSorted(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSource loadSorted(Comparator<BytesRef> comp)
|
||||
throws IOException {
|
||||
IndexInput indexIn = cloneIndex();
|
||||
return new Source(cloneData(), indexIn, comp, indexIn.readLong());
|
||||
}
|
||||
|
||||
private static class Source extends BytesBaseSortedSource {
|
||||
private final PackedInts.Reader docToOrdIndex;
|
||||
private final PackedInts.Reader ordToOffsetIndex; // 0-based
|
||||
private final long totBytes;
|
||||
private final int valueCount;
|
||||
|
||||
public Source(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comp, long dataLength) throws IOException {
|
||||
super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), dataLength);
|
||||
totBytes = dataLength;
|
||||
docToOrdIndex = PackedInts.getReader(idxIn);
|
||||
ordToOffsetIndex = PackedInts.getReader(idxIn);
|
||||
valueCount = ordToOffsetIndex.size();
|
||||
closeIndexInput();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int ord(int docID) {
|
||||
return (int) docToOrdIndex.get(docID) - 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByValue(BytesRef bytes, BytesRef tmpRef) {
|
||||
return binarySearch(bytes, tmpRef, 0, valueCount - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
// ord is 0-based
|
||||
@Override
|
||||
protected BytesRef deref(int ord, BytesRef bytesRef) {
|
||||
final long nextOffset;
|
||||
if (ord == valueCount - 1) {
|
||||
nextOffset = totBytes;
|
||||
} else {
|
||||
nextOffset = ordToOffsetIndex.get(1 + ord);
|
||||
}
|
||||
final long offset = ordToOffsetIndex.get(ord);
|
||||
data.fillSlice(bytesRef, offset, (int) (nextOffset - offset));
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_SORTED;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return docToOrdIndex.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new VarSortedBytesEnum(source, cloneData(), cloneIndex());
|
||||
}
|
||||
|
||||
private static class VarSortedBytesEnum extends ValuesEnum {
|
||||
private PackedInts.Reader docToOrdIndex;
|
||||
private PackedInts.Reader ordToOffsetIndex;
|
||||
private IndexInput idxIn;
|
||||
private IndexInput datIn;
|
||||
private int valueCount;
|
||||
private long totBytes;
|
||||
private int docCount;
|
||||
private int pos = -1;
|
||||
private final long fp;
|
||||
|
||||
protected VarSortedBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
IndexInput idxIn) throws IOException {
|
||||
super(source, ValueType.BYTES_VAR_SORTED);
|
||||
totBytes = idxIn.readLong();
|
||||
// keep that in memory to prevent lots of disk seeks
|
||||
docToOrdIndex = PackedInts.getReader(idxIn);
|
||||
ordToOffsetIndex = PackedInts.getReader(idxIn);
|
||||
valueCount = ordToOffsetIndex.size();
|
||||
docCount = docToOrdIndex.size();
|
||||
fp = datIn.getFilePointer();
|
||||
this.idxIn = idxIn;
|
||||
this.datIn = datIn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
idxIn.close();
|
||||
datIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= docCount) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
int ord;
|
||||
while ((ord = (int) docToOrdIndex.get(target)) == 0) {
|
||||
if (++target >= docCount) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
}
|
||||
final long offset = ordToOffsetIndex.get(--ord);
|
||||
final long nextOffset;
|
||||
if (ord == valueCount - 1) {
|
||||
nextOffset = totBytes;
|
||||
} else {
|
||||
nextOffset = ordToOffsetIndex.get(1 + ord);
|
||||
}
|
||||
final int length = (int) (nextOffset - offset);
|
||||
datIn.seek(fp + offset);
|
||||
if (bytesRef.bytes.length < length)
|
||||
bytesRef.grow(length);
|
||||
datIn.readBytes(bytesRef.bytes, 0, length);
|
||||
bytesRef.length = length;
|
||||
bytesRef.offset = 0;
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (pos >= docCount) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_SORTED;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,247 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesBaseSource;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
// Variable length byte[] per document, no sharing
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class VarStraightBytesImpl {
|
||||
|
||||
static final String CODEC_NAME = "VarStraightBytes";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static class Writer extends BytesWriterBase {
|
||||
private long address;
|
||||
// start at -1 if the first added value is > 0
|
||||
private int lastDocID = -1;
|
||||
private long[] docToAddress;
|
||||
|
||||
public Writer(Directory dir, String id, AtomicLong bytesUsed)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed);
|
||||
docToAddress = new long[1];
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
|
||||
public Writer(Directory dir, String id) throws IOException {
|
||||
this(dir, id, new AtomicLong());
|
||||
}
|
||||
|
||||
// Fills up to but not including this docID
|
||||
private void fill(final int docID) {
|
||||
if (docID >= docToAddress.length) {
|
||||
int oldSize = docToAddress.length;
|
||||
docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
|
||||
bytesUsed.addAndGet((docToAddress.length - oldSize)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
for (int i = lastDocID + 1; i < docID; i++) {
|
||||
docToAddress[i] = address;
|
||||
}
|
||||
lastDocID = docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0)
|
||||
return; // default
|
||||
fill(docID);
|
||||
docToAddress[docID] = address;
|
||||
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
address += bytes.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
try {
|
||||
if (lastDocID == -1) {
|
||||
idxOut.writeVLong(0);
|
||||
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
|
||||
PackedInts.bitsRequired(0));
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
w.add(0);
|
||||
}
|
||||
w.finish();
|
||||
} else {
|
||||
fill(docCount);
|
||||
idxOut.writeVLong(address);
|
||||
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
|
||||
PackedInts.bitsRequired(address));
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
w.add(docToAddress[i]);
|
||||
}
|
||||
w.finish();
|
||||
}
|
||||
} finally {
|
||||
bytesUsed.addAndGet(-(docToAddress.length)
|
||||
* RamUsageEstimator.NUM_BYTES_INT);
|
||||
docToAddress = null;
|
||||
super.finish(docCount);
|
||||
}
|
||||
}
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return bytesUsed.get();
|
||||
}
|
||||
}
|
||||
|
||||
public static class Reader extends BytesReaderBase {
|
||||
private final int maxDoc;
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true);
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new Source(cloneData(), cloneIndex());
|
||||
}
|
||||
|
||||
private class Source extends BytesBaseSource {
|
||||
private final PackedInts.Reader addresses;
|
||||
|
||||
public Source(IndexInput datIn, IndexInput idxIn) throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong());
|
||||
addresses = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final long address = addresses.get(docID);
|
||||
final int length = docID == maxDoc - 1 ? (int) (totalLengthInBytes - address)
|
||||
: (int) (addresses.get(1 + docID) - address);
|
||||
return data.fillSlice(bytesRef, address, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
|
||||
return new SourceEnum(attrSource, type(), this, maxDoc()) {
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (target >= numDocs) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
source.getBytes(target, bytesRef);
|
||||
return pos = target;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_STRAIGHT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int maxDoc() {
|
||||
return addresses.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValuesEnum getEnum(AttributeSource source) throws IOException {
|
||||
return new VarStraightBytesEnum(source, cloneData(), cloneIndex());
|
||||
}
|
||||
|
||||
private class VarStraightBytesEnum extends ValuesEnum {
|
||||
private final PackedInts.Reader addresses;
|
||||
private final IndexInput datIn;
|
||||
private final IndexInput idxIn;
|
||||
private final long fp;
|
||||
private final long totBytes;
|
||||
private int pos = -1;
|
||||
|
||||
protected VarStraightBytesEnum(AttributeSource source, IndexInput datIn,
|
||||
IndexInput idxIn) throws IOException {
|
||||
super(source, ValueType.BYTES_VAR_STRAIGHT);
|
||||
totBytes = idxIn.readVLong();
|
||||
fp = datIn.getFilePointer();
|
||||
addresses = PackedInts.getReader(idxIn);
|
||||
this.datIn = datIn;
|
||||
this.idxIn = idxIn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
datIn.close();
|
||||
idxIn.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(final int target) throws IOException {
|
||||
if (target >= maxDoc) {
|
||||
return pos = NO_MORE_DOCS;
|
||||
}
|
||||
final long addr = addresses.get(target);
|
||||
if (addr == totBytes) { // empty values at the end
|
||||
bytesRef.length = 0;
|
||||
bytesRef.offset = 0;
|
||||
return pos = target;
|
||||
}
|
||||
datIn.seek(fp + addr);
|
||||
final int size = (int) (target == maxDoc - 1 ? totBytes - addr
|
||||
: addresses.get(target + 1) - addr);
|
||||
if (bytesRef.bytes.length < size) {
|
||||
bytesRef.grow(size);
|
||||
}
|
||||
bytesRef.length = size;
|
||||
datIn.readBytes(bytesRef.bytes, 0, size);
|
||||
return pos = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return advance(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.BYTES_VAR_STRAIGHT;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,228 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* 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 java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Abstract API for per-document stored primitive values of type <tt>byte[]</tt>
|
||||
* , <tt>long</tt> or <tt>double</tt>. The API accepts a single value for each
|
||||
* document. The underlying storage mechanism, file formats, data-structures and
|
||||
* representations depend on the actual implementation.
|
||||
* <p>
|
||||
* Document IDs passed to this API must always be increasing unless stated
|
||||
* otherwise.
|
||||
* </p>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class Writer extends DocValuesConsumer {
|
||||
|
||||
/**
|
||||
* Creates a new {@link Writer}.
|
||||
*
|
||||
* @param bytesUsed
|
||||
* bytes-usage tracking reference used by implementation to track
|
||||
* internally allocated memory. All tracked bytes must be released
|
||||
* once {@link #finish(int)} has been called.
|
||||
*/
|
||||
protected Writer(AtomicLong bytesUsed) {
|
||||
super(bytesUsed);
|
||||
}
|
||||
|
||||
/**
|
||||
* Filename extension for index files
|
||||
*/
|
||||
public static final String INDEX_EXTENSION = "idx";
|
||||
|
||||
/**
|
||||
* Filename extension for data files.
|
||||
*/
|
||||
public static final String DATA_EXTENSION = "dat";
|
||||
|
||||
/**
|
||||
* Records the specified <tt>long</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>long</tt> values
|
||||
*/
|
||||
public void add(int docID, long value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified <tt>double</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>double</tt> values
|
||||
*/
|
||||
public void add(int docID, double value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified {@link BytesRef} value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* {@link BytesRef} values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record {@link BytesRef} values
|
||||
*/
|
||||
public void add(int docID, BytesRef value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Records a value from the given document id. The methods implementation
|
||||
* obtains the value for the document id from the last {@link ValuesEnum}
|
||||
* set to {@link #setNextEnum(ValuesEnum)}.
|
||||
* <p>
|
||||
* This method is used during merging to provide implementation agnostic
|
||||
* default merge implementation.
|
||||
* </p>
|
||||
* <p>
|
||||
* The given document id must be the same document id returned from
|
||||
* {@link ValuesEnum#docID()} when this method is called. All documents IDs
|
||||
* between the given ID and the previously given ID or <tt>0</tt> if the
|
||||
* method is call the first time are filled with default values depending on
|
||||
* the {@link Writer} implementation. The given document ID must always be
|
||||
* greater than the previous ID or <tt>0</tt> if called the first time.
|
||||
*/
|
||||
protected abstract void add(int docID) throws IOException;
|
||||
|
||||
/**
|
||||
* Sets the next {@link ValuesEnum} to consume values from on calls to
|
||||
* {@link #add(int)}
|
||||
*
|
||||
* @param valuesEnum
|
||||
* the next {@link ValuesEnum}, this must not be null
|
||||
*/
|
||||
protected abstract void setNextEnum(ValuesEnum valuesEnum);
|
||||
|
||||
/**
|
||||
* Finish writing and close any files and resources used by this Writer.
|
||||
*
|
||||
* @param docCount
|
||||
* the total number of documents for this writer. This must be
|
||||
* greater that or equal to the largest document id passed to one of
|
||||
* the add methods after the {@link Writer} was created.
|
||||
*/
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void merge(MergeState state) throws IOException {
|
||||
// This enables bulk copies in subclasses per MergeState, subclasses can
|
||||
// simply override this and decide if they want to merge
|
||||
// segments using this generic implementation or if a bulk merge is possible
|
||||
// / feasible.
|
||||
final ValuesEnum valEnum = state.reader.getEnum();
|
||||
assert valEnum != null;
|
||||
try {
|
||||
setNextEnum(valEnum); // set the current enum we are working on - the
|
||||
// impl. will get the correct reference for the type
|
||||
// it supports
|
||||
int docID = state.docBase;
|
||||
final Bits bits = state.bits;
|
||||
final int docCount = state.docCount;
|
||||
int currentDocId;
|
||||
if ((currentDocId = valEnum.advance(0)) != ValuesEnum.NO_MORE_DOCS) {
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
if (bits == null || !bits.get(i)) {
|
||||
if (currentDocId < i) {
|
||||
if ((currentDocId = valEnum.advance(i)) == ValuesEnum.NO_MORE_DOCS) {
|
||||
break; // advance can jump over default values
|
||||
}
|
||||
}
|
||||
if (currentDocId == i) { // we are on the doc to merge
|
||||
add(docID);
|
||||
}
|
||||
++docID;
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
valEnum.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory method to create a {@link Writer} instance for a given type. This
|
||||
* method returns default implementations for each of the different types
|
||||
* defined in the {@link ValueType} enumeration.
|
||||
*
|
||||
* @param type
|
||||
* the {@link ValueType} to create the {@link Writer} for
|
||||
* @param id
|
||||
* the file name id used to create files within the writer.
|
||||
* @param directory
|
||||
* the {@link Directory} to create the files from.
|
||||
* @param comp
|
||||
* a {@link BytesRef} comparator used for {@link Bytes} variants. If
|
||||
* <code>null</code>
|
||||
* {@link BytesRef#getUTF8SortedAsUnicodeComparator()} is used as the
|
||||
* default.
|
||||
* @param bytesUsed
|
||||
* a byte-usage tracking reference
|
||||
* @return a new {@link Writer} instance for the given {@link ValueType}
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Writer create(ValueType type, String id, Directory directory,
|
||||
Comparator<BytesRef> comp, AtomicLong bytesUsed) throws IOException {
|
||||
if (comp == null) {
|
||||
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
switch (type) {
|
||||
case INTS:
|
||||
return Ints.getWriter(directory, id, true, bytesUsed);
|
||||
case FLOAT_32:
|
||||
return Floats.getWriter(directory, id, 4, bytesUsed);
|
||||
case FLOAT_64:
|
||||
return Floats.getWriter(directory, id, 8, bytesUsed);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, true,
|
||||
bytesUsed);
|
||||
case BYTES_FIXED_DEREF:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, true,
|
||||
bytesUsed);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, true,
|
||||
bytesUsed);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, false,
|
||||
bytesUsed);
|
||||
case BYTES_VAR_DEREF:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, false,
|
||||
bytesUsed);
|
||||
case BYTES_VAR_SORTED:
|
||||
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, false,
|
||||
bytesUsed);
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown Values: " + type);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -32,7 +32,6 @@ import org.apache.lucene.util.Version;
|
|||
/**
|
||||
* A QueryParser which constructs queries to search multiple fields.
|
||||
*
|
||||
* @version $Revision$
|
||||
*/
|
||||
public class MultiFieldQueryParser extends QueryParser
|
||||
{
|
||||
|
|
|
@ -20,8 +20,10 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.search.FieldCache.DocTermsIndex;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.search.FieldCache.DocTerms;
|
||||
import org.apache.lucene.search.FieldCache.DocTermsIndex;
|
||||
import org.apache.lucene.search.cache.ByteValuesCreator;
|
||||
import org.apache.lucene.search.cache.CachedArray;
|
||||
import org.apache.lucene.search.cache.CachedArrayCreator;
|
||||
|
@ -38,9 +40,9 @@ import org.apache.lucene.search.cache.CachedArray.LongValues;
|
|||
import org.apache.lucene.search.cache.CachedArray.ShortValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.Direct8;
|
||||
import org.apache.lucene.util.packed.Direct16;
|
||||
import org.apache.lucene.util.packed.Direct32;
|
||||
import org.apache.lucene.util.packed.Direct8;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
|
@ -157,7 +159,6 @@ public abstract class FieldComparator {
|
|||
* comparators can just return "this" to reuse the same
|
||||
* comparator across segments
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract FieldComparator setNextReader(AtomicReaderContext context) throws IOException;
|
||||
|
||||
|
@ -328,6 +329,68 @@ public abstract class FieldComparator {
|
|||
}
|
||||
}
|
||||
|
||||
/** Uses float index values to sort by ascending value */
|
||||
public static final class FloatDocValuesComparator extends FieldComparator {
|
||||
private final double[] values;
|
||||
private Source currentReaderValues;
|
||||
private final String field;
|
||||
private double bottom;
|
||||
|
||||
FloatDocValuesComparator(int numHits, String field) {
|
||||
values = new double[numHits];
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
final double v1 = values[slot1];
|
||||
final double v2 = values[slot2];
|
||||
if (v1 > v2) {
|
||||
return 1;
|
||||
} else if (v1 < v2) {
|
||||
return -1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
final double v2 = currentReaderValues.getFloat(doc);
|
||||
if (bottom > v2) {
|
||||
return 1;
|
||||
} else if (bottom < v2) {
|
||||
return -1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
values[slot] = currentReaderValues.getFloat(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
final IndexDocValues docValues = context.reader.docValues(field);
|
||||
if (docValues != null) {
|
||||
currentReaderValues = docValues.getSource();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(final int bottom) {
|
||||
this.bottom = values[bottom];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparable<Double> value(int slot) {
|
||||
return Double.valueOf(values[slot]);
|
||||
}
|
||||
}
|
||||
|
||||
/** Parses field's values as float (using {@link
|
||||
* FieldCache#getFloats} and sorts by ascending value */
|
||||
public static final class FloatComparator extends NumericComparator<FloatValues> {
|
||||
|
@ -536,6 +599,72 @@ public abstract class FieldComparator {
|
|||
}
|
||||
}
|
||||
|
||||
/** Loads int index values and sorts by ascending value. */
|
||||
public static final class IntDocValuesComparator extends FieldComparator {
|
||||
private final long[] values;
|
||||
private Source currentReaderValues;
|
||||
private final String field;
|
||||
private long bottom;
|
||||
|
||||
IntDocValuesComparator(int numHits, String field) {
|
||||
values = new long[numHits];
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(int slot1, int slot2) {
|
||||
// TODO: there are sneaky non-branch ways to compute
|
||||
// -1/+1/0 sign
|
||||
final long v1 = values[slot1];
|
||||
final long v2 = values[slot2];
|
||||
if (v1 > v2) {
|
||||
return 1;
|
||||
} else if (v1 < v2) {
|
||||
return -1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
// TODO: there are sneaky non-branch ways to compute
|
||||
// -1/+1/0 sign
|
||||
final long v2 = currentReaderValues.getInt(doc);
|
||||
if (bottom > v2) {
|
||||
return 1;
|
||||
} else if (bottom < v2) {
|
||||
return -1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
values[slot] = currentReaderValues.getInt(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
IndexDocValues docValues = context.reader.docValues(field);
|
||||
if (docValues != null) {
|
||||
currentReaderValues = docValues.getSource();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBottom(final int bottom) {
|
||||
this.bottom = values[bottom];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparable<Long> value(int slot) {
|
||||
return Long.valueOf(values[slot]);
|
||||
}
|
||||
}
|
||||
|
||||
/** Parses field's values as long (using {@link
|
||||
* FieldCache#getLongs} and sorts by ascending value */
|
||||
public static final class LongComparator extends NumericComparator<LongValues> {
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.lucene.util.Bits;
|
|||
* prefix using IndexReader.terms(Term), and use MultiPhraseQuery.add(Term[]
|
||||
* terms) to add them to the query.
|
||||
*
|
||||
* @version 1.0
|
||||
*/
|
||||
public class MultiPhraseQuery extends Query {
|
||||
private String field;
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.io.IOException;
|
|||
/** A Scorer for queries with a required subscorer
|
||||
* and an excluding (prohibited) sub DocIdSetIterator.
|
||||
* <br>
|
||||
* This <code>Scorer</code> implements {@link Scorer#skipTo(int)},
|
||||
* This <code>Scorer</code> implements {@link Scorer#advance(int)},
|
||||
* and it uses the skipTo() on the given scorers.
|
||||
*/
|
||||
class ReqExclScorer extends Scorer {
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
/** A Scorer for queries with a required part and an optional part.
|
||||
* Delays skipTo() on the optional part until a score() is needed.
|
||||
* <br>
|
||||
* This <code>Scorer</code> implements {@link Scorer#skipTo(int)}.
|
||||
* This <code>Scorer</code> implements {@link Scorer#advance(int)}.
|
||||
*/
|
||||
class ReqOptSumScorer extends Scorer {
|
||||
/** The scorers passed from the constructor.
|
||||
|
|
|
@ -18,9 +18,15 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.search.cache.*;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
// TODO(simonw) -- for cleaner transition, maybe we should make
|
||||
// a new SortField that subclasses this one and always uses
|
||||
// index values?
|
||||
|
||||
/**
|
||||
* Stores information about how to sort documents by terms in an individual
|
||||
|
@ -81,6 +87,9 @@ public class SortField {
|
|||
* uses ordinals to do the sorting. */
|
||||
public static final int STRING_VAL = 11;
|
||||
|
||||
/** Sort use byte[] index values. */
|
||||
public static final int BYTES = 12;
|
||||
|
||||
/** Represents sorting by document score (relevance). */
|
||||
public static final SortField FIELD_SCORE = new SortField (null, SCORE);
|
||||
|
||||
|
@ -390,6 +399,26 @@ public class SortField {
|
|||
return hash;
|
||||
}
|
||||
|
||||
private boolean useIndexValues;
|
||||
|
||||
public void setUseIndexValues(boolean b) {
|
||||
useIndexValues = b;
|
||||
}
|
||||
|
||||
public boolean getUseIndexValues() {
|
||||
return useIndexValues;
|
||||
}
|
||||
|
||||
private Comparator<BytesRef> bytesComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
|
||||
public void setBytesComparator(Comparator<BytesRef> b) {
|
||||
bytesComparator = b;
|
||||
}
|
||||
|
||||
public Comparator<BytesRef> getBytesComparator() {
|
||||
return bytesComparator;
|
||||
}
|
||||
|
||||
/** Returns the {@link FieldComparator} to use for
|
||||
* sorting.
|
||||
*
|
||||
|
@ -412,10 +441,18 @@ public class SortField {
|
|||
return new FieldComparator.DocComparator(numHits);
|
||||
|
||||
case SortField.INT:
|
||||
if (useIndexValues) {
|
||||
return new FieldComparator.IntDocValuesComparator(numHits, field);
|
||||
} else {
|
||||
return new FieldComparator.IntComparator(numHits, (IntValuesCreator)creator, (Integer) missingValue);
|
||||
}
|
||||
|
||||
case SortField.FLOAT:
|
||||
if (useIndexValues) {
|
||||
return new FieldComparator.FloatDocValuesComparator(numHits, field);
|
||||
} else {
|
||||
return new FieldComparator.FloatComparator(numHits, (FloatValuesCreator) creator, (Float) missingValue);
|
||||
}
|
||||
|
||||
case SortField.LONG:
|
||||
return new FieldComparator.LongComparator(numHits, (LongValuesCreator)creator, (Long)missingValue );
|
||||
|
|
|
@ -0,0 +1,114 @@
|
|||
package org.apache.lucene.search.function;
|
||||
|
||||
/**
|
||||
* 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.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
|
||||
/**
|
||||
* Expert: obtains numeric field values from a {@link IndexDocValues} field.
|
||||
* This {@link ValueSource} is compatible with all numerical
|
||||
* {@link IndexDocValues}
|
||||
*
|
||||
* @lucene.experimental
|
||||
*
|
||||
*/
|
||||
public class NumericIndexDocValueSource extends ValueSource {
|
||||
|
||||
private final String field;
|
||||
|
||||
public NumericIndexDocValueSource(String field) {
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues getValues(AtomicReaderContext context) throws IOException {
|
||||
final IndexDocValues.Source source = context.reader.docValues(field)
|
||||
.getSource();
|
||||
ValueType type = source.type();
|
||||
switch (type) {
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
return new DocValues() {
|
||||
|
||||
@Override
|
||||
public String toString(int doc) {
|
||||
return "float: [" + floatVal(doc) + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public float floatVal(int doc) {
|
||||
return (float) source.getFloat(doc);
|
||||
}
|
||||
};
|
||||
|
||||
case INTS:
|
||||
return new DocValues() {
|
||||
@Override
|
||||
public String toString(int doc) {
|
||||
return "float: [" + floatVal(doc) + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public float floatVal(int doc) {
|
||||
return (float) source.getInt(doc);
|
||||
}
|
||||
};
|
||||
default:
|
||||
throw new IOException("Type: " + type + "is not numeric");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String description() {
|
||||
return toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + ((field == null) ? 0 : field.hashCode());
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
return true;
|
||||
if (obj == null)
|
||||
return false;
|
||||
if (getClass() != obj.getClass())
|
||||
return false;
|
||||
NumericIndexDocValueSource other = (NumericIndexDocValueSource) obj;
|
||||
if (field == null) {
|
||||
if (other.field != null)
|
||||
return false;
|
||||
} else if (!field.equals(other.field))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DocValues float(" + field + ')';
|
||||
}
|
||||
}
|
|
@ -255,6 +255,19 @@ public final class ArrayUtil {
|
|||
return grow(array, 1 + array.length);
|
||||
}
|
||||
|
||||
public static double[] grow(double[] array, int minSize) {
|
||||
if (array.length < minSize) {
|
||||
double[] newArray = new double[oversize(minSize, RamUsageEstimator.NUM_BYTES_DOUBLE)];
|
||||
System.arraycopy(array, 0, newArray, 0, array.length);
|
||||
return newArray;
|
||||
} else
|
||||
return array;
|
||||
}
|
||||
|
||||
public static double[] grow(double[] array) {
|
||||
return grow(array, 1 + array.length);
|
||||
}
|
||||
|
||||
public static short[] shrink(short[] array, int targetSize) {
|
||||
final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_SHORT);
|
||||
if (newSize != array.length) {
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
|
||||
/**
|
||||
|
@ -79,6 +81,33 @@ public final class ByteBlockPool {
|
|||
|
||||
}
|
||||
|
||||
public static class DirectTrackingAllocator extends Allocator {
|
||||
private final AtomicLong bytesUsed;
|
||||
|
||||
public DirectTrackingAllocator(AtomicLong bytesUsed) {
|
||||
this(BYTE_BLOCK_SIZE, bytesUsed);
|
||||
}
|
||||
|
||||
public DirectTrackingAllocator(int blockSize, AtomicLong bytesUsed) {
|
||||
super(blockSize);
|
||||
this.bytesUsed = bytesUsed;
|
||||
}
|
||||
|
||||
public byte[] getByteBlock() {
|
||||
bytesUsed.addAndGet(blockSize);
|
||||
return new byte[blockSize];
|
||||
}
|
||||
@Override
|
||||
public void recycleByteBlocks(byte[][] blocks, int start, int end) {
|
||||
bytesUsed.addAndGet(-((end-start)* blockSize));
|
||||
for (int i = start; i < end; i++) {
|
||||
blocks[i] = null;
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
|
||||
public byte[][] buffers = new byte[10][];
|
||||
|
||||
int bufferUpto = -1; // Which buffer we are upto
|
||||
|
@ -93,6 +122,20 @@ public final class ByteBlockPool {
|
|||
this.allocator = allocator;
|
||||
}
|
||||
|
||||
public void dropBuffersAndReset() {
|
||||
if (bufferUpto != -1) {
|
||||
// Recycle all but the first buffer
|
||||
allocator.recycleByteBlocks(buffers, 0, 1+bufferUpto);
|
||||
|
||||
// Re-use the first buffer
|
||||
bufferUpto = -1;
|
||||
byteUpto = BYTE_BLOCK_SIZE;
|
||||
byteOffset = -BYTE_BLOCK_SIZE;
|
||||
buffers = new byte[10][];
|
||||
buffer = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
if (bufferUpto != -1) {
|
||||
// We allocated at least one buffer
|
||||
|
|
|
@ -227,8 +227,9 @@ public final class BytesRefHash {
|
|||
public void clear(boolean resetPool) {
|
||||
lastCount = count;
|
||||
count = 0;
|
||||
if (resetPool)
|
||||
pool.reset();
|
||||
if (resetPool) {
|
||||
pool.dropBuffersAndReset();
|
||||
}
|
||||
bytesStart = bytesStartArray.clear();
|
||||
if (lastCount != -1 && shrink(lastCount)) {
|
||||
// shrink clears the hash entries
|
||||
|
@ -241,6 +242,16 @@ public final class BytesRefHash {
|
|||
clear(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the BytesRefHash and releases all internally used memory
|
||||
*/
|
||||
public void close() {
|
||||
clear(true);
|
||||
ords = null;
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
|
||||
* -hashSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new {@link BytesRef}
|
||||
*
|
||||
|
@ -332,6 +343,7 @@ public final class BytesRefHash {
|
|||
// 1 byte to store length
|
||||
buffer[bufferUpto] = (byte) length;
|
||||
pool.byteUpto += length + 1;
|
||||
assert length >= 0: "Length must be positive: " + length;
|
||||
System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 1,
|
||||
length);
|
||||
} else {
|
||||
|
@ -452,10 +464,16 @@ public final class BytesRefHash {
|
|||
* effect.
|
||||
*/
|
||||
public void reinit() {
|
||||
if (bytesStart == null)
|
||||
if (bytesStart == null) {
|
||||
bytesStart = bytesStartArray.init();
|
||||
}
|
||||
|
||||
if (ords == null) {
|
||||
ords = new int[hashSize];
|
||||
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * hashSize);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the bytesStart offset into the internally used
|
||||
* {@link ByteBlockPool} for the given ord
|
||||
|
@ -515,16 +533,61 @@ public final class BytesRefHash {
|
|||
public abstract AtomicLong bytesUsed();
|
||||
}
|
||||
|
||||
public static class DirectBytesStartArray extends BytesStartArray {
|
||||
|
||||
/**
|
||||
* A direct {@link BytesStartArray} that tracks all memory allocation using an {@link AtomicLong} instance.
|
||||
*/
|
||||
public static class TrackingDirectBytesStartArray extends BytesStartArray {
|
||||
protected final int initSize;
|
||||
private int[] bytesStart;
|
||||
private final AtomicLong bytesUsed = new AtomicLong(0);
|
||||
protected final AtomicLong bytesUsed;
|
||||
|
||||
public TrackingDirectBytesStartArray(int initSize, AtomicLong bytesUsed) {
|
||||
this.initSize = initSize;
|
||||
this.bytesUsed = bytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] clear() {
|
||||
if (bytesStart != null) {
|
||||
bytesUsed.addAndGet(-bytesStart.length * RamUsageEstimator.NUM_BYTES_INT);
|
||||
}
|
||||
return bytesStart = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] grow() {
|
||||
assert bytesStart != null;
|
||||
final int oldSize = bytesStart.length;
|
||||
bytesStart = ArrayUtil.grow(bytesStart, bytesStart.length + 1);
|
||||
bytesUsed.addAndGet((bytesStart.length - oldSize) * RamUsageEstimator.NUM_BYTES_INT);
|
||||
return bytesStart;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] init() {
|
||||
bytesStart = new int[ArrayUtil.oversize(initSize,
|
||||
RamUsageEstimator.NUM_BYTES_INT)];
|
||||
bytesUsed.addAndGet((bytesStart.length) * RamUsageEstimator.NUM_BYTES_INT);
|
||||
return bytesStart;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AtomicLong bytesUsed() {
|
||||
return bytesUsed;
|
||||
}
|
||||
}
|
||||
|
||||
public static class DirectBytesStartArray extends BytesStartArray {
|
||||
protected final int initSize;
|
||||
private int[] bytesStart;
|
||||
private final AtomicLong bytesUsed;
|
||||
|
||||
public DirectBytesStartArray(int initSize) {
|
||||
this.bytesUsed = new AtomicLong(0);
|
||||
this.initSize = initSize;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int[] clear() {
|
||||
return bytesStart = null;
|
||||
|
@ -546,6 +609,5 @@ public final class BytesRefHash {
|
|||
public AtomicLong bytesUsed() {
|
||||
return bytesUsed;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,109 @@
|
|||
package org.apache.lucene.util;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Represents double[], as a slice (offset + length) into an existing double[].
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public final class FloatsRef implements Cloneable {
|
||||
public double[] floats;
|
||||
public int offset;
|
||||
public int length;
|
||||
|
||||
public FloatsRef() {
|
||||
}
|
||||
|
||||
public FloatsRef(int capacity) {
|
||||
floats = new double[capacity];
|
||||
}
|
||||
|
||||
public void set(double value) {
|
||||
floats[offset] = value;
|
||||
}
|
||||
|
||||
public double get() {
|
||||
return floats[offset];
|
||||
}
|
||||
|
||||
public FloatsRef(double[] floats, int offset, int length) {
|
||||
this.floats = floats;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
public FloatsRef(FloatsRef other) {
|
||||
copy(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object clone() {
|
||||
return new FloatsRef(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 0;
|
||||
final int end = offset + length;
|
||||
for(int i = offset; i < end; i++) {
|
||||
long value = Double.doubleToLongBits(floats[i]);
|
||||
result = prime * result + (int) (value ^ (value >>> 32));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
return other instanceof FloatsRef && this.floatsEquals((FloatsRef) other);
|
||||
}
|
||||
|
||||
public boolean floatsEquals(FloatsRef other) {
|
||||
if (length == other.length) {
|
||||
int otherUpto = other.offset;
|
||||
final double[] otherFloats = other.floats;
|
||||
final int end = offset + length;
|
||||
for(int upto=offset;upto<end;upto++,otherUpto++) {
|
||||
if (floats[upto] != otherFloats[otherUpto]) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public void copy(FloatsRef other) {
|
||||
if (floats == null) {
|
||||
floats = new double[other.length];
|
||||
} else {
|
||||
floats = ArrayUtil.grow(floats, other.length);
|
||||
}
|
||||
System.arraycopy(other.floats, other.offset, floats, 0, other.length);
|
||||
length = other.length;
|
||||
offset = 0;
|
||||
}
|
||||
|
||||
public void grow(int newLength) {
|
||||
if (floats.length < newLength) {
|
||||
floats = ArrayUtil.grow(floats, newLength);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,109 @@
|
|||
package org.apache.lucene.util;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Represents long[], as a slice (offset + length) into an existing long[].
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public final class LongsRef implements Cloneable {
|
||||
public long[] ints;
|
||||
public int offset;
|
||||
public int length;
|
||||
|
||||
public LongsRef() {
|
||||
}
|
||||
|
||||
public LongsRef(int capacity) {
|
||||
ints = new long[capacity];
|
||||
}
|
||||
|
||||
public LongsRef(long[] ints, int offset, int length) {
|
||||
this.ints = ints;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
public LongsRef(LongsRef other) {
|
||||
copy(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object clone() {
|
||||
return new LongsRef(this);
|
||||
}
|
||||
|
||||
public void set(long value) {
|
||||
ints[offset] = value;
|
||||
}
|
||||
|
||||
public long get() {
|
||||
return ints[offset];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 0;
|
||||
final int end = offset + length;
|
||||
for (int i = offset; i < end; i++) {
|
||||
long value = ints[i];
|
||||
result = prime * result + (int) (value ^ (value >>> 32));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
return this.intsEquals((LongsRef) other);
|
||||
}
|
||||
|
||||
public boolean intsEquals(LongsRef other) {
|
||||
if (length == other.length) {
|
||||
int otherUpto = other.offset;
|
||||
final long[] otherInts = other.ints;
|
||||
final int end = offset + length;
|
||||
for (int upto = offset; upto < end; upto++, otherUpto++) {
|
||||
if (ints[upto] != otherInts[otherUpto]) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public void copy(LongsRef other) {
|
||||
if (ints == null) {
|
||||
ints = new long[other.length];
|
||||
} else {
|
||||
ints = ArrayUtil.grow(ints, other.length);
|
||||
}
|
||||
System.arraycopy(other.ints, other.offset, ints, 0, other.length);
|
||||
length = other.length;
|
||||
offset = 0;
|
||||
}
|
||||
|
||||
public void grow(int newLength) {
|
||||
if (ints.length < newLength) {
|
||||
ints = ArrayUtil.grow(ints, newLength);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -185,6 +185,55 @@ public final class PagedBytes {
|
|||
return start;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Gets a slice out of {@link PagedBytes} starting at <i>start</i>, the
|
||||
* length is read as 1 or 2 byte vInt prefix. Iff the slice spans across a
|
||||
* block border this method will allocate sufficient resources and copy the
|
||||
* paged data.
|
||||
* <p>
|
||||
* Slices spanning more than one block are not supported.
|
||||
* </p>
|
||||
*
|
||||
* @lucene.internal
|
||||
**/
|
||||
public BytesRef fillSliceWithPrefix(BytesRef b, long start) {
|
||||
final int index = (int) (start >> blockBits);
|
||||
int offset = (int) (start & blockMask);
|
||||
final byte[] block = blocks[index];
|
||||
final int length;
|
||||
if ((block[offset] & 128) == 0) {
|
||||
length = block[offset];
|
||||
offset = offset+1;
|
||||
} else {
|
||||
length = ((block[offset] & 0x7f) << 8) | (block[1+offset] & 0xff);
|
||||
offset = offset+2;
|
||||
assert length > 0;
|
||||
}
|
||||
assert length >= 0: "length=" + length;
|
||||
b.length = length;
|
||||
if (blockSize - offset >= length) {
|
||||
// Within block
|
||||
b.offset = offset;
|
||||
b.bytes = blocks[index];
|
||||
} else {
|
||||
// Split
|
||||
byte[] buffer = threadBuffers.get();
|
||||
if (buffer == null) {
|
||||
buffer = new byte[length];
|
||||
threadBuffers.set(buffer);
|
||||
} else if (buffer.length < length) {
|
||||
buffer = ArrayUtil.grow(buffer, length);
|
||||
threadBuffers.set(buffer);
|
||||
}
|
||||
b.bytes = buffer;
|
||||
b.offset = 0;
|
||||
System.arraycopy(blocks[index], offset, buffer, 0, blockSize-offset);
|
||||
System.arraycopy(blocks[1+index], 0, buffer, blockSize-offset, length-(blockSize-offset));
|
||||
}
|
||||
return b;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public byte[][] getBlocks() {
|
||||
return blocks;
|
||||
|
|
|
@ -182,7 +182,7 @@ class Packed64 extends PackedInts.ReaderImpl implements PackedInts.Mutable {
|
|||
final int bitPos = (int)(majorBitPos & MOD_MASK); // % BLOCK_SIZE);
|
||||
|
||||
final int base = bitPos * FAC_BITPOS;
|
||||
|
||||
assert elementPos < blocks.length : "elementPos: " + elementPos + "; blocks.len: " + blocks.length;
|
||||
return ((blocks[elementPos] << shifts[base]) >>> shifts[base+1]) |
|
||||
((blocks[elementPos+1] >>> shifts[base+2]) & readMasks[bitPos]);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,10 @@
|
|||
|
||||
use strict;
|
||||
use warnings;
|
||||
use JSON;
|
||||
use LWP::Simple;
|
||||
|
||||
my $project_info_url = 'https://issues.apache.org/jira/rest/api/2.0.alpha1/project/LUCENE';
|
||||
my $jira_url_prefix = 'http://issues.apache.org/jira/browse/';
|
||||
my $bugzilla_url_prefix = 'http://issues.apache.org/bugzilla/show_bug.cgi?id=';
|
||||
my %release_dates = &setup_release_dates;
|
||||
|
@ -648,8 +651,13 @@ sub get_release_date {
|
|||
# Returns a list of alternating release names and dates, for use in populating
|
||||
# the %release_dates hash.
|
||||
#
|
||||
# Pulls release dates via the JIRA REST API. JIRA does not list
|
||||
# X.Y RCZ releases independently from releases X.Y, so the RC dates
|
||||
# as well as those named "final" are included below.
|
||||
#
|
||||
sub setup_release_dates {
|
||||
return ( '0.01' => '2000-03-30', '0.04' => '2000-04-19',
|
||||
my %release_dates
|
||||
= ( '0.01' => '2000-03-30', '0.04' => '2000-04-19',
|
||||
'1.0' => '2000-10-04', '1.01b' => '2001-06-02',
|
||||
'1.2 RC1' => '2001-10-02', '1.2 RC2' => '2001-10-19',
|
||||
'1.2 RC3' => '2002-01-27', '1.2 RC4' => '2002-02-14',
|
||||
|
@ -667,6 +675,20 @@ sub setup_release_dates {
|
|||
'2.4.0' => '2008-10-06', '2.4.1' => '2009-03-09',
|
||||
'2.9.0' => '2009-09-23', '2.9.1' => '2009-11-06',
|
||||
'3.0.0' => '2009-11-25');
|
||||
my $project_info_json = get($project_info_url);
|
||||
my $project_info = decode_json($project_info_json);
|
||||
for my $version (@{$project_info->{versions}}) {
|
||||
if ($version->{releaseDate}) {
|
||||
my $date = substr($version->{releaseDate}, 0, 10);
|
||||
my $version_name = $version->{name};
|
||||
$release_dates{$version->{name}} = $date;
|
||||
if ($version_name =~ /^\d+\.\d+$/) {
|
||||
my $full_version_name = "$version->{name}.0";
|
||||
$release_dates{$full_version_name} = $date;
|
||||
}
|
||||
}
|
||||
}
|
||||
return %release_dates;
|
||||
}
|
||||
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue