mirror of https://github.com/apache/lucene.git
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
This commit is contained in:
commit
8683da80ed
|
@ -14,7 +14,6 @@
|
|||
<orderEntry type="library" scope="TEST" name="JUnit" level="project" />
|
||||
<orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
|
||||
<orderEntry type="module" module-name="grouping" />
|
||||
<orderEntry type="module" module-name="backward-codecs" />
|
||||
<orderEntry type="module" module-name="lucene-core" />
|
||||
</component>
|
||||
</module>
|
||||
|
|
|
@ -17,6 +17,5 @@
|
|||
<orderEntry type="module" module-name="lucene-core" />
|
||||
<orderEntry type="module" module-name="queries" />
|
||||
<orderEntry type="module" module-name="sandbox" />
|
||||
<orderEntry type="module" module-name="backward-codecs" />
|
||||
</component>
|
||||
</module>
|
||||
|
|
|
@ -190,16 +190,23 @@ def checkJARMetaData(desc, jarFile, gitRevision, version):
|
|||
'Implementation-Vendor: The Apache Software Foundation',
|
||||
# Make sure 1.8 compiler was used to build release bits:
|
||||
'X-Compile-Source-JDK: 8',
|
||||
# Make sure 1.8 ant was used to build release bits: (this will match 1.8+)
|
||||
'Ant-Version: Apache Ant 1.8',
|
||||
# Make sure 1.8 or 1.9 ant was used to build release bits: (this will match 1.8.x, 1.9.x)
|
||||
('Ant-Version: Apache Ant 1.8', 'Ant-Version: Apache Ant 1.9'),
|
||||
# Make sure .class files are 1.8 format:
|
||||
'X-Compile-Target-JDK: 8',
|
||||
'Specification-Version: %s' % version,
|
||||
# Make sure the release was compiled with 1.8:
|
||||
'Created-By: 1.8'):
|
||||
if s.find(verify) == -1:
|
||||
raise RuntimeError('%s is missing "%s" inside its META-INF/MANIFEST.MF' % \
|
||||
(desc, verify))
|
||||
if type(verify) is not tuple:
|
||||
verify = (verify,)
|
||||
for x in verify:
|
||||
if s.find(x) != -1:
|
||||
break
|
||||
else:
|
||||
if len(verify) == 1:
|
||||
raise RuntimeError('%s is missing "%s" inside its META-INF/MANIFEST.MF' % (desc, verify[0]))
|
||||
else:
|
||||
raise RuntimeError('%s is missing one of "%s" inside its META-INF/MANIFEST.MF' % (desc, verify))
|
||||
|
||||
if gitRevision != 'skip':
|
||||
# Make sure this matches the version and git revision we think we are releasing:
|
||||
|
|
|
@ -31,7 +31,12 @@ Other
|
|||
* LUCENE-7360: Remove Explanation.toHtml() (Alan Woodward)
|
||||
|
||||
======================= Lucene 6.3.0 =======================
|
||||
(No Changes)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-7416: BooleanQuery optimizes queries that have queries that occur both
|
||||
in the sets of SHOULD and FILTER clauses, or both in MUST/FILTER and MUST_NOT
|
||||
clauses. (Spyros Kapnissis via Adrien Grand)
|
||||
|
||||
======================= Lucene 6.2.0 =======================
|
||||
|
||||
|
|
|
@ -20,17 +20,12 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.CannedTokenStream;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.core.KeywordTokenizer;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -43,7 +38,7 @@ public class TestTrimFilter extends BaseTokenStreamTestCase {
|
|||
char[] whitespace = " ".toCharArray();
|
||||
char[] empty = "".toCharArray();
|
||||
|
||||
TokenStream ts = new IterTokenStream(new Token(new String(a, 0, a.length), 1, 5),
|
||||
TokenStream ts = new CannedTokenStream(new Token(new String(a, 0, a.length), 1, 5),
|
||||
new Token(new String(b, 0, b.length), 6, 10),
|
||||
new Token(new String(ccc, 0, ccc.length), 11, 15),
|
||||
new Token(new String(whitespace, 0, whitespace.length), 16, 20),
|
||||
|
@ -53,43 +48,6 @@ public class TestTrimFilter extends BaseTokenStreamTestCase {
|
|||
assertTokenStreamContents(ts, new String[] { "a", "b", "cCc", "", ""});
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated (3.0) does not support custom attributes
|
||||
*/
|
||||
@Deprecated
|
||||
private static class IterTokenStream extends TokenStream {
|
||||
final Token tokens[];
|
||||
int index = 0;
|
||||
CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
|
||||
TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
|
||||
PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
|
||||
|
||||
public IterTokenStream(Token... tokens) {
|
||||
super();
|
||||
this.tokens = tokens;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (index >= tokens.length)
|
||||
return false;
|
||||
else {
|
||||
clearAttributes();
|
||||
Token token = tokens[index++];
|
||||
termAtt.setEmpty().append(token);
|
||||
offsetAtt.setOffset(token.startOffset(), token.endOffset());
|
||||
posIncAtt.setPositionIncrement(token.getPositionIncrement());
|
||||
flagsAtt.setFlags(token.getFlags());
|
||||
typeAtt.setType(token.type());
|
||||
payloadAtt.setPayload(token.getPayload());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** blast some random strings through the analyzer */
|
||||
public void testRandomStrings() throws Exception {
|
||||
Analyzer a = new Analyzer() {
|
||||
|
|
|
@ -1,170 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 5.0 index format, with configurable per-field postings
|
||||
* and docvalues formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene50 package documentation for file format details.
|
||||
* @deprecated Only for reading old 5.0-5.2 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene50Codec extends Codec {
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
|
||||
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene50Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene50Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final StoredFieldsFormat storedFieldsFormat;
|
||||
|
||||
/**
|
||||
* Instantiates a new codec.
|
||||
*/
|
||||
public Lucene50Codec() {
|
||||
this(Mode.BEST_SPEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instantiates a new codec, specifying the stored fields compression
|
||||
* mode to use.
|
||||
* @param mode stored fields compression mode to use for newly
|
||||
* flushed/merged segments.
|
||||
*/
|
||||
public Lucene50Codec(Mode mode) {
|
||||
super("Lucene50");
|
||||
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final CompoundFormat compoundFormat() {
|
||||
return compoundFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
/** Returns the docvalues format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene50NormsFormat();
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -1,658 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.io.Closeable; // javadocs
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LongsRef;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.DirectWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
|
||||
|
||||
/** writer for {@link Lucene50DocValuesFormat} */
|
||||
class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
|
||||
|
||||
IndexOutput data, meta;
|
||||
final int maxDoc;
|
||||
|
||||
/** expert: Creates a new writer */
|
||||
public Lucene50DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.createOutput(dataName, state.context);
|
||||
CodecUtil.writeIndexHeader(data, dataCodec, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeIndexHeader(meta, metaCodec, Lucene50DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
addNumericField(field, values, true);
|
||||
}
|
||||
|
||||
void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
|
||||
long count = 0;
|
||||
long minValue = Long.MAX_VALUE;
|
||||
long maxValue = Long.MIN_VALUE;
|
||||
long gcd = 0;
|
||||
long missingCount = 0;
|
||||
long zeroCount = 0;
|
||||
// TODO: more efficient?
|
||||
HashSet<Long> uniqueValues = null;
|
||||
if (optimizeStorage) {
|
||||
uniqueValues = new HashSet<>();
|
||||
|
||||
for (Number nv : values) {
|
||||
final long v;
|
||||
if (nv == null) {
|
||||
v = 0;
|
||||
missingCount++;
|
||||
zeroCount++;
|
||||
} else {
|
||||
v = nv.longValue();
|
||||
if (v == 0) {
|
||||
zeroCount++;
|
||||
}
|
||||
}
|
||||
|
||||
if (gcd != 1) {
|
||||
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
|
||||
// in that case v - minValue might overflow and make the GCD computation return
|
||||
// wrong results. Since these extreme values are unlikely, we just discard
|
||||
// GCD computation for them
|
||||
gcd = 1;
|
||||
} else if (count != 0) { // minValue needs to be set first
|
||||
gcd = MathUtil.gcd(gcd, v - minValue);
|
||||
}
|
||||
}
|
||||
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (uniqueValues.add(v)) {
|
||||
if (uniqueValues.size() > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++count;
|
||||
}
|
||||
} else {
|
||||
for (Number nv : values) {
|
||||
long v = nv.longValue();
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
++count;
|
||||
}
|
||||
}
|
||||
|
||||
final long delta = maxValue - minValue;
|
||||
final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
|
||||
final int tableBitsRequired = uniqueValues == null
|
||||
? Integer.MAX_VALUE
|
||||
: DirectWriter.bitsRequired(uniqueValues.size() - 1);
|
||||
|
||||
final int format;
|
||||
if (uniqueValues != null
|
||||
&& count <= Integer.MAX_VALUE
|
||||
&& (uniqueValues.size() == 1
|
||||
|| (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
|
||||
// either one unique value C or two unique values: "missing" and C
|
||||
format = CONST_COMPRESSED;
|
||||
} else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
|
||||
format = TABLE_COMPRESSED;
|
||||
} else if (gcd != 0 && gcd != 1) {
|
||||
final long gcdDelta = (maxValue - minValue) / gcd;
|
||||
final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
|
||||
format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
|
||||
} else {
|
||||
format = DELTA_COMPRESSED;
|
||||
}
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(format);
|
||||
if (missingCount == 0) {
|
||||
meta.writeLong(ALL_LIVE);
|
||||
} else if (missingCount == count) {
|
||||
meta.writeLong(ALL_MISSING);
|
||||
} else {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
}
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(count);
|
||||
|
||||
switch (format) {
|
||||
case CONST_COMPRESSED:
|
||||
// write the constant (nonzero value in the n=2 case, singleton value otherwise)
|
||||
meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
|
||||
break;
|
||||
case GCD_COMPRESSED:
|
||||
meta.writeLong(minValue);
|
||||
meta.writeLong(gcd);
|
||||
final long maxDelta = (maxValue - minValue) / gcd;
|
||||
final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
|
||||
meta.writeVInt(bits);
|
||||
final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
|
||||
for (Number nv : values) {
|
||||
long value = nv == null ? 0 : nv.longValue();
|
||||
quotientWriter.add((value - minValue) / gcd);
|
||||
}
|
||||
quotientWriter.finish();
|
||||
break;
|
||||
case DELTA_COMPRESSED:
|
||||
final long minDelta = delta < 0 ? 0 : minValue;
|
||||
meta.writeLong(minDelta);
|
||||
meta.writeVInt(deltaBitsRequired);
|
||||
final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
|
||||
for (Number nv : values) {
|
||||
long v = nv == null ? 0 : nv.longValue();
|
||||
writer.add(v - minDelta);
|
||||
}
|
||||
writer.finish();
|
||||
break;
|
||||
case TABLE_COMPRESSED:
|
||||
final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
|
||||
Arrays.sort(decode);
|
||||
final HashMap<Long,Integer> encode = new HashMap<>();
|
||||
meta.writeVInt(decode.length);
|
||||
for (int i = 0; i < decode.length; i++) {
|
||||
meta.writeLong(decode[i]);
|
||||
encode.put(decode[i], i);
|
||||
}
|
||||
meta.writeVInt(tableBitsRequired);
|
||||
final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
|
||||
for (Number nv : values) {
|
||||
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
|
||||
}
|
||||
ordsWriter.finish();
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
meta.writeLong(data.getFilePointer());
|
||||
}
|
||||
|
||||
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
|
||||
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
|
||||
void writeMissingBitset(Iterable<?> values) throws IOException {
|
||||
byte bits = 0;
|
||||
int count = 0;
|
||||
for (Object v : values) {
|
||||
if (count == 8) {
|
||||
data.writeByte(bits);
|
||||
count = 0;
|
||||
bits = 0;
|
||||
}
|
||||
if (v != null) {
|
||||
bits |= 1 << (count & 7);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
if (count > 0) {
|
||||
data.writeByte(bits);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
|
||||
// write the byte[] data
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.BINARY);
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
final long startFP = data.getFilePointer();
|
||||
long count = 0;
|
||||
long missingCount = 0;
|
||||
for(BytesRef v : values) {
|
||||
final int length;
|
||||
if (v == null) {
|
||||
length = 0;
|
||||
missingCount++;
|
||||
} else {
|
||||
length = v.length;
|
||||
}
|
||||
minLength = Math.min(minLength, length);
|
||||
maxLength = Math.max(maxLength, length);
|
||||
if (v != null) {
|
||||
data.writeBytes(v.bytes, v.offset, v.length);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
|
||||
if (missingCount == 0) {
|
||||
meta.writeLong(ALL_LIVE);
|
||||
} else if (missingCount == count) {
|
||||
meta.writeLong(ALL_MISSING);
|
||||
} else {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
writeMissingBitset(values);
|
||||
}
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
meta.writeVLong(count);
|
||||
meta.writeLong(startFP);
|
||||
|
||||
// if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit)
|
||||
// otherwise, we need to record the length fields...
|
||||
if (minLength != maxLength) {
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(MONOTONIC_BLOCK_SIZE);
|
||||
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (BytesRef v : values) {
|
||||
if (v != null) {
|
||||
addr += v.length;
|
||||
}
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
/** expert: writes a value dictionary for a sorted/sortedset field */
|
||||
private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
|
||||
// first check if it's a "fixed-length" terms dict
|
||||
int minLength = Integer.MAX_VALUE;
|
||||
int maxLength = Integer.MIN_VALUE;
|
||||
long numValues = 0;
|
||||
for (BytesRef v : values) {
|
||||
minLength = Math.min(minLength, v.length);
|
||||
maxLength = Math.max(maxLength, v.length);
|
||||
numValues++;
|
||||
}
|
||||
if (minLength == maxLength) {
|
||||
// no index needed: direct addressing by mult
|
||||
addBinaryField(field, values);
|
||||
} else if (numValues < REVERSE_INTERVAL_COUNT) {
|
||||
// low cardinality: waste a few KB of ram, but can't really use fancy index etc
|
||||
addBinaryField(field, values);
|
||||
} else {
|
||||
assert numValues > 0; // we don't have to handle the empty case
|
||||
// header
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.BINARY);
|
||||
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
// now write the bytes: sharing prefixes within a block
|
||||
final long startFP = data.getFilePointer();
|
||||
// currently, we have to store the delta from expected for every 1/nth term
|
||||
// we could avoid this, but it's not much and less overall RAM than the previous approach!
|
||||
RAMOutputStream addressBuffer = new RAMOutputStream();
|
||||
MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
|
||||
// buffers up 16 terms
|
||||
RAMOutputStream bytesBuffer = new RAMOutputStream();
|
||||
// buffers up block header
|
||||
RAMOutputStream headerBuffer = new RAMOutputStream();
|
||||
BytesRefBuilder lastTerm = new BytesRefBuilder();
|
||||
lastTerm.grow(maxLength);
|
||||
long count = 0;
|
||||
int suffixDeltas[] = new int[INTERVAL_COUNT];
|
||||
for (BytesRef v : values) {
|
||||
int termPosition = (int) (count & INTERVAL_MASK);
|
||||
if (termPosition == 0) {
|
||||
termAddresses.add(data.getFilePointer() - startFP);
|
||||
// abs-encode first term
|
||||
headerBuffer.writeVInt(v.length);
|
||||
headerBuffer.writeBytes(v.bytes, v.offset, v.length);
|
||||
lastTerm.copyBytes(v);
|
||||
} else {
|
||||
// prefix-code: we only share at most 255 characters, to encode the length as a single
|
||||
// byte and have random access. Larger terms just get less compression.
|
||||
int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
|
||||
bytesBuffer.writeByte((byte) sharedPrefix);
|
||||
bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
|
||||
// we can encode one smaller, because terms are unique.
|
||||
suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
|
||||
}
|
||||
|
||||
count++;
|
||||
// flush block
|
||||
if ((count & INTERVAL_MASK) == 0) {
|
||||
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
|
||||
}
|
||||
}
|
||||
// flush trailing crap
|
||||
int leftover = (int) (count & INTERVAL_MASK);
|
||||
if (leftover > 0) {
|
||||
Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
|
||||
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
|
||||
}
|
||||
final long indexStartFP = data.getFilePointer();
|
||||
// write addresses of indexed terms
|
||||
termAddresses.finish();
|
||||
addressBuffer.writeTo(data);
|
||||
addressBuffer = null;
|
||||
termAddresses = null;
|
||||
meta.writeVInt(minLength);
|
||||
meta.writeVInt(maxLength);
|
||||
meta.writeVLong(count);
|
||||
meta.writeLong(startFP);
|
||||
meta.writeLong(indexStartFP);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(MONOTONIC_BLOCK_SIZE);
|
||||
addReverseTermIndex(field, values, maxLength);
|
||||
}
|
||||
}
|
||||
|
||||
// writes term dictionary "block"
|
||||
// first term is absolute encoded as vint length + bytes.
|
||||
// lengths of subsequent N terms are encoded as either N bytes or N shorts.
|
||||
// in the double-byte case, the first byte is indicated with -1.
|
||||
// subsequent terms are encoded as byte suffixLength + bytes.
|
||||
private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
|
||||
boolean twoByte = false;
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
if (suffixDeltas[i] > 254) {
|
||||
twoByte = true;
|
||||
}
|
||||
}
|
||||
if (twoByte) {
|
||||
headerBuffer.writeByte((byte)255);
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
headerBuffer.writeShort((short) suffixDeltas[i]);
|
||||
}
|
||||
} else {
|
||||
for (int i = 1; i < suffixDeltas.length; i++) {
|
||||
headerBuffer.writeByte((byte) suffixDeltas[i]);
|
||||
}
|
||||
}
|
||||
headerBuffer.writeTo(data);
|
||||
headerBuffer.reset();
|
||||
bytesBuffer.writeTo(data);
|
||||
bytesBuffer.reset();
|
||||
}
|
||||
|
||||
// writes reverse term index: used for binary searching a term into a range of 64 blocks
|
||||
// for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
|
||||
// terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
|
||||
private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
|
||||
long count = 0;
|
||||
BytesRefBuilder priorTerm = new BytesRefBuilder();
|
||||
priorTerm.grow(maxLength);
|
||||
BytesRef indexTerm = new BytesRef();
|
||||
long startFP = data.getFilePointer();
|
||||
PagedBytes pagedBytes = new PagedBytes(15);
|
||||
MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
|
||||
|
||||
for (BytesRef b : values) {
|
||||
int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
|
||||
if (termPosition == 0) {
|
||||
int len = StringHelper.sortKeyLength(priorTerm.get(), b);
|
||||
indexTerm.bytes = b.bytes;
|
||||
indexTerm.offset = b.offset;
|
||||
indexTerm.length = len;
|
||||
addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
|
||||
} else if (termPosition == REVERSE_INTERVAL_MASK) {
|
||||
priorTerm.copyBytes(b);
|
||||
}
|
||||
count++;
|
||||
}
|
||||
addresses.finish();
|
||||
long numBytes = pagedBytes.getPointer();
|
||||
pagedBytes.freeze(true);
|
||||
PagedBytesDataInput in = pagedBytes.getDataInput();
|
||||
meta.writeLong(startFP);
|
||||
data.writeVLong(numBytes);
|
||||
data.copyBytes(in, numBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.SORTED);
|
||||
addTermsDict(field, values);
|
||||
addNumericField(field, docToOrd, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.SORTED_NUMERIC);
|
||||
if (isSingleValued(docToValueCount)) {
|
||||
meta.writeVInt(SORTED_SINGLE_VALUED);
|
||||
// The field is single-valued, we can encode it as NUMERIC
|
||||
addNumericField(field, singletonView(docToValueCount, values, null));
|
||||
} else {
|
||||
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
|
||||
if (uniqueValueSets != null) {
|
||||
meta.writeVInt(SORTED_SET_TABLE);
|
||||
|
||||
// write the set_id -> values mapping
|
||||
writeDictionary(uniqueValueSets);
|
||||
|
||||
// write the doc -> set_id as a numeric field
|
||||
addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), false);
|
||||
} else {
|
||||
meta.writeVInt(SORTED_WITH_ADDRESSES);
|
||||
// write the stream of values as a numeric field
|
||||
addNumericField(field, values, true);
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addAddresses(field, docToValueCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.SORTED_SET);
|
||||
|
||||
if (isSingleValued(docToOrdCount)) {
|
||||
meta.writeVInt(SORTED_SINGLE_VALUED);
|
||||
// The field is single-valued, we can encode it as SORTED
|
||||
addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
|
||||
} else {
|
||||
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
|
||||
if (uniqueValueSets != null) {
|
||||
meta.writeVInt(SORTED_SET_TABLE);
|
||||
|
||||
// write the set_id -> ords mapping
|
||||
writeDictionary(uniqueValueSets);
|
||||
|
||||
// write the ord -> byte[] as a binary field
|
||||
addTermsDict(field, values);
|
||||
|
||||
// write the doc -> set_id as a numeric field
|
||||
addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), false);
|
||||
} else {
|
||||
meta.writeVInt(SORTED_WITH_ADDRESSES);
|
||||
|
||||
// write the ord -> byte[] as a binary field
|
||||
addTermsDict(field, values);
|
||||
|
||||
// write the stream of ords as a numeric field
|
||||
// NOTE: we could return an iterator that delta-encodes these within a doc
|
||||
addNumericField(field, ords, false);
|
||||
|
||||
// write the doc -> ord count as a absolute index to the stream
|
||||
addAddresses(field, docToOrdCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
|
||||
Set<LongsRef> uniqueValueSet = new HashSet<>();
|
||||
LongsRef docValues = new LongsRef(256);
|
||||
|
||||
Iterator<Number> valueCountIterator = docToValueCount.iterator();
|
||||
Iterator<Number> valueIterator = values.iterator();
|
||||
int totalDictSize = 0;
|
||||
while (valueCountIterator.hasNext()) {
|
||||
docValues.length = valueCountIterator.next().intValue();
|
||||
if (docValues.length > 256) {
|
||||
return null;
|
||||
}
|
||||
for (int i = 0; i < docValues.length; ++i) {
|
||||
docValues.longs[i] = valueIterator.next().longValue();
|
||||
}
|
||||
if (uniqueValueSet.contains(docValues)) {
|
||||
continue;
|
||||
}
|
||||
totalDictSize += docValues.length;
|
||||
if (totalDictSize > 256) {
|
||||
return null;
|
||||
}
|
||||
uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
|
||||
}
|
||||
assert valueIterator.hasNext() == false;
|
||||
return new TreeSet<>(uniqueValueSet);
|
||||
}
|
||||
|
||||
private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
|
||||
int lengthSum = 0;
|
||||
for (LongsRef longs : uniqueValueSets) {
|
||||
lengthSum += longs.length;
|
||||
}
|
||||
|
||||
meta.writeInt(lengthSum);
|
||||
for (LongsRef valueSet : uniqueValueSets) {
|
||||
for (int i = 0; i < valueSet.length; ++i) {
|
||||
meta.writeLong(valueSet.longs[valueSet.offset + i]);
|
||||
}
|
||||
}
|
||||
|
||||
meta.writeInt(uniqueValueSets.size());
|
||||
for (LongsRef valueSet : uniqueValueSets) {
|
||||
meta.writeInt(valueSet.length);
|
||||
}
|
||||
}
|
||||
|
||||
private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
|
||||
final Map<LongsRef, Integer> setIds = new HashMap<>();
|
||||
int i = 0;
|
||||
for (LongsRef set : uniqueValueSets) {
|
||||
setIds.put(set, i++);
|
||||
}
|
||||
assert i == uniqueValueSets.size();
|
||||
|
||||
return new Iterable<Number>() {
|
||||
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
final Iterator<Number> valueCountIterator = docToValueCount.iterator();
|
||||
final Iterator<Number> valueIterator = values.iterator();
|
||||
final LongsRef docValues = new LongsRef(256);
|
||||
return new Iterator<Number>() {
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return valueCountIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number next() {
|
||||
docValues.length = valueCountIterator.next().intValue();
|
||||
for (int i = 0; i < docValues.length; ++i) {
|
||||
docValues.longs[i] = valueIterator.next().longValue();
|
||||
}
|
||||
final Integer id = setIds.get(docValues);
|
||||
assert id != null;
|
||||
return id;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// writes addressing information as MONOTONIC_COMPRESSED integer
|
||||
private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
meta.writeVInt(field.number);
|
||||
meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
|
||||
meta.writeVInt(MONOTONIC_COMPRESSED);
|
||||
meta.writeLong(-1L);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
meta.writeVLong(maxDoc);
|
||||
meta.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
meta.writeVInt(MONOTONIC_BLOCK_SIZE);
|
||||
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
|
||||
long addr = 0;
|
||||
writer.add(addr);
|
||||
for (Number v : values) {
|
||||
addr += v.longValue();
|
||||
writer.add(addr);
|
||||
}
|
||||
writer.finish();
|
||||
meta.writeLong(data.getFilePointer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
CodecUtil.writeFooter(meta); // write checksum
|
||||
}
|
||||
if (data != null) {
|
||||
CodecUtil.writeFooter(data); // write checksum
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
meta = data = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,115 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Lucene 5.0 Doc values format.
|
||||
* @deprecated Only for reading old 5.0-5.3 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene50DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene50DocValuesFormat() {
|
||||
super("Lucene50");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "Lucene50DocValuesData";
|
||||
static final String DATA_EXTENSION = "dvd";
|
||||
static final String META_CODEC = "Lucene50DocValuesMetadata";
|
||||
static final String META_EXTENSION = "dvm";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_SORTEDSET_TABLE = 1;
|
||||
static final int VERSION_CURRENT = VERSION_SORTEDSET_TABLE;
|
||||
|
||||
// indicates docvalues type
|
||||
static final byte NUMERIC = 0;
|
||||
static final byte BINARY = 1;
|
||||
static final byte SORTED = 2;
|
||||
static final byte SORTED_SET = 3;
|
||||
static final byte SORTED_NUMERIC = 4;
|
||||
|
||||
// address terms in blocks of 16 terms
|
||||
static final int INTERVAL_SHIFT = 4;
|
||||
static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
|
||||
static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
|
||||
|
||||
// build reverse index from every 1024th term
|
||||
static final int REVERSE_INTERVAL_SHIFT = 10;
|
||||
static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
|
||||
static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
|
||||
|
||||
// for conversion from reverse index to block
|
||||
static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
|
||||
static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
|
||||
static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
|
||||
|
||||
/** Compressed using packed blocks of ints. */
|
||||
static final int DELTA_COMPRESSED = 0;
|
||||
/** Compressed by computing the GCD. */
|
||||
static final int GCD_COMPRESSED = 1;
|
||||
/** Compressed by giving IDs to unique values. */
|
||||
static final int TABLE_COMPRESSED = 2;
|
||||
/** Compressed with monotonically increasing values */
|
||||
static final int MONOTONIC_COMPRESSED = 3;
|
||||
/** Compressed with constant value (uses only missing bitset) */
|
||||
static final int CONST_COMPRESSED = 4;
|
||||
|
||||
/** Uncompressed binary, written directly (fixed length). */
|
||||
static final int BINARY_FIXED_UNCOMPRESSED = 0;
|
||||
/** Uncompressed binary, written directly (variable length). */
|
||||
static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
|
||||
/** Compressed binary with shared prefixes */
|
||||
static final int BINARY_PREFIX_COMPRESSED = 2;
|
||||
|
||||
/** Standard storage for sorted set values with 1 level of indirection:
|
||||
* {@code docId -> address -> ord}. */
|
||||
static final int SORTED_WITH_ADDRESSES = 0;
|
||||
/** Single-valued sorted set values, encoded as sorted values, so no level
|
||||
* of indirection: {@code docId -> ord}. */
|
||||
static final int SORTED_SINGLE_VALUED = 1;
|
||||
/** Compressed giving IDs to unique sets of values:
|
||||
* {@code docId -> setId -> ords} */
|
||||
static final int SORTED_SET_TABLE = 2;
|
||||
|
||||
/** placeholder for missing offset that means there are no missing values */
|
||||
static final int ALL_LIVE = -1;
|
||||
/** placeholder for missing offset that means all values are missing */
|
||||
static final int ALL_MISSING = -2;
|
||||
|
||||
// addressing uses 16k blocks
|
||||
static final int MONOTONIC_BLOCK_SIZE = 16384;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,62 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Lucene 5.0 Score normalization format.
|
||||
* @deprecated Only for reading old 5.0-5.2 segments
|
||||
*/
|
||||
@Deprecated
|
||||
class Lucene50NormsFormat extends NormsFormat {
|
||||
|
||||
/** Sole Constructor */
|
||||
public Lucene50NormsFormat() {}
|
||||
|
||||
@Override
|
||||
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene50NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||
}
|
||||
|
||||
static final String DATA_CODEC = "Lucene50NormsData";
|
||||
static final String DATA_EXTENSION = "nvd";
|
||||
static final String METADATA_CODEC = "Lucene50NormsMetadata";
|
||||
static final String METADATA_EXTENSION = "nvm";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static final byte DELTA_COMPRESSED = 0;
|
||||
static final byte TABLE_COMPRESSED = 1;
|
||||
static final byte CONST_COMPRESSED = 2;
|
||||
static final byte UNCOMPRESSED = 3;
|
||||
static final byte INDIRECT = 4;
|
||||
static final byte PATCHED_BITSET = 5;
|
||||
static final byte PATCHED_TABLE = 6;
|
||||
}
|
|
@ -1,481 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.CONST_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.INDIRECT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.PATCHED_BITSET;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.PATCHED_TABLE;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.SparseFixedBitSet;
|
||||
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Reader for {@link Lucene50NormsFormat}
|
||||
* @deprecated Only for reading old 5.0-5.2 segments
|
||||
*/
|
||||
@Deprecated
|
||||
final class Lucene50NormsProducer extends NormsProducer {
|
||||
// metadata maps (just file pointers and minimal stuff)
|
||||
private final Map<String,NormsEntry> norms = new HashMap<>();
|
||||
private final IndexInput data;
|
||||
|
||||
// ram instances we have already loaded
|
||||
final Map<String,Norms> instances = new HashMap<>();
|
||||
|
||||
private final AtomicLong ramBytesUsed;
|
||||
private final AtomicInteger activeCount = new AtomicInteger();
|
||||
private final int maxDoc;
|
||||
|
||||
private final boolean merging;
|
||||
|
||||
// clone for merge: when merging we don't do any instances.put()s
|
||||
Lucene50NormsProducer(Lucene50NormsProducer original) {
|
||||
assert Thread.holdsLock(original);
|
||||
norms.putAll(original.norms);
|
||||
data = original.data.clone();
|
||||
instances.putAll(original.instances);
|
||||
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||
activeCount.set(original.activeCount.get());
|
||||
maxDoc = original.maxDoc;
|
||||
merging = true;
|
||||
}
|
||||
|
||||
Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
merging = false;
|
||||
maxDoc = state.segmentInfo.maxDoc();
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||
int version = -1;
|
||||
|
||||
// read in the entries from the metadata file.
|
||||
try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
|
||||
Throwable priorE = null;
|
||||
try {
|
||||
version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
readFields(in, state.fieldInfos);
|
||||
} catch (Throwable exception) {
|
||||
priorE = exception;
|
||||
} finally {
|
||||
CodecUtil.checkFooter(in, priorE);
|
||||
}
|
||||
}
|
||||
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
this.data = state.directory.openInput(dataName, state.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
if (version != version2) {
|
||||
throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
|
||||
}
|
||||
|
||||
// NOTE: data file is too costly to verify checksum against all the bytes on open,
|
||||
// but for now we at least verify proper structure of the checksum footer: which looks
|
||||
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
|
||||
// such as file truncation.
|
||||
CodecUtil.retrieveChecksum(data);
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this.data);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||
int fieldNumber = meta.readVInt();
|
||||
while (fieldNumber != -1) {
|
||||
FieldInfo info = infos.fieldInfo(fieldNumber);
|
||||
if (info == null) {
|
||||
throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
|
||||
} else if (!info.hasNorms()) {
|
||||
throw new CorruptIndexException("Invalid field: " + info.name, meta);
|
||||
}
|
||||
NormsEntry entry = readEntry(info, meta);
|
||||
norms.put(info.name, entry);
|
||||
fieldNumber = meta.readVInt();
|
||||
}
|
||||
}
|
||||
|
||||
private NormsEntry readEntry(FieldInfo info, IndexInput meta) throws IOException {
|
||||
NormsEntry entry = new NormsEntry();
|
||||
entry.count = meta.readVInt();
|
||||
entry.format = meta.readByte();
|
||||
entry.offset = meta.readLong();
|
||||
switch(entry.format) {
|
||||
case CONST_COMPRESSED:
|
||||
case UNCOMPRESSED:
|
||||
case TABLE_COMPRESSED:
|
||||
case DELTA_COMPRESSED:
|
||||
break;
|
||||
case PATCHED_BITSET:
|
||||
case PATCHED_TABLE:
|
||||
case INDIRECT:
|
||||
if (meta.readVInt() != info.number) {
|
||||
throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
|
||||
}
|
||||
entry.nested = readEntry(info, meta);
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("Unknown format: " + entry.format, meta);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
|
||||
Norms instance = instances.get(field.name);
|
||||
if (instance == null) {
|
||||
instance = loadNorms(norms.get(field.name));
|
||||
if (!merging) {
|
||||
instances.put(field.name, instance);
|
||||
activeCount.incrementAndGet();
|
||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ramBytesUsed.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", instances);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
CodecUtil.checksumEntireFile(data);
|
||||
}
|
||||
|
||||
private Norms loadNorms(NormsEntry entry) throws IOException {
|
||||
switch(entry.format) {
|
||||
case CONST_COMPRESSED: {
|
||||
final long v = entry.offset;
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 8;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "constant";
|
||||
}
|
||||
};
|
||||
}
|
||||
case UNCOMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
final byte bytes[] = new byte[entry.count];
|
||||
data.readBytes(bytes, 0, bytes.length);
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return bytes[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "byte array";
|
||||
}
|
||||
};
|
||||
}
|
||||
case DELTA_COMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, entry.count, false);
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return reader.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return reader.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(reader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delta compressed";
|
||||
}
|
||||
};
|
||||
}
|
||||
case TABLE_COMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
int packedIntsVersion = data.readVInt();
|
||||
final int formatID = data.readVInt();
|
||||
final int bitsPerValue = data.readVInt();
|
||||
|
||||
if (bitsPerValue != 1 && bitsPerValue != 2 && bitsPerValue != 4) {
|
||||
throw new CorruptIndexException("TABLE_COMPRESSED only supports bpv=1, bpv=2 and bpv=4, got=" + bitsPerValue, data);
|
||||
}
|
||||
int size = 1 << bitsPerValue;
|
||||
final byte decode[] = new byte[size];
|
||||
final int ordsSize = data.readVInt();
|
||||
for (int i = 0; i < ordsSize; ++i) {
|
||||
decode[i] = data.readByte();
|
||||
}
|
||||
for (int i = ordsSize; i < size; ++i) {
|
||||
decode[i] = 0;
|
||||
}
|
||||
|
||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return decode[(int)ordsReader.get(docID)];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(ordsReader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "table compressed";
|
||||
}
|
||||
};
|
||||
}
|
||||
case INDIRECT: {
|
||||
data.seek(entry.offset);
|
||||
final long common = data.readLong();
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
|
||||
final Norms nestedInstance = loadNorms(entry.nested);
|
||||
final int upperBound = entry.count-1;
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
int low = 0;
|
||||
int high = upperBound;
|
||||
while (low <= high) {
|
||||
int mid = (low + high) >>> 1;
|
||||
long doc = live.get(mid);
|
||||
|
||||
if (doc < docID) {
|
||||
low = mid + 1;
|
||||
} else if (doc > docID) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return nestedInstance.get(mid);
|
||||
}
|
||||
}
|
||||
return common;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return live.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", live));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "indirect";
|
||||
}
|
||||
};
|
||||
}
|
||||
case PATCHED_BITSET: {
|
||||
data.seek(entry.offset);
|
||||
final long common = data.readLong();
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, true);
|
||||
final SparseFixedBitSet set = new SparseFixedBitSet(maxDoc);
|
||||
for (int i = 0; i < live.size(); i++) {
|
||||
int doc = (int) live.get(i);
|
||||
set.set(doc);
|
||||
}
|
||||
Norms nestedInstance = loadNorms(entry.nested);
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
if (set.get(docID)) {
|
||||
return nestedInstance.get(docID);
|
||||
} else {
|
||||
return common;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return set.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", set));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "patched bitset";
|
||||
}
|
||||
};
|
||||
}
|
||||
case PATCHED_TABLE: {
|
||||
data.seek(entry.offset);
|
||||
int packedIntsVersion = data.readVInt();
|
||||
final int formatID = data.readVInt();
|
||||
final int bitsPerValue = data.readVInt();
|
||||
|
||||
if (bitsPerValue != 2 && bitsPerValue != 4) {
|
||||
throw new CorruptIndexException("PATCHED_TABLE only supports bpv=2 and bpv=4, got=" + bitsPerValue, data);
|
||||
}
|
||||
final int size = 1 << bitsPerValue;
|
||||
final int ordsSize = data.readVInt();
|
||||
final byte decode[] = new byte[ordsSize];
|
||||
assert ordsSize + 1 == size;
|
||||
for (int i = 0; i < ordsSize; ++i) {
|
||||
decode[i] = data.readByte();
|
||||
}
|
||||
|
||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
|
||||
final Norms nestedInstance = loadNorms(entry.nested);
|
||||
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
int ord = (int)ordsReader.get(docID);
|
||||
try {
|
||||
// doing a try/catch here eliminates a seemingly unavoidable branch in hotspot...
|
||||
return decode[ord];
|
||||
} catch (IndexOutOfBoundsException e) {
|
||||
return nestedInstance.get(docID);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("common", ordsReader));
|
||||
children.add(Accountables.namedAccountable("uncommon", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "patched table";
|
||||
}
|
||||
};
|
||||
}
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
data.close();
|
||||
}
|
||||
|
||||
static class NormsEntry {
|
||||
byte format;
|
||||
long offset;
|
||||
int count;
|
||||
NormsEntry nested;
|
||||
}
|
||||
|
||||
static abstract class Norms extends NumericDocValues implements Accountable {
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized NormsProducer getMergeInstance() throws IOException {
|
||||
return new Lucene50NormsProducer(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "(fields=" + norms.size() + ",active=" + activeCount.get() + ")";
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene50;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -50,7 +49,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
|
|||
Throwable priorE = null;
|
||||
SegmentInfo si = null;
|
||||
try {
|
||||
int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
|
||||
CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
|
||||
Lucene50SegmentInfoFormat.VERSION_START,
|
||||
Lucene50SegmentInfoFormat.VERSION_CURRENT,
|
||||
segmentID, "");
|
||||
|
@ -62,19 +61,9 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
|
|||
}
|
||||
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
|
||||
|
||||
final Map<String,String> diagnostics;
|
||||
final Set<String> files;
|
||||
final Map<String,String> attributes;
|
||||
|
||||
if (format >= VERSION_SAFE_MAPS) {
|
||||
diagnostics = input.readMapOfStrings();
|
||||
files = input.readSetOfStrings();
|
||||
attributes = input.readMapOfStrings();
|
||||
} else {
|
||||
diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
files = Collections.unmodifiableSet(input.readStringSet());
|
||||
attributes = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
}
|
||||
final Map<String,String> diagnostics = input.readMapOfStrings();
|
||||
final Set<String> files = input.readSetOfStrings();
|
||||
final Map<String,String> attributes = input.readMapOfStrings();
|
||||
|
||||
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
|
||||
si.setFiles(files);
|
||||
|
@ -95,7 +84,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
|
|||
/** File extension used to store {@link SegmentInfo}. */
|
||||
public final static String SI_EXTENSION = "si";
|
||||
static final String CODEC_NAME = "Lucene50SegmentInfo";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_SAFE_MAPS = 1;
|
||||
static final int VERSION_START = VERSION_SAFE_MAPS;
|
||||
static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
|
||||
}
|
||||
|
|
|
@ -1,176 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene53;
|
||||
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 5.3 index format, with configurable per-field postings
|
||||
* and docvalues formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene53 package documentation for file format details.
|
||||
* @deprecated Only for reading old 5.3 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene53Codec extends Codec {
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
|
||||
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene53Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene53Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final StoredFieldsFormat storedFieldsFormat;
|
||||
|
||||
/**
|
||||
* Instantiates a new codec.
|
||||
*/
|
||||
public Lucene53Codec() {
|
||||
this(Mode.BEST_SPEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instantiates a new codec, specifying the stored fields compression
|
||||
* mode to use.
|
||||
* @param mode stored fields compression mode to use for newly
|
||||
* flushed/merged segments.
|
||||
*/
|
||||
public Lucene53Codec(Mode mode) {
|
||||
super("Lucene53");
|
||||
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final CompoundFormat compoundFormat() {
|
||||
return compoundFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
/** Returns the docvalues format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene53NormsFormat();
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -1,25 +0,0 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Lucene 5.3 file format.
|
||||
</body>
|
||||
</html>
|
|
@ -1,178 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene54;
|
||||
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 5.4 index format, with configurable per-field postings
|
||||
* and docvalues formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene54 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for 5.x back compat
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene54Codec extends Codec {
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
|
||||
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene54Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene54Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final StoredFieldsFormat storedFieldsFormat;
|
||||
|
||||
/**
|
||||
* Instantiates a new codec.
|
||||
*/
|
||||
public Lucene54Codec() {
|
||||
this(Mode.BEST_SPEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instantiates a new codec, specifying the stored fields compression
|
||||
* mode to use.
|
||||
* @param mode stored fields compression mode to use for newly
|
||||
* flushed/merged segments.
|
||||
*/
|
||||
public Lucene54Codec(Mode mode) {
|
||||
super("Lucene54");
|
||||
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final CompoundFormat compoundFormat() {
|
||||
return compoundFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene50".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
/** Returns the docvalues format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene54".
|
||||
* <p>
|
||||
* <b>WARNING:</b> if you subclass, you are responsible for index
|
||||
* backwards compatibility: future version of Lucene are only
|
||||
* guaranteed to be able to read the default implementation.
|
||||
*/
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return defaultDVFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
|
||||
|
||||
private final NormsFormat normsFormat = new Lucene53NormsFormat();
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
}
|
|
@ -1,25 +0,0 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Lucene 5.4 file format.
|
||||
</body>
|
||||
</html>
|
|
@ -13,7 +13,4 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene50.Lucene50Codec
|
||||
org.apache.lucene.codecs.lucene53.Lucene53Codec
|
||||
org.apache.lucene.codecs.lucene54.Lucene54Codec
|
||||
org.apache.lucene.codecs.lucene60.Lucene60Codec
|
||||
|
|
|
@ -13,4 +13,3 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat
|
||||
|
|
|
@ -1,403 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.FilterIterator;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||
|
||||
/**
|
||||
* Writer for {@link Lucene50NormsFormat}
|
||||
* @deprecated Only for testing old 5.0-5.2 segments
|
||||
*/
|
||||
@Deprecated
|
||||
final class Lucene50NormsConsumer extends NormsConsumer {
|
||||
static final int BLOCK_SIZE = 1 << 14;
|
||||
|
||||
// threshold for indirect encoding, computed as 1 - 1/log2(maxint)
|
||||
// norms are only read for matching postings... so this is the threshold
|
||||
// where n log n operations < maxdoc (e.g. it performs similar to other fields)
|
||||
static final float INDIRECT_THRESHOLD = 1 - 1 / 31F;
|
||||
|
||||
IndexOutput data, meta;
|
||||
|
||||
Lucene50NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||
data = state.directory.createOutput(dataName, state.context);
|
||||
CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||
meta = state.directory.createOutput(metaName, state.context);
|
||||
CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// we explicitly use only certain bits per value and a specified format, so we statically check this will work
|
||||
static {
|
||||
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(1);
|
||||
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(2);
|
||||
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(4);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||
writeNormsField(field, values, 0);
|
||||
}
|
||||
|
||||
private void writeNormsField(FieldInfo field, Iterable<Number> values, int level) throws IOException {
|
||||
assert level <= 1; // we only "recurse" once in the indirect case
|
||||
meta.writeVInt(field.number);
|
||||
NormMap uniqueValues = new NormMap();
|
||||
int count = 0;
|
||||
|
||||
for (Number nv : values) {
|
||||
if (nv == null) {
|
||||
throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
|
||||
}
|
||||
final long v = nv.longValue();
|
||||
|
||||
if (uniqueValues != null) {
|
||||
if (v >= Byte.MIN_VALUE && v <= Byte.MAX_VALUE) {
|
||||
if (uniqueValues.add((byte) v)) {
|
||||
if (uniqueValues.size > 256) {
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// anything outside an 8 bit float comes from a custom scorer, which is an extreme edge case
|
||||
uniqueValues = null;
|
||||
}
|
||||
}
|
||||
count++;
|
||||
}
|
||||
|
||||
if (uniqueValues == null) {
|
||||
addDeltaCompressed(values, count);
|
||||
} else if (uniqueValues.size == 1) {
|
||||
// 0 bpv
|
||||
addConstant(uniqueValues.values[0]);
|
||||
} else {
|
||||
// small number of unique values: this is the typical case
|
||||
uniqueValues.optimizeOrdinals();
|
||||
|
||||
int numCommonValues = -1;
|
||||
int commonValuesCount = 0;
|
||||
if (level == 0 && count > 256) {
|
||||
float threshold_count = count * INDIRECT_THRESHOLD;
|
||||
if (uniqueValues.freqs[0] > threshold_count) {
|
||||
numCommonValues = 1;
|
||||
} else if ((commonValuesCount = sum(uniqueValues.freqs, 0, 3)) > threshold_count && uniqueValues.size > 4) {
|
||||
numCommonValues = 3;
|
||||
} else if ((commonValuesCount = sum(uniqueValues.freqs, 0, 15)) > threshold_count && uniqueValues.size > 16) {
|
||||
numCommonValues = 15;
|
||||
}
|
||||
}
|
||||
|
||||
if (numCommonValues == -1) {
|
||||
// no pattern in values, just find the most efficient way to pack the values
|
||||
FormatAndBits compression = fastestFormatAndBits(uniqueValues.size - 1);
|
||||
if (compression.bitsPerValue == 8) {
|
||||
addUncompressed(values, count);
|
||||
} else {
|
||||
addTableCompressed(values, compression, count, uniqueValues);
|
||||
}
|
||||
|
||||
} else if (numCommonValues == 1) {
|
||||
byte commonValue = uniqueValues.values[0];
|
||||
if (commonValue == 0) {
|
||||
// if the common value is missing, don't waste RAM on a bitset, since we won't be searching those docs
|
||||
addIndirect(field, values, count, uniqueValues, 0);
|
||||
} else {
|
||||
// otherwise, write a sparse bitset, where 1 indicates 'uncommon value'.
|
||||
addPatchedBitset(field, values, count, uniqueValues);
|
||||
}
|
||||
} else {
|
||||
addPatchedTable(field, values, numCommonValues, commonValuesCount, count, uniqueValues);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int sum(int[] freqs, int start, int end) {
|
||||
int accum = 0;
|
||||
for (int i = start; i < end; ++i) {
|
||||
accum += freqs[i];
|
||||
}
|
||||
return accum;
|
||||
}
|
||||
|
||||
private FormatAndBits fastestFormatAndBits(int max) {
|
||||
// we only use bpv=1,2,4,8
|
||||
PackedInts.Format format = PackedInts.Format.PACKED_SINGLE_BLOCK;
|
||||
int bitsPerValue = PackedInts.bitsRequired(max);
|
||||
if (bitsPerValue == 3) {
|
||||
bitsPerValue = 4;
|
||||
} else if (bitsPerValue > 4) {
|
||||
bitsPerValue = 8;
|
||||
}
|
||||
return new FormatAndBits(format, bitsPerValue);
|
||||
}
|
||||
|
||||
private void addConstant(byte constant) throws IOException {
|
||||
meta.writeVInt(0);
|
||||
meta.writeByte(Lucene50NormsFormat.CONST_COMPRESSED);
|
||||
meta.writeLong(constant);
|
||||
}
|
||||
|
||||
private void addUncompressed(Iterable<Number> values, int count) throws IOException {
|
||||
meta.writeVInt(count);
|
||||
meta.writeByte(Lucene50NormsFormat.UNCOMPRESSED); // uncompressed byte[]
|
||||
meta.writeLong(data.getFilePointer());
|
||||
for (Number nv : values) {
|
||||
data.writeByte(nv.byteValue());
|
||||
}
|
||||
}
|
||||
|
||||
private void addTableCompressed(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues) throws IOException {
|
||||
meta.writeVInt(count);
|
||||
meta.writeByte(Lucene50NormsFormat.TABLE_COMPRESSED); // table-compressed
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
writeTable(values, compression, count, uniqueValues, uniqueValues.size);
|
||||
}
|
||||
|
||||
private void writeTable(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues, int numOrds) throws IOException {
|
||||
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(compression.format.getId());
|
||||
data.writeVInt(compression.bitsPerValue);
|
||||
|
||||
data.writeVInt(numOrds);
|
||||
for (int i = 0; i < numOrds; i++) {
|
||||
data.writeByte(uniqueValues.values[i]);
|
||||
}
|
||||
|
||||
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||
for(Number nv : values) {
|
||||
int ord = uniqueValues.ord(nv.byteValue());
|
||||
if (ord < numOrds) {
|
||||
writer.add(ord);
|
||||
} else {
|
||||
writer.add(numOrds); // collapses all ords >= numOrds into a single value
|
||||
}
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
||||
private void addDeltaCompressed(Iterable<Number> values, int count) throws IOException {
|
||||
meta.writeVInt(count);
|
||||
meta.writeByte(Lucene50NormsFormat.DELTA_COMPRESSED); // delta-compressed
|
||||
meta.writeLong(data.getFilePointer());
|
||||
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||
for (Number nv : values) {
|
||||
writer.add(nv.longValue());
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
||||
// encodes only uncommon values in a sparse bitset
|
||||
// access is constant time, and the common case is predictable
|
||||
// exceptions nest either to CONST (if there are only 2 values), or INDIRECT (if there are > 2 values)
|
||||
private void addPatchedBitset(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
|
||||
int commonCount = uniqueValues.freqs[0];
|
||||
|
||||
meta.writeVInt(count - commonCount);
|
||||
meta.writeByte(Lucene50NormsFormat.PATCHED_BITSET);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
// write docs with value
|
||||
writeDocsWithValue(values, uniqueValues, 0);
|
||||
|
||||
// write exceptions: only two cases make sense
|
||||
// bpv = 1 (folded into sparse bitset already)
|
||||
// bpv > 1 (add indirect exception table)
|
||||
meta.writeVInt(field.number);
|
||||
if (uniqueValues.size == 2) {
|
||||
// special case: implicit in bitset
|
||||
addConstant(uniqueValues.values[1]);
|
||||
} else {
|
||||
// exception table
|
||||
addIndirect(field, values, count, uniqueValues, 0);
|
||||
}
|
||||
}
|
||||
|
||||
// encodes common values in a table, and the rest of the values as exceptions using INDIRECT.
|
||||
// the exceptions should not be accessed very often, since the values are uncommon
|
||||
private void addPatchedTable(FieldInfo field, final Iterable<Number> values, final int numCommonValues, int commonValuesCount, int count, final NormMap uniqueValues) throws IOException {
|
||||
meta.writeVInt(count);
|
||||
meta.writeByte(Lucene50NormsFormat.PATCHED_TABLE);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
assert numCommonValues == 3 || numCommonValues == 15;
|
||||
FormatAndBits compression = fastestFormatAndBits(numCommonValues);
|
||||
|
||||
writeTable(values, compression, count, uniqueValues, numCommonValues);
|
||||
|
||||
meta.writeVInt(field.number);
|
||||
addIndirect(field, values, count - commonValuesCount, uniqueValues, numCommonValues);
|
||||
}
|
||||
|
||||
// encodes values as sparse array: keys[] and values[]
|
||||
// access is log(N) where N = keys.length (slow!)
|
||||
// so this is only appropriate as an exception table for patched, or when common value is 0 (wont be accessed by searching)
|
||||
private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, final NormMap uniqueValues, final int minOrd) throws IOException {
|
||||
int commonCount = uniqueValues.freqs[minOrd];
|
||||
|
||||
meta.writeVInt(count - commonCount);
|
||||
meta.writeByte(Lucene50NormsFormat.INDIRECT);
|
||||
meta.writeLong(data.getFilePointer());
|
||||
|
||||
// write docs with value
|
||||
writeDocsWithValue(values, uniqueValues, minOrd);
|
||||
|
||||
// write actual values
|
||||
writeNormsField(field, new Iterable<Number>() {
|
||||
@Override
|
||||
public Iterator<Number> iterator() {
|
||||
return new FilterIterator<Number, Number>(values.iterator()) {
|
||||
@Override
|
||||
protected boolean predicateFunction(Number value) {
|
||||
return uniqueValues.ord(value.byteValue()) > minOrd;
|
||||
}
|
||||
};
|
||||
}
|
||||
}, 1);
|
||||
}
|
||||
|
||||
private void writeDocsWithValue(final Iterable<Number> values, NormMap uniqueValues, int minOrd) throws IOException {
|
||||
data.writeLong(uniqueValues.values[minOrd]);
|
||||
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||
data.writeVInt(BLOCK_SIZE);
|
||||
|
||||
// write docs with value
|
||||
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||
int doc = 0;
|
||||
for (Number n : values) {
|
||||
int ord = uniqueValues.ord(n.byteValue());
|
||||
if (ord > minOrd) {
|
||||
writer.add(doc);
|
||||
}
|
||||
doc++;
|
||||
}
|
||||
writer.finish();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
if (meta != null) {
|
||||
meta.writeVInt(-1); // write EOF marker
|
||||
CodecUtil.writeFooter(meta); // write checksum
|
||||
}
|
||||
if (data != null) {
|
||||
CodecUtil.writeFooter(data); // write checksum
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(data, meta);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(data, meta);
|
||||
}
|
||||
meta = data = null;
|
||||
}
|
||||
}
|
||||
|
||||
// specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range.
|
||||
static class NormMap {
|
||||
// we use short: at most we will add 257 values to this map before it's rejected as too big above.
|
||||
private final short[] ords = new short[256];
|
||||
final int[] freqs = new int[257];
|
||||
final byte[] values = new byte[257];
|
||||
int size;
|
||||
|
||||
{
|
||||
Arrays.fill(ords, (short)-1);
|
||||
}
|
||||
|
||||
// adds an item to the mapping. returns true if actually added
|
||||
public boolean add(byte l) {
|
||||
assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy
|
||||
int index = (int)l + 128;
|
||||
short previous = ords[index];
|
||||
if (previous < 0) {
|
||||
short slot = (short)size;
|
||||
ords[index] = slot;
|
||||
freqs[slot]++;
|
||||
values[slot] = l;
|
||||
size++;
|
||||
return true;
|
||||
} else {
|
||||
freqs[previous]++;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public int ord(byte value) {
|
||||
return ords[(int)value + 128];
|
||||
}
|
||||
|
||||
// reassign ordinals so higher frequencies have lower ordinals
|
||||
public void optimizeOrdinals() {
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
return freqs[j] - freqs[i]; // sort descending
|
||||
}
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
// swap ordinal i with ordinal j
|
||||
ords[(int)values[i] + 128] = (short)j;
|
||||
ords[(int)values[j] + 128] = (short)i;
|
||||
|
||||
int tmpFreq = freqs[i];
|
||||
byte tmpValue = values[i];
|
||||
freqs[i] = freqs[j];
|
||||
values[i] = values[j];
|
||||
freqs[j] = tmpFreq;
|
||||
values[j] = tmpValue;
|
||||
}
|
||||
}.sort(0, size);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,41 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
|
||||
/**
|
||||
* Codec for testing 5.0 index format
|
||||
* @deprecated Only for testing old 5.0-5.2 segments
|
||||
*/
|
||||
@Deprecated
|
||||
final class Lucene50RWCodec extends Lucene50Codec {
|
||||
private final NormsFormat normsFormat = new Lucene50RWNormsFormat();
|
||||
private final SegmentInfoFormat segmentInfoFormat = new Lucene50RWSegmentInfoFormat();
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfoFormat segmentInfoFormat() {
|
||||
return segmentInfoFormat;
|
||||
}
|
||||
}
|
|
@ -1,36 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Read-write version of 5.0 norms format for testing
|
||||
* @deprecated for test purposes only
|
||||
*/
|
||||
@Deprecated
|
||||
final class Lucene50RWNormsFormat extends Lucene50NormsFormat {
|
||||
|
||||
@Override
|
||||
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||
return new Lucene50NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene50;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -50,7 +49,7 @@ public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat {
|
|||
Throwable priorE = null;
|
||||
SegmentInfo si = null;
|
||||
try {
|
||||
int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
|
||||
CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
|
||||
Lucene50SegmentInfoFormat.VERSION_START,
|
||||
Lucene50SegmentInfoFormat.VERSION_CURRENT,
|
||||
segmentID, "");
|
||||
|
@ -62,19 +61,9 @@ public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat {
|
|||
}
|
||||
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
|
||||
|
||||
final Map<String,String> diagnostics;
|
||||
final Set<String> files;
|
||||
final Map<String,String> attributes;
|
||||
|
||||
if (format >= VERSION_SAFE_MAPS) {
|
||||
diagnostics = input.readMapOfStrings();
|
||||
files = input.readSetOfStrings();
|
||||
attributes = input.readMapOfStrings();
|
||||
} else {
|
||||
diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
files = Collections.unmodifiableSet(input.readStringSet());
|
||||
attributes = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
}
|
||||
final Map<String,String> diagnostics = input.readMapOfStrings();
|
||||
final Set<String> files = input.readSetOfStrings();
|
||||
final Map<String,String> attributes = input.readMapOfStrings();
|
||||
|
||||
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
|
||||
si.setFiles(files);
|
||||
|
@ -130,7 +119,7 @@ public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat {
|
|||
/** File extension used to store {@link SegmentInfo}. */
|
||||
public final static String SI_EXTENSION = "si";
|
||||
static final String CODEC_NAME = "Lucene50SegmentInfo";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_SAFE_MAPS = 1;
|
||||
static final int VERSION_START = VERSION_SAFE_MAPS;
|
||||
static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
|
||||
}
|
||||
|
|
|
@ -1,281 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests Lucene50DocValuesFormat
|
||||
*/
|
||||
public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
|
||||
// for now just keep them here, as we want to test this for this format.
|
||||
|
||||
@Slow
|
||||
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testSortedVariableLengthBigVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedVsStoredFields(atLeast(300), 1, 32766);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testSortedVariableLengthManyVsStoredFields() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testTermsEnumFixedWidth() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
|
||||
}
|
||||
}
|
||||
|
||||
@Slow
|
||||
public void testTermsEnumVariableWidth() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
@Nightly
|
||||
public void testTermsEnumRandomMany() throws Exception {
|
||||
int numIterations = atLeast(1);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: try to refactor this and some termsenum tests into the base class.
|
||||
// to do this we need to fix the test class to get a DVF not a Codec so we can setup
|
||||
// the postings format correctly.
|
||||
private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
|
||||
Directory dir = newFSDirectory(createTempDir());
|
||||
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
conf.setMergeScheduler(new SerialMergeScheduler());
|
||||
// set to duel against a codec which has ordinals:
|
||||
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
|
||||
final DocValuesFormat dv = new Lucene50DocValuesFormat();
|
||||
conf.setCodec(new AssertingCodec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return pf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return dv;
|
||||
}
|
||||
});
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
|
||||
|
||||
// index some docs
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
|
||||
doc.add(idField);
|
||||
final int length = TestUtil.nextInt(random(), minLength, maxLength);
|
||||
int numValues = random().nextInt(17);
|
||||
// create a random list of strings
|
||||
List<String> values = new ArrayList<>();
|
||||
for (int v = 0; v < numValues; v++) {
|
||||
values.add(TestUtil.randomSimpleString(random(), minLength, length));
|
||||
}
|
||||
|
||||
// add in any order to the indexed field
|
||||
ArrayList<String> unordered = new ArrayList<>(values);
|
||||
Collections.shuffle(unordered, random());
|
||||
for (String v : values) {
|
||||
doc.add(newStringField("indexed", v, Field.Store.NO));
|
||||
}
|
||||
|
||||
// add in any order to the dv field
|
||||
ArrayList<String> unordered2 = new ArrayList<>(values);
|
||||
Collections.shuffle(unordered2, random());
|
||||
for (String v : unordered2) {
|
||||
doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
|
||||
}
|
||||
|
||||
writer.addDocument(doc);
|
||||
if (random().nextInt(31) == 0) {
|
||||
writer.commit();
|
||||
}
|
||||
}
|
||||
|
||||
// delete some docs
|
||||
int numDeletions = random().nextInt(numDocs/10);
|
||||
for (int i = 0; i < numDeletions; i++) {
|
||||
int id = random().nextInt(numDocs);
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(id)));
|
||||
}
|
||||
|
||||
// compare per-segment
|
||||
DirectoryReader ir = writer.getReader();
|
||||
for (LeafReaderContext context : ir.leaves()) {
|
||||
LeafReader r = context.reader();
|
||||
Terms terms = r.terms("indexed");
|
||||
if (terms != null) {
|
||||
SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
|
||||
assertEquals(terms.size(), ssdv.getValueCount());
|
||||
TermsEnum expected = terms.iterator();
|
||||
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
|
||||
assertEquals(terms.size(), expected, actual);
|
||||
|
||||
doTestSortedSetEnumAdvanceIndependently(ssdv);
|
||||
}
|
||||
}
|
||||
ir.close();
|
||||
|
||||
writer.forceMerge(1);
|
||||
|
||||
// now compare again after the merge
|
||||
ir = writer.getReader();
|
||||
LeafReader ar = getOnlyLeafReader(ir);
|
||||
Terms terms = ar.terms("indexed");
|
||||
if (terms != null) {
|
||||
assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
|
||||
TermsEnum expected = terms.iterator();
|
||||
TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
|
||||
assertEquals(terms.size(), expected, actual);
|
||||
}
|
||||
ir.close();
|
||||
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
|
||||
BytesRef ref;
|
||||
|
||||
// sequential next() through all terms
|
||||
while ((ref = expected.next()) != null) {
|
||||
assertEquals(ref, actual.next());
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
assertNull(actual.next());
|
||||
|
||||
// sequential seekExact(ord) through all terms
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
expected.seekExact(i);
|
||||
actual.seekExact(i);
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
|
||||
// sequential seekExact(BytesRef) through all terms
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
expected.seekExact(i);
|
||||
assertTrue(actual.seekExact(expected.term()));
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
|
||||
// sequential seekCeil(BytesRef) through all terms
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
expected.seekExact(i);
|
||||
assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
|
||||
// random seekExact(ord)
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
|
||||
expected.seekExact(randomOrd);
|
||||
actual.seekExact(randomOrd);
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
|
||||
// random seekExact(BytesRef)
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
|
||||
expected.seekExact(randomOrd);
|
||||
actual.seekExact(expected.term());
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
|
||||
// random seekCeil(BytesRef)
|
||||
for (long i = 0; i < numOrds; i++) {
|
||||
BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
|
||||
SeekStatus expectedStatus = expected.seekCeil(target);
|
||||
assertEquals(expectedStatus, actual.seekCeil(target));
|
||||
if (expectedStatus != SeekStatus.END) {
|
||||
assertEquals(expected.ord(), actual.ord());
|
||||
assertEquals(expected.term(), actual.term());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,130 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene50;
|
||||
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.NormMap;
|
||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests Lucene50NormsFormat
|
||||
*/
|
||||
public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
|
||||
private final Codec codec = new Lucene50RWCodec();
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
// NormMap is rather complicated, doing domain encoding / tracking frequencies etc.
|
||||
// test it directly some here...
|
||||
|
||||
public void testNormMapSimple() {
|
||||
NormMap map = new NormMap();
|
||||
map.add((byte)4);
|
||||
map.add((byte) 10);
|
||||
map.add((byte) 5);
|
||||
map.add((byte)10);
|
||||
assertEquals(3, map.size);
|
||||
|
||||
// first come, first serve ord assignment
|
||||
assertEquals(0, map.ord((byte) 4));
|
||||
assertEquals(1, map.ord((byte) 10));
|
||||
assertEquals(2, map.ord((byte) 5));
|
||||
|
||||
assertEquals(4, map.values[0]);
|
||||
assertEquals(10, map.values[1]);
|
||||
assertEquals(5, map.values[2]);
|
||||
|
||||
assertEquals(1, map.freqs[0]);
|
||||
assertEquals(2, map.freqs[1]);
|
||||
assertEquals(1, map.freqs[2]);
|
||||
|
||||
// optimizing reorders the ordinals
|
||||
map.optimizeOrdinals();
|
||||
assertEquals(0, map.ord((byte)10));
|
||||
assertEquals(1, map.ord((byte)4));
|
||||
assertEquals(2, map.ord((byte)5));
|
||||
|
||||
assertEquals(10, map.values[0]);
|
||||
assertEquals(4, map.values[1]);
|
||||
assertEquals(5, map.values[2]);
|
||||
|
||||
assertEquals(2, map.freqs[0]);
|
||||
assertEquals(1, map.freqs[1]);
|
||||
assertEquals(1, map.freqs[2]);
|
||||
}
|
||||
|
||||
public void testNormMapRandom() {
|
||||
|
||||
Set<Byte> uniqueValuesSet = new HashSet<>();
|
||||
int numUniqValues = TestUtil.nextInt(random(), 1, 256);
|
||||
for (int i = 0; i < numUniqValues; i++) {
|
||||
uniqueValuesSet.add(Byte.valueOf((byte)TestUtil.nextInt(random(), Byte.MIN_VALUE, Byte.MAX_VALUE)));
|
||||
}
|
||||
Byte uniqueValues[] = uniqueValuesSet.toArray(new Byte[uniqueValuesSet.size()]);
|
||||
|
||||
Map<Byte,Integer> freqs = new HashMap<>();
|
||||
NormMap map = new NormMap();
|
||||
int numdocs = TestUtil.nextInt(random(), 1, 100000);
|
||||
for (int i = 0; i < numdocs; i++) {
|
||||
byte value = uniqueValues[random().nextInt(uniqueValues.length)];
|
||||
// now add to both expected and actual
|
||||
map.add(value);
|
||||
if (freqs.containsKey(value)) {
|
||||
freqs.put(value, freqs.get(value) + 1);
|
||||
} else {
|
||||
freqs.put(value, 1);
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(freqs.size(), map.size);
|
||||
for (Map.Entry<Byte,Integer> kv : freqs.entrySet()) {
|
||||
byte value = kv.getKey();
|
||||
int freq = kv.getValue();
|
||||
int ord = map.ord(value);
|
||||
assertEquals(freq, map.freqs[ord]);
|
||||
assertEquals(value, map.values[ord]);
|
||||
}
|
||||
|
||||
// optimizing should reorder ordinals from greatest to least frequency
|
||||
map.optimizeOrdinals();
|
||||
// recheck consistency
|
||||
assertEquals(freqs.size(), map.size);
|
||||
for (Map.Entry<Byte,Integer> kv : freqs.entrySet()) {
|
||||
byte value = kv.getKey();
|
||||
int freq = kv.getValue();
|
||||
int ord = map.ord(value);
|
||||
assertEquals(freq, map.freqs[ord]);
|
||||
assertEquals(value, map.values[ord]);
|
||||
}
|
||||
// also check descending freq
|
||||
int prevFreq = map.freqs[0];
|
||||
for (int i = 1; i < map.size; ++i) {
|
||||
assertTrue(prevFreq >= map.freqs[i]);
|
||||
prevFreq = map.freqs[i];
|
||||
}
|
||||
}
|
||||
}
|
|
@ -226,7 +226,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
"6.0.1-cfs",
|
||||
"6.0.1-nocfs",
|
||||
"6.1.0-cfs",
|
||||
"6.1.0-nocfs"
|
||||
"6.1.0-nocfs",
|
||||
"6.2.0-cfs",
|
||||
"6.2.0-nocfs"
|
||||
};
|
||||
|
||||
final String[] unsupportedNames = {
|
||||
|
|
Binary file not shown.
Binary file not shown.
|
@ -112,7 +112,7 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
|
|||
Throwable priorE = null;
|
||||
FieldInfo infos[] = null;
|
||||
try {
|
||||
int format = CodecUtil.checkIndexHeader(input, Lucene50FieldInfosFormat.CODEC_NAME,
|
||||
CodecUtil.checkIndexHeader(input, Lucene50FieldInfosFormat.CODEC_NAME,
|
||||
Lucene50FieldInfosFormat.FORMAT_START,
|
||||
Lucene50FieldInfosFormat.FORMAT_CURRENT,
|
||||
segmentInfo.getId(), segmentSuffix);
|
||||
|
@ -139,12 +139,8 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
|
|||
// DV Types are packed in one byte
|
||||
final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
|
||||
final long dvGen = input.readLong();
|
||||
Map<String,String> attributes;
|
||||
if (format >= FORMAT_SAFE_MAPS) {
|
||||
attributes = input.readMapOfStrings();
|
||||
} else {
|
||||
attributes = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
}
|
||||
Map<String,String> attributes = input.readMapOfStrings();
|
||||
|
||||
// just use the last field's map if its the same
|
||||
if (attributes.equals(lastAttributes)) {
|
||||
attributes = lastAttributes;
|
||||
|
@ -288,8 +284,8 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
|
|||
|
||||
// Codec header
|
||||
static final String CODEC_NAME = "Lucene50FieldInfos";
|
||||
static final int FORMAT_START = 0;
|
||||
static final int FORMAT_SAFE_MAPS = 1;
|
||||
static final int FORMAT_START = FORMAT_SAFE_MAPS;
|
||||
static final int FORMAT_CURRENT = FORMAT_SAFE_MAPS;
|
||||
|
||||
// Field flags
|
||||
|
|
|
@ -4685,24 +4685,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
testPoint("finishStartCommit");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the index in the named directory is
|
||||
* currently locked.
|
||||
* @param directory the directory to check for a lock
|
||||
* @throws IOException if there is a low-level IO error
|
||||
* @deprecated Use of this method can only lead to race conditions. Try
|
||||
* to actually obtain a lock instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public static boolean isLocked(Directory directory) throws IOException {
|
||||
try {
|
||||
directory.obtainLock(WRITE_LOCK_NAME).close();
|
||||
return false;
|
||||
} catch (LockObtainFailedException failed) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/** If {@link DirectoryReader#open(IndexWriter)} has
|
||||
* been called (ie, this writer is in near real-time
|
||||
* mode), then after a merge completes, this class can be
|
||||
|
|
|
@ -119,10 +119,6 @@ import org.apache.lucene.util.Version;
|
|||
*/
|
||||
public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo> {
|
||||
|
||||
/** The file format version for the segments_N codec header, since 5.0+ */
|
||||
public static final int VERSION_50 = 4;
|
||||
/** The file format version for the segments_N codec header, since 5.1+ */
|
||||
public static final int VERSION_51 = 5; // use safe maps
|
||||
/** Adds the {@link Version} that committed this segments_N file, as well as the {@link Version} of the oldest segment, since 5.3+ */
|
||||
public static final int VERSION_53 = 6;
|
||||
|
||||
|
@ -294,7 +290,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
if (magic != CodecUtil.CODEC_MAGIC) {
|
||||
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
|
||||
}
|
||||
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
|
||||
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_53, VERSION_CURRENT);
|
||||
byte id[] = new byte[StringHelper.ID_LENGTH];
|
||||
input.readBytes(id, 0, id.length);
|
||||
CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
|
||||
|
@ -351,11 +347,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
long fieldInfosGen = input.readLong();
|
||||
long dvGen = input.readLong();
|
||||
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
|
||||
if (format >= VERSION_51) {
|
||||
siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
|
||||
} else {
|
||||
siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
|
||||
}
|
||||
final Map<Integer,Set<String>> dvUpdateFiles;
|
||||
final int numDVFields = input.readInt();
|
||||
if (numDVFields == 0) {
|
||||
|
@ -363,11 +355,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
} else {
|
||||
Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
|
||||
for (int i = 0; i < numDVFields; i++) {
|
||||
if (format >= VERSION_51) {
|
||||
map.put(input.readInt(), input.readSetOfStrings());
|
||||
} else {
|
||||
map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
|
||||
}
|
||||
}
|
||||
dvUpdateFiles = Collections.unmodifiableMap(map);
|
||||
}
|
||||
|
@ -381,11 +369,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
}
|
||||
}
|
||||
|
||||
if (format >= VERSION_51) {
|
||||
infos.userData = input.readMapOfStrings();
|
||||
} else {
|
||||
infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
|
||||
}
|
||||
|
||||
CodecUtil.checkFooter(input);
|
||||
|
||||
|
|
|
@ -272,6 +272,17 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
}
|
||||
|
||||
// Check whether some clauses are both required and excluded
|
||||
if (clauseSets.get(Occur.MUST_NOT).size() > 0) {
|
||||
final Set<Query> reqAndExclQueries = new HashSet<Query>(clauseSets.get(Occur.FILTER));
|
||||
reqAndExclQueries.addAll(clauseSets.get(Occur.MUST));
|
||||
reqAndExclQueries.retainAll(clauseSets.get(Occur.MUST_NOT));
|
||||
|
||||
if (reqAndExclQueries.isEmpty() == false) {
|
||||
return new MatchNoDocsQuery("FILTER or MUST clause also in MUST_NOT");
|
||||
}
|
||||
}
|
||||
|
||||
// remove FILTER clauses that are also MUST clauses
|
||||
// or that match all documents
|
||||
if (clauseSets.get(Occur.MUST).size() > 0 && clauseSets.get(Occur.FILTER).size() > 0) {
|
||||
|
@ -293,6 +304,35 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
}
|
||||
|
||||
// convert FILTER clauses that are also SHOULD clauses to MUST clauses
|
||||
if (clauseSets.get(Occur.SHOULD).size() > 0 && clauseSets.get(Occur.FILTER).size() > 0) {
|
||||
final Collection<Query> filters = clauseSets.get(Occur.FILTER);
|
||||
final Collection<Query> shoulds = clauseSets.get(Occur.SHOULD);
|
||||
|
||||
Set<Query> intersection = new HashSet<>(filters);
|
||||
intersection.retainAll(shoulds);
|
||||
|
||||
if (intersection.isEmpty() == false) {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
int minShouldMatch = getMinimumNumberShouldMatch();
|
||||
|
||||
for (BooleanClause clause : clauses) {
|
||||
if (intersection.contains(clause.getQuery())) {
|
||||
if (clause.getOccur() == Occur.SHOULD) {
|
||||
builder.add(new BooleanClause(clause.getQuery(), Occur.MUST));
|
||||
minShouldMatch--;
|
||||
}
|
||||
} else {
|
||||
builder.add(clause);
|
||||
}
|
||||
}
|
||||
|
||||
builder.setMinimumNumberShouldMatch(Math.max(0, minShouldMatch));
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Rewrite queries whose single scoring clause is a MUST clause on a
|
||||
// MatchAllDocsQuery to a ConstantScoreQuery
|
||||
{
|
||||
|
|
|
@ -256,24 +256,6 @@ public abstract class DataInput implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
/** Reads a Map<String,String> previously written
|
||||
* with {@link DataOutput#writeStringStringMap(Map)}.
|
||||
* @deprecated Only for reading existing formats. Encode maps with
|
||||
* {@link DataOutput#writeMapOfStrings(Map)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public Map<String,String> readStringStringMap() throws IOException {
|
||||
final Map<String,String> map = new HashMap<>();
|
||||
final int count = readInt();
|
||||
for(int i=0;i<count;i++) {
|
||||
final String key = readString();
|
||||
final String val = readString();
|
||||
map.put(key, val);
|
||||
}
|
||||
|
||||
return map;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a Map<String,String> previously written
|
||||
* with {@link DataOutput#writeMapOfStrings(Map)}.
|
||||
|
@ -296,21 +278,6 @@ public abstract class DataInput implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
/** Reads a Set<String> previously written
|
||||
* with {@link DataOutput#writeStringSet(Set)}.
|
||||
* @deprecated Only for reading existing formats. Encode maps with
|
||||
* {@link DataOutput#writeSetOfStrings(Set)} instead. */
|
||||
@Deprecated
|
||||
public Set<String> readStringSet() throws IOException {
|
||||
final Set<String> set = new HashSet<>();
|
||||
final int count = readInt();
|
||||
for(int i=0;i<count;i++) {
|
||||
set.add(readString());
|
||||
}
|
||||
|
||||
return set;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a Set<String> previously written
|
||||
* with {@link DataOutput#writeSetOfStrings(Set)}.
|
||||
|
|
|
@ -280,29 +280,6 @@ public abstract class DataOutput {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a String map.
|
||||
* <p>
|
||||
* First the size is written as an {@link #writeInt(int) Int32},
|
||||
* followed by each key-value pair written as two consecutive
|
||||
* {@link #writeString(String) String}s.
|
||||
*
|
||||
* @param map Input map. May be null (equivalent to an empty map)
|
||||
* @deprecated Use {@link #writeMapOfStrings(Map)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void writeStringStringMap(Map<String,String> map) throws IOException {
|
||||
if (map == null) {
|
||||
writeInt(0);
|
||||
} else {
|
||||
writeInt(map.size());
|
||||
for(final Map.Entry<String, String> entry: map.entrySet()) {
|
||||
writeString(entry.getKey());
|
||||
writeString(entry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a String map.
|
||||
* <p>
|
||||
|
@ -321,28 +298,6 @@ public abstract class DataOutput {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a String set.
|
||||
* <p>
|
||||
* First the size is written as an {@link #writeInt(int) Int32},
|
||||
* followed by each value written as a
|
||||
* {@link #writeString(String) String}.
|
||||
*
|
||||
* @param set Input set. May be null (equivalent to an empty set)
|
||||
* @deprecated Use {@link #writeMapOfStrings(Map)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public void writeStringSet(Set<String> set) throws IOException {
|
||||
if (set == null) {
|
||||
writeInt(0);
|
||||
} else {
|
||||
writeInt(set.size());
|
||||
for(String value : set) {
|
||||
writeString(value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a String set.
|
||||
* <p>
|
||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.RAMDirectory;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -84,7 +86,8 @@ public class TestAllFilesCheckIndexHeader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private void checkOneFile(Directory dir, String victim) throws IOException {
|
||||
try (BaseDirectoryWrapper dirCopy = newDirectory()) {
|
||||
// use ramdir explicit, as we do evil things like try to generate broken files, deletes must work.
|
||||
try (BaseDirectoryWrapper dirCopy = new MockDirectoryWrapper(random(), new RAMDirectory())) {
|
||||
dirCopy.setCheckIndexOnClose(false);
|
||||
|
||||
long victimLength = dir.fileLength(victim);
|
||||
|
@ -117,6 +120,8 @@ public class TestAllFilesCheckIndexHeader extends LuceneTestCase {
|
|||
break;
|
||||
}
|
||||
}
|
||||
// we have to try again, delete the first attempt and retry the loop
|
||||
dirCopy.deleteFile(name);
|
||||
}
|
||||
}
|
||||
dirCopy.sync(Collections.singleton(name));
|
||||
|
|
|
@ -206,6 +206,65 @@ public class TestBooleanRewrites extends LuceneTestCase {
|
|||
assertEquals(expected, searcher.rewrite(bq));
|
||||
}
|
||||
|
||||
// Duplicate Should and Filter query is converted to Must (with minShouldMatch -1)
|
||||
public void testConvertShouldAndFilterToMust() throws IOException {
|
||||
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||
|
||||
// no minShouldMatch
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.build();
|
||||
assertEquals(new TermQuery(new Term("foo", "bar")), searcher.rewrite(bq));
|
||||
|
||||
|
||||
// minShouldMatch is set to -1
|
||||
bq = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "quz")), Occur.SHOULD)
|
||||
.setMinimumNumberShouldMatch(2)
|
||||
.build();
|
||||
|
||||
BooleanQuery expected = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "quz")), Occur.SHOULD)
|
||||
.setMinimumNumberShouldMatch(1)
|
||||
.build();
|
||||
assertEquals(expected, searcher.rewrite(bq));
|
||||
}
|
||||
|
||||
// Duplicate Must or Filter with MustNot returns no match
|
||||
public void testDuplicateMustOrFilterWithMustNot() throws IOException {
|
||||
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||
|
||||
// Test Must with MustNot
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST)
|
||||
// other terms
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
|
||||
.add(new TermQuery(new Term("foo", "bad")), Occur.SHOULD)
|
||||
//
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
|
||||
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(bq));
|
||||
|
||||
// Test Filter with MustNot
|
||||
BooleanQuery bq2 = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
// other terms
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
|
||||
.add(new TermQuery(new Term("foo", "bad")), Occur.SHOULD)
|
||||
//
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
|
||||
.build();
|
||||
|
||||
assertEquals(new MatchNoDocsQuery(), searcher.rewrite(bq2));
|
||||
}
|
||||
|
||||
public void testRemoveMatchAllFilter() throws IOException {
|
||||
IndexSearcher searcher = newSearcher(new MultiReader());
|
||||
|
||||
|
|
|
@ -75,7 +75,7 @@ public final class CustomSeparatorBreakIterator extends BreakIterator {
|
|||
throw new IllegalArgumentException("offset out of bounds");
|
||||
} else if (pos == text.getEndIndex()) {
|
||||
// this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
|
||||
// http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909
|
||||
// https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
text.setIndex(text.getEndIndex());
|
||||
current = text.getIndex();
|
||||
return DONE;
|
||||
|
@ -112,7 +112,7 @@ public final class CustomSeparatorBreakIterator extends BreakIterator {
|
|||
throw new IllegalArgumentException("offset out of bounds");
|
||||
} else if (pos == text.getBeginIndex()) {
|
||||
// this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
|
||||
// http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909
|
||||
// https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
text.setIndex(text.getBeginIndex());
|
||||
current = text.getIndex();
|
||||
return DONE;
|
||||
|
|
|
@ -42,7 +42,7 @@ public final class WholeBreakIterator extends BreakIterator {
|
|||
throw new IllegalArgumentException("offset out of bounds");
|
||||
} else if (pos == end) {
|
||||
// this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
|
||||
// http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909
|
||||
// https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
current = end;
|
||||
return DONE;
|
||||
} else {
|
||||
|
@ -89,7 +89,7 @@ public final class WholeBreakIterator extends BreakIterator {
|
|||
throw new IllegalArgumentException("offset out of bounds");
|
||||
} else if (pos == start) {
|
||||
// this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
|
||||
// http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909
|
||||
// https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
current = start;
|
||||
return DONE;
|
||||
} else {
|
||||
|
|
|
@ -26,7 +26,6 @@
|
|||
|
||||
<path id="classpath">
|
||||
<pathelement path="${grouping.jar}"/>
|
||||
<pathelement path="${backward-codecs.jar}"/>
|
||||
<path refid="base.classpath"/>
|
||||
</path>
|
||||
|
||||
|
@ -35,14 +34,13 @@
|
|||
<pathelement location="${build.dir}/classes/java"/>
|
||||
</path>
|
||||
|
||||
<target name="init" depends="module-build.init,jar-grouping,jar-backward-codecs"/>
|
||||
<target name="init" depends="module-build.init,jar-grouping"/>
|
||||
|
||||
<target name="javadocs" depends="javadocs-grouping,javadocs-backward-codecs,compile-core,check-javadocs-uptodate"
|
||||
<target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
|
||||
unless="javadocs-uptodate-${name}">
|
||||
<invoke-module-javadoc>
|
||||
<links>
|
||||
<link href="../grouping"/>
|
||||
<link href="../backward-codecs"/>
|
||||
</links>
|
||||
</invoke-module-javadoc>
|
||||
</target>
|
||||
|
|
|
@ -17,20 +17,13 @@
|
|||
package org.apache.lucene.search.join;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.LongConsumer;
|
||||
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.legacy.LegacyNumericUtils;
|
||||
import org.apache.lucene.search.SimpleCollector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
|
||||
abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
|
||||
|
||||
|
@ -57,79 +50,4 @@ abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
|
|||
static Function<SortedSetDocValues> sortedSetDocValues(String field) {
|
||||
return (ctx) -> DocValues.getSortedSet(ctx, field);
|
||||
}
|
||||
|
||||
static Function<BinaryDocValues> numericAsBinaryDocValues(String field, LegacyNumericType numTyp) {
|
||||
return (ctx) -> {
|
||||
final NumericDocValues numeric = DocValues.getNumeric(ctx, field);
|
||||
final BytesRefBuilder bytes = new BytesRefBuilder();
|
||||
|
||||
final LongConsumer coder = coder(bytes, numTyp, field);
|
||||
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
public BytesRef get(int docID) {
|
||||
final long lVal = numeric.get(docID);
|
||||
coder.accept(lVal);
|
||||
return bytes.get();
|
||||
}
|
||||
};
|
||||
};
|
||||
}
|
||||
|
||||
static LongConsumer coder(BytesRefBuilder bytes, LegacyNumericType type, String fieldName){
|
||||
switch(type){
|
||||
case INT:
|
||||
return (l) -> LegacyNumericUtils.intToPrefixCoded((int) l, 0, bytes);
|
||||
case LONG:
|
||||
return (l) -> LegacyNumericUtils.longToPrefixCoded(l, 0, bytes);
|
||||
default:
|
||||
throw new IllegalArgumentException("Unsupported "+type+
|
||||
". Only "+ LegacyNumericType.INT+" and "+ LegacyNumericType.LONG+" are supported."
|
||||
+ "Field "+fieldName );
|
||||
}
|
||||
}
|
||||
|
||||
/** this adapter is quite weird. ords are per doc index, don't use ords across different docs*/
|
||||
static Function<SortedSetDocValues> sortedNumericAsSortedSetDocValues(String field, LegacyNumericType numTyp) {
|
||||
return (ctx) -> {
|
||||
final SortedNumericDocValues numerics = DocValues.getSortedNumeric(ctx, field);
|
||||
final BytesRefBuilder bytes = new BytesRefBuilder();
|
||||
|
||||
final LongConsumer coder = coder(bytes, numTyp, field);
|
||||
|
||||
return new SortedSetDocValues() {
|
||||
|
||||
private int index = Integer.MIN_VALUE;
|
||||
|
||||
@Override
|
||||
public long nextOrd() {
|
||||
return index < numerics.count()-1 ? ++index : NO_MORE_ORDS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int docID) {
|
||||
numerics.setDocument(docID);
|
||||
index=-1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef lookupOrd(long ord) {
|
||||
assert ord>=0 && ord<numerics.count();
|
||||
final long value = numerics.valueAt((int)ord);
|
||||
coder.accept(value);
|
||||
return bytes.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getValueCount() {
|
||||
throw new UnsupportedOperationException("it's just number encoding wrapper");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lookupTerm(BytesRef key) {
|
||||
throw new UnsupportedOperationException("it's just number encoding wrapper");
|
||||
}
|
||||
};
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.util.function.BiConsumer;
|
|||
import java.util.function.LongFunction;
|
||||
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
|
@ -111,50 +110,6 @@ public final class JoinUtil {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Because {@link LegacyNumericType} is deprecated, instead use {@link #createJoinQuery(String, boolean, String, Class, Query, IndexSearcher, ScoreMode)}
|
||||
*
|
||||
* Method for query time joining for numeric fields. It supports multi- and single- values longs and ints.
|
||||
* All considerations from {@link JoinUtil#createJoinQuery(String, boolean, String, Query, IndexSearcher, ScoreMode)} are applicable here too,
|
||||
* though memory consumption might be higher.
|
||||
* <p>
|
||||
*
|
||||
* @param fromField The from field to join from
|
||||
* @param multipleValuesPerDocument Whether the from field has multiple terms per document
|
||||
* when true fromField might be {@link DocValuesType#SORTED_NUMERIC},
|
||||
* otherwise fromField should be {@link DocValuesType#NUMERIC}
|
||||
* @param toField The to field to join to, should be {@link org.apache.lucene.legacy.LegacyIntField} or {@link org.apache.lucene.legacy.LegacyLongField}
|
||||
* @param numericType either {@link LegacyNumericType#INT} or {@link LegacyNumericType#LONG}, it should correspond to fromField and toField types
|
||||
* @param fromQuery The query to match documents on the from side
|
||||
* @param fromSearcher The searcher that executed the specified fromQuery
|
||||
* @param scoreMode Instructs how scores from the fromQuery are mapped to the returned query
|
||||
* @return a {@link Query} instance that can be used to join documents based on the
|
||||
* terms in the from and to field
|
||||
* @throws IOException If I/O related errors occur
|
||||
*/
|
||||
@Deprecated
|
||||
public static Query createJoinQuery(String fromField,
|
||||
boolean multipleValuesPerDocument,
|
||||
String toField, LegacyNumericType numericType,
|
||||
Query fromQuery,
|
||||
IndexSearcher fromSearcher,
|
||||
ScoreMode scoreMode) throws IOException {
|
||||
|
||||
final GenericTermsCollector termsCollector;
|
||||
|
||||
if (multipleValuesPerDocument) {
|
||||
Function<SortedSetDocValues> mvFunction = DocValuesTermsCollector.sortedNumericAsSortedSetDocValues(fromField,numericType);
|
||||
termsCollector = GenericTermsCollector.createCollectorMV(mvFunction, scoreMode);
|
||||
} else {
|
||||
Function<BinaryDocValues> svFunction = DocValuesTermsCollector.numericAsBinaryDocValues(fromField,numericType);
|
||||
termsCollector = GenericTermsCollector.createCollectorSV(svFunction, scoreMode);
|
||||
}
|
||||
|
||||
return createJoinQuery(multipleValuesPerDocument, toField, fromQuery, fromSearcher, scoreMode,
|
||||
termsCollector);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Method for query time joining for numeric fields. It supports multi- and single- values longs, ints, floats and longs.
|
||||
* All considerations from {@link JoinUtil#createJoinQuery(String, boolean, String, Query, IndexSearcher, ScoreMode)} are applicable here too,
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.index.PostingsEnum;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.legacy.LegacyNumericUtils;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -245,14 +244,6 @@ class TermsIncludingScoreQuery extends Query {
|
|||
for (int i = 0; i < terms.size(); i++) {
|
||||
terms.get(ords[i], ref);
|
||||
out.print(ref+" "+ref.utf8ToString()+" ");
|
||||
try {
|
||||
out.print(Long.toHexString(LegacyNumericUtils.prefixCodedToLong(ref))+"L");
|
||||
} catch (Exception e) {
|
||||
try {
|
||||
out.print(Integer.toHexString(LegacyNumericUtils.prefixCodedToInt(ref))+"i");
|
||||
} catch (Exception ee) {
|
||||
}
|
||||
}
|
||||
out.println(" score="+scores[ords[i]]);
|
||||
out.println("");
|
||||
}
|
||||
|
|
|
@ -56,9 +56,6 @@ import org.apache.lucene.index.LeafReader;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiDocValues;
|
||||
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
|
||||
import org.apache.lucene.legacy.LegacyIntField;
|
||||
import org.apache.lucene.legacy.LegacyLongField;
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
|
@ -971,7 +968,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
final String fromField = from ? "from":"to";
|
||||
final String toField = from ? "to":"from";
|
||||
|
||||
int surpriseMe = random().nextInt(3);
|
||||
int surpriseMe = random().nextInt(2);
|
||||
switch (surpriseMe) {
|
||||
case 0:
|
||||
Class<? extends Number> numType;
|
||||
|
@ -992,10 +989,6 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
joinQuery = JoinUtil.createJoinQuery(fromField + suffix, muliValsQuery, toField + suffix, numType, actualQuery, indexSearcher, scoreMode);
|
||||
break;
|
||||
case 1:
|
||||
final LegacyNumericType legacyNumType = random().nextBoolean() ? LegacyNumericType.INT: LegacyNumericType.LONG ;
|
||||
joinQuery = JoinUtil.createJoinQuery(fromField+legacyNumType, muliValsQuery, toField+legacyNumType, legacyNumType, actualQuery, indexSearcher, scoreMode);
|
||||
break;
|
||||
case 2:
|
||||
joinQuery = JoinUtil.createJoinQuery(fromField, muliValsQuery, toField, actualQuery, indexSearcher, scoreMode);
|
||||
break;
|
||||
default:
|
||||
|
@ -1336,26 +1329,24 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
document.add(newTextField(random, fieldName, linkValue, Field.Store.NO));
|
||||
|
||||
final int linkInt = Integer.parseUnsignedInt(linkValue,16);
|
||||
document.add(new LegacyIntField(fieldName + LegacyNumericType.INT, linkInt, Field.Store.NO));
|
||||
document.add(new IntPoint(fieldName + LegacyNumericType.INT, linkInt));
|
||||
document.add(new IntPoint(fieldName + "INT", linkInt));
|
||||
document.add(new FloatPoint(fieldName + "FLOAT", linkInt));
|
||||
|
||||
final long linkLong = linkInt<<32 | linkInt;
|
||||
document.add(new LegacyLongField(fieldName + LegacyNumericType.LONG, linkLong, Field.Store.NO));
|
||||
document.add(new LongPoint(fieldName + LegacyNumericType.LONG, linkLong));
|
||||
document.add(new LongPoint(fieldName + "LONG", linkLong));
|
||||
document.add(new DoublePoint(fieldName + "DOUBLE", linkLong));
|
||||
|
||||
if (multipleValuesPerDocument) {
|
||||
document.add(new SortedSetDocValuesField(fieldName, new BytesRef(linkValue)));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ LegacyNumericType.INT, linkInt));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ "INT", linkInt));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ "FLOAT", Float.floatToRawIntBits(linkInt)));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ LegacyNumericType.LONG, linkLong));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ "LONG", linkLong));
|
||||
document.add(new SortedNumericDocValuesField(fieldName+ "DOUBLE", Double.doubleToRawLongBits(linkLong)));
|
||||
} else {
|
||||
document.add(new SortedDocValuesField(fieldName, new BytesRef(linkValue)));
|
||||
document.add(new NumericDocValuesField(fieldName+ LegacyNumericType.INT, linkInt));
|
||||
document.add(new NumericDocValuesField(fieldName+ "INT", linkInt));
|
||||
document.add(new FloatDocValuesField(fieldName+ "FLOAT", linkInt));
|
||||
document.add(new NumericDocValuesField(fieldName+ LegacyNumericType.LONG, linkLong));
|
||||
document.add(new NumericDocValuesField(fieldName+ "LONG", linkLong));
|
||||
document.add(new DoubleDocValuesField(fieldName+ "DOUBLE", linkLong));
|
||||
}
|
||||
if (globalOrdinalJoin) {
|
||||
|
|
|
@ -25,17 +25,15 @@
|
|||
<path id="classpath">
|
||||
<pathelement path="${queries.jar}"/>
|
||||
<pathelement path="${sandbox.jar}"/>
|
||||
<pathelement path="${backward-codecs.jar}"/>
|
||||
<path refid="base.classpath"/>
|
||||
</path>
|
||||
|
||||
<target name="compile-core" depends="jar-backward-codecs,jar-queries,jar-sandbox,common.compile-core"/>
|
||||
<target name="compile-core" depends="jar-queries,jar-sandbox,common.compile-core"/>
|
||||
|
||||
<target name="javadocs" depends="javadocs-backward-codecs,javadocs-queries,javadocs-sandbox,compile-core,check-javadocs-uptodate"
|
||||
<target name="javadocs" depends="javadocs-queries,javadocs-sandbox,compile-core,check-javadocs-uptodate"
|
||||
unless="javadocs-uptodate-${name}">
|
||||
<invoke-module-javadoc>
|
||||
<links>
|
||||
<link href="../backward-codecs"/>
|
||||
<link href="../queries"/>
|
||||
<link href="../sandbox"/>
|
||||
</links>
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.queryparser.flexible.core.QueryParserHelper;
|
|||
import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
|
||||
import org.apache.lucene.queryparser.flexible.standard.builders.StandardQueryTreeBuilder;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.FuzzyConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
|
||||
|
@ -293,24 +292,6 @@ public class StandardQueryParser extends QueryParserHelper implements CommonQuer
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets field configuration for legacy numeric fields
|
||||
* @deprecated Index with points instead and use {@link #setPointsConfigMap(Map)}
|
||||
*/
|
||||
@Deprecated
|
||||
public void setLegacyNumericConfigMap(Map<String,LegacyNumericConfig> legacyNumericConfigMap) {
|
||||
getQueryConfigHandler().set(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP, legacyNumericConfigMap);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets field configuration for legacy numeric fields
|
||||
* @deprecated Index with points instead and use {@link #getPointsConfigMap()}
|
||||
*/
|
||||
@Deprecated
|
||||
public Map<String,LegacyNumericConfig> getLegacyNumericConfigMap() {
|
||||
return getQueryConfigHandler().get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP);
|
||||
}
|
||||
|
||||
public void setPointsConfigMap(Map<String,PointsConfig> pointsConfigMap) {
|
||||
getQueryConfigHandler().set(ConfigurationKeys.POINTS_CONFIG_MAP, pointsConfigMap);
|
||||
}
|
||||
|
|
|
@ -1,93 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.builders;
|
||||
|
||||
import org.apache.lucene.legacy.LegacyNumericRangeQuery;
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
|
||||
import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
|
||||
import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
|
||||
|
||||
/**
|
||||
* Builds {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s out of {@link LegacyNumericRangeQueryNode}s.
|
||||
*
|
||||
* @see org.apache.lucene.legacy.LegacyNumericRangeQuery
|
||||
* @see LegacyNumericRangeQueryNode
|
||||
* @deprecated Index with points and use {@link PointRangeQueryNodeBuilder} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericRangeQueryNodeBuilder implements StandardQueryBuilder {
|
||||
|
||||
/**
|
||||
* Constructs a {@link LegacyNumericRangeQueryNodeBuilder} object.
|
||||
*/
|
||||
public LegacyNumericRangeQueryNodeBuilder() {
|
||||
// empty constructor
|
||||
}
|
||||
|
||||
@Override
|
||||
public LegacyNumericRangeQuery<? extends Number> build(QueryNode queryNode)
|
||||
throws QueryNodeException {
|
||||
LegacyNumericRangeQueryNode numericRangeNode = (LegacyNumericRangeQueryNode) queryNode;
|
||||
|
||||
LegacyNumericQueryNode lowerNumericNode = numericRangeNode.getLowerBound();
|
||||
LegacyNumericQueryNode upperNumericNode = numericRangeNode.getUpperBound();
|
||||
|
||||
Number lowerNumber = lowerNumericNode.getValue();
|
||||
Number upperNumber = upperNumericNode.getValue();
|
||||
|
||||
LegacyNumericConfig numericConfig = numericRangeNode.getNumericConfig();
|
||||
LegacyNumericType numberType = numericConfig.getType();
|
||||
String field = StringUtils.toString(numericRangeNode.getField());
|
||||
boolean minInclusive = numericRangeNode.isLowerInclusive();
|
||||
boolean maxInclusive = numericRangeNode.isUpperInclusive();
|
||||
int precisionStep = numericConfig.getPrecisionStep();
|
||||
|
||||
switch (numberType) {
|
||||
|
||||
case LONG:
|
||||
return LegacyNumericRangeQuery.newLongRange(field, precisionStep,
|
||||
(Long) lowerNumber, (Long) upperNumber, minInclusive, maxInclusive);
|
||||
|
||||
case INT:
|
||||
return LegacyNumericRangeQuery.newIntRange(field, precisionStep,
|
||||
(Integer) lowerNumber, (Integer) upperNumber, minInclusive,
|
||||
maxInclusive);
|
||||
|
||||
case FLOAT:
|
||||
return LegacyNumericRangeQuery.newFloatRange(field, precisionStep,
|
||||
(Float) lowerNumber, (Float) upperNumber, minInclusive,
|
||||
maxInclusive);
|
||||
|
||||
case DOUBLE:
|
||||
return LegacyNumericRangeQuery.newDoubleRange(field, precisionStep,
|
||||
(Double) lowerNumber, (Double) upperNumber, minInclusive,
|
||||
maxInclusive);
|
||||
|
||||
default :
|
||||
throw new QueryNodeException(new MessageImpl(
|
||||
QueryParserMessages.UNSUPPORTED_NUMERIC_DATA_TYPE, numberType));
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -32,8 +32,6 @@ import org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNod
|
|||
import org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.PointQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.PointRangeQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.PrefixWildcardQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.TermRangeQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
|
||||
|
@ -59,8 +57,6 @@ public class StandardQueryTreeBuilder extends QueryTreeBuilder implements
|
|||
setBuilder(FieldQueryNode.class, new FieldQueryNodeBuilder());
|
||||
setBuilder(BooleanQueryNode.class, new BooleanQueryNodeBuilder());
|
||||
setBuilder(FuzzyQueryNode.class, new FuzzyQueryNodeBuilder());
|
||||
setBuilder(LegacyNumericQueryNode.class, new DummyQueryNodeBuilder());
|
||||
setBuilder(LegacyNumericRangeQueryNode.class, new LegacyNumericRangeQueryNodeBuilder());
|
||||
setBuilder(PointQueryNode.class, new DummyQueryNodeBuilder());
|
||||
setBuilder(PointRangeQueryNode.class, new PointRangeQueryNodeBuilder());
|
||||
setBuilder(BoostQueryNode.class, new BoostQueryNodeBuilder());
|
||||
|
|
|
@ -1,165 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.config;
|
||||
|
||||
import java.text.NumberFormat;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
|
||||
/**
|
||||
* This class holds the configuration used to parse numeric queries and create
|
||||
* {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s.
|
||||
*
|
||||
* @see org.apache.lucene.legacy.LegacyNumericRangeQuery
|
||||
* @see NumberFormat
|
||||
* @deprecated Index with Points instead and use {@link PointsConfig}
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericConfig {
|
||||
|
||||
private int precisionStep;
|
||||
|
||||
private NumberFormat format;
|
||||
|
||||
private LegacyNumericType type;
|
||||
|
||||
/**
|
||||
* Constructs a {@link LegacyNumericConfig} object.
|
||||
*
|
||||
* @param precisionStep
|
||||
* the precision used to index the numeric values
|
||||
* @param format
|
||||
* the {@link NumberFormat} used to parse a {@link String} to
|
||||
* {@link Number}
|
||||
* @param type
|
||||
* the numeric type used to index the numeric values
|
||||
*
|
||||
* @see LegacyNumericConfig#setPrecisionStep(int)
|
||||
* @see LegacyNumericConfig#setNumberFormat(NumberFormat)
|
||||
* @see #setType(LegacyNumericType)
|
||||
*/
|
||||
public LegacyNumericConfig(int precisionStep, NumberFormat format,
|
||||
LegacyNumericType type) {
|
||||
setPrecisionStep(precisionStep);
|
||||
setNumberFormat(format);
|
||||
setType(type);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the precision used to index the numeric values
|
||||
*
|
||||
* @return the precision used to index the numeric values
|
||||
*
|
||||
* @see org.apache.lucene.legacy.LegacyNumericRangeQuery#getPrecisionStep()
|
||||
*/
|
||||
public int getPrecisionStep() {
|
||||
return precisionStep;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the precision used to index the numeric values
|
||||
*
|
||||
* @param precisionStep
|
||||
* the precision used to index the numeric values
|
||||
*
|
||||
* @see org.apache.lucene.legacy.LegacyNumericRangeQuery#getPrecisionStep()
|
||||
*/
|
||||
public void setPrecisionStep(int precisionStep) {
|
||||
this.precisionStep = precisionStep;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link NumberFormat} used to parse a {@link String} to
|
||||
* {@link Number}
|
||||
*
|
||||
* @return the {@link NumberFormat} used to parse a {@link String} to
|
||||
* {@link Number}
|
||||
*/
|
||||
public NumberFormat getNumberFormat() {
|
||||
return format;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the numeric type used to index the numeric values
|
||||
*
|
||||
* @return the numeric type used to index the numeric values
|
||||
*/
|
||||
public LegacyNumericType getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the numeric type used to index the numeric values
|
||||
*
|
||||
* @param type the numeric type used to index the numeric values
|
||||
*/
|
||||
public void setType(LegacyNumericType type) {
|
||||
|
||||
if (type == null) {
|
||||
throw new IllegalArgumentException("type must not be null!");
|
||||
}
|
||||
|
||||
this.type = type;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link NumberFormat} used to parse a {@link String} to
|
||||
* {@link Number}
|
||||
*
|
||||
* @param format
|
||||
* the {@link NumberFormat} used to parse a {@link String} to
|
||||
* {@link Number}, must not be <code>null</code>
|
||||
*/
|
||||
public void setNumberFormat(NumberFormat format) {
|
||||
|
||||
if (format == null) {
|
||||
throw new IllegalArgumentException("format must not be null!");
|
||||
}
|
||||
|
||||
this.format = format;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
|
||||
if (obj == this) return true;
|
||||
|
||||
if (obj instanceof LegacyNumericConfig) {
|
||||
LegacyNumericConfig other = (LegacyNumericConfig) obj;
|
||||
|
||||
if (this.precisionStep == other.precisionStep
|
||||
&& this.type == other.type
|
||||
&& (this.format == other.format || (this.format.equals(other.format)))) {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return false;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(precisionStep, type, format);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,75 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.config;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.FieldConfigListener;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
|
||||
|
||||
/**
|
||||
* This listener is used to listen to {@link FieldConfig} requests in
|
||||
* {@link QueryConfigHandler} and add {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG}
|
||||
* based on the {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG_MAP} set in the
|
||||
* {@link QueryConfigHandler}.
|
||||
*
|
||||
* @see LegacyNumericConfig
|
||||
* @see QueryConfigHandler
|
||||
* @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
|
||||
* @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG_MAP
|
||||
* @deprecated Index with Points instead and use {@link PointsConfigListener}
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericFieldConfigListener implements FieldConfigListener {
|
||||
|
||||
final private QueryConfigHandler config;
|
||||
|
||||
/**
|
||||
* Constructs a {@link LegacyNumericFieldConfigListener} object using the given {@link QueryConfigHandler}.
|
||||
*
|
||||
* @param config the {@link QueryConfigHandler} it will listen too
|
||||
*/
|
||||
public LegacyNumericFieldConfigListener(QueryConfigHandler config) {
|
||||
|
||||
if (config == null) {
|
||||
throw new IllegalArgumentException("config must not be null!");
|
||||
}
|
||||
|
||||
this.config = config;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void buildFieldConfig(FieldConfig fieldConfig) {
|
||||
Map<String,LegacyNumericConfig> numericConfigMap = config
|
||||
.get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP);
|
||||
|
||||
if (numericConfigMap != null) {
|
||||
LegacyNumericConfig numericConfig = numericConfigMap
|
||||
.get(fieldConfig.getField());
|
||||
|
||||
if (numericConfig != null) {
|
||||
fieldConfig.set(ConfigurationKeys.LEGACY_NUMERIC_CONFIG, numericConfig);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -158,39 +158,19 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
|
|||
*/
|
||||
final public static ConfigurationKey<Float> BOOST = ConfigurationKey.newInstance();
|
||||
|
||||
/**
|
||||
* Key used to set a field to its {@link LegacyNumericConfig}.
|
||||
*
|
||||
* @see StandardQueryParser#setLegacyNumericConfigMap(Map)
|
||||
* @see StandardQueryParser#getLegacyNumericConfigMap()
|
||||
* @deprecated Index with Points instead and use {@link #POINTS_CONFIG}
|
||||
*/
|
||||
@Deprecated
|
||||
final public static ConfigurationKey<LegacyNumericConfig> LEGACY_NUMERIC_CONFIG = ConfigurationKey.newInstance();
|
||||
|
||||
/**
|
||||
* Key used to set the {@link LegacyNumericConfig} in {@link FieldConfig} for numeric fields.
|
||||
*
|
||||
* @see StandardQueryParser#setLegacyNumericConfigMap(Map)
|
||||
* @see StandardQueryParser#getLegacyNumericConfigMap()
|
||||
* @deprecated Index with Points instead and use {@link #POINTS_CONFIG_MAP}
|
||||
*/
|
||||
@Deprecated
|
||||
final public static ConfigurationKey<Map<String,LegacyNumericConfig>> LEGACY_NUMERIC_CONFIG_MAP = ConfigurationKey.newInstance();
|
||||
|
||||
/**
|
||||
* Key used to set a field to its {@link PointsConfig}.
|
||||
*
|
||||
* @see StandardQueryParser#setLegacyNumericConfigMap(Map)
|
||||
* @see StandardQueryParser#getLegacyNumericConfigMap()
|
||||
* @see StandardQueryParser#setPointsConfigMap(Map)
|
||||
* @see StandardQueryParser#getPointsConfigMap()
|
||||
*/
|
||||
final public static ConfigurationKey<PointsConfig> POINTS_CONFIG = ConfigurationKey.newInstance();
|
||||
|
||||
/**
|
||||
* Key used to set the {@link PointsConfig} in {@link FieldConfig} for point fields.
|
||||
*
|
||||
* @see StandardQueryParser#setLegacyNumericConfigMap(Map)
|
||||
* @see StandardQueryParser#getLegacyNumericConfigMap()
|
||||
* @see StandardQueryParser#setPointsConfigMap(Map)
|
||||
* @see StandardQueryParser#getPointsConfigMap()
|
||||
*/
|
||||
final public static ConfigurationKey<Map<String,PointsConfig>> POINTS_CONFIG_MAP = ConfigurationKey.newInstance();
|
||||
|
||||
|
@ -207,7 +187,6 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
|
|||
// Add listener that will build the FieldConfig.
|
||||
addFieldConfigListener(new FieldBoostMapFCListener(this));
|
||||
addFieldConfigListener(new FieldDateResolutionFCListener(this));
|
||||
addFieldConfigListener(new LegacyNumericFieldConfigListener(this));
|
||||
addFieldConfigListener(new PointsConfigListener(this));
|
||||
|
||||
// Default Values
|
||||
|
|
|
@ -1,153 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.nodes;
|
||||
|
||||
import java.text.NumberFormat;
|
||||
import java.util.Locale;
|
||||
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.FieldValuePairQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.QueryNodeImpl;
|
||||
import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
|
||||
import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax.Type;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
|
||||
/**
|
||||
* This query node represents a field query that holds a numeric value. It is
|
||||
* similar to {@link FieldQueryNode}, however the {@link #getValue()} returns a
|
||||
* {@link Number}.
|
||||
*
|
||||
* @see LegacyNumericConfig
|
||||
* @deprecated Index with Points instead and use {@link PointQueryNode} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericQueryNode extends QueryNodeImpl implements
|
||||
FieldValuePairQueryNode<Number> {
|
||||
|
||||
private NumberFormat numberFormat;
|
||||
|
||||
private CharSequence field;
|
||||
|
||||
private Number value;
|
||||
|
||||
/**
|
||||
* Creates a {@link LegacyNumericQueryNode} object using the given field,
|
||||
* {@link Number} value and {@link NumberFormat} used to convert the value to
|
||||
* {@link String}.
|
||||
*
|
||||
* @param field the field associated with this query node
|
||||
* @param value the value hold by this node
|
||||
* @param numberFormat the {@link NumberFormat} used to convert the value to {@link String}
|
||||
*/
|
||||
public LegacyNumericQueryNode(CharSequence field, Number value,
|
||||
NumberFormat numberFormat) {
|
||||
|
||||
super();
|
||||
|
||||
setNumberFormat(numberFormat);
|
||||
setField(field);
|
||||
setValue(value);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the field associated with this node.
|
||||
*
|
||||
* @return the field associated with this node
|
||||
*/
|
||||
@Override
|
||||
public CharSequence getField() {
|
||||
return this.field;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the field associated with this node.
|
||||
*
|
||||
* @param fieldName the field associated with this node
|
||||
*/
|
||||
@Override
|
||||
public void setField(CharSequence fieldName) {
|
||||
this.field = fieldName;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is used to get the value converted to {@link String} and
|
||||
* escaped using the given {@link EscapeQuerySyntax}.
|
||||
*
|
||||
* @param escaper the {@link EscapeQuerySyntax} used to escape the value {@link String}
|
||||
*
|
||||
* @return the value converte to {@link String} and escaped
|
||||
*/
|
||||
protected CharSequence getTermEscaped(EscapeQuerySyntax escaper) {
|
||||
return escaper.escape(numberFormat.format(this.value),
|
||||
Locale.ROOT, Type.NORMAL);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
|
||||
if (isDefaultField(this.field)) {
|
||||
return getTermEscaped(escapeSyntaxParser);
|
||||
} else {
|
||||
return this.field + ":" + getTermEscaped(escapeSyntaxParser);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link NumberFormat} used to convert the value to {@link String}.
|
||||
*
|
||||
* @param format the {@link NumberFormat} used to convert the value to {@link String}
|
||||
*/
|
||||
public void setNumberFormat(NumberFormat format) {
|
||||
this.numberFormat = format;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link NumberFormat} used to convert the value to {@link String}.
|
||||
*
|
||||
* @return the {@link NumberFormat} used to convert the value to {@link String}
|
||||
*/
|
||||
public NumberFormat getNumberFormat() {
|
||||
return this.numberFormat;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the numeric value as {@link Number}.
|
||||
*
|
||||
* @return the numeric value
|
||||
*/
|
||||
@Override
|
||||
public Number getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the numeric value.
|
||||
*
|
||||
* @param value the numeric value
|
||||
*/
|
||||
@Override
|
||||
public void setValue(Number value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "<numeric field='" + this.field + "' number='"
|
||||
+ numberFormat.format(value) + "'/>";
|
||||
}
|
||||
|
||||
}
|
|
@ -1,152 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.nodes;
|
||||
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
|
||||
import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
|
||||
import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
|
||||
/**
|
||||
* This query node represents a range query composed by {@link LegacyNumericQueryNode}
|
||||
* bounds, which means the bound values are {@link Number}s.
|
||||
*
|
||||
* @see LegacyNumericQueryNode
|
||||
* @see AbstractRangeQueryNode
|
||||
* @deprecated Index with Points instead and use {@link PointRangeQueryNode} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericRangeQueryNode extends
|
||||
AbstractRangeQueryNode<LegacyNumericQueryNode> {
|
||||
|
||||
public LegacyNumericConfig numericConfig;
|
||||
|
||||
/**
|
||||
* Constructs a {@link LegacyNumericRangeQueryNode} object using the given
|
||||
* {@link LegacyNumericQueryNode} as its bounds and {@link LegacyNumericConfig}.
|
||||
*
|
||||
* @param lower the lower bound
|
||||
* @param upper the upper bound
|
||||
* @param lowerInclusive <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
|
||||
* @param upperInclusive <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
|
||||
* @param numericConfig the {@link LegacyNumericConfig} that represents associated with the upper and lower bounds
|
||||
*
|
||||
* @see #setBounds(LegacyNumericQueryNode, LegacyNumericQueryNode, boolean, boolean, LegacyNumericConfig)
|
||||
*/
|
||||
public LegacyNumericRangeQueryNode(LegacyNumericQueryNode lower, LegacyNumericQueryNode upper,
|
||||
boolean lowerInclusive, boolean upperInclusive, LegacyNumericConfig numericConfig) throws QueryNodeException {
|
||||
setBounds(lower, upper, lowerInclusive, upperInclusive, numericConfig);
|
||||
}
|
||||
|
||||
private static LegacyNumericType getNumericDataType(Number number) throws QueryNodeException {
|
||||
|
||||
if (number instanceof Long) {
|
||||
return LegacyNumericType.LONG;
|
||||
} else if (number instanceof Integer) {
|
||||
return LegacyNumericType.INT;
|
||||
} else if (number instanceof Double) {
|
||||
return LegacyNumericType.DOUBLE;
|
||||
} else if (number instanceof Float) {
|
||||
return LegacyNumericType.FLOAT;
|
||||
} else {
|
||||
throw new QueryNodeException(
|
||||
new MessageImpl(
|
||||
QueryParserMessages.NUMBER_CLASS_NOT_SUPPORTED_BY_NUMERIC_RANGE_QUERY,
|
||||
number.getClass()));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the upper and lower bounds of this range query node and the
|
||||
* {@link LegacyNumericConfig} associated with these bounds.
|
||||
*
|
||||
* @param lower the lower bound
|
||||
* @param upper the upper bound
|
||||
* @param lowerInclusive <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
|
||||
* @param upperInclusive <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
|
||||
* @param numericConfig the {@link LegacyNumericConfig} that represents associated with the upper and lower bounds
|
||||
*
|
||||
*/
|
||||
public void setBounds(LegacyNumericQueryNode lower, LegacyNumericQueryNode upper,
|
||||
boolean lowerInclusive, boolean upperInclusive, LegacyNumericConfig numericConfig) throws QueryNodeException {
|
||||
|
||||
if (numericConfig == null) {
|
||||
throw new IllegalArgumentException("numericConfig must not be null!");
|
||||
}
|
||||
|
||||
LegacyNumericType lowerNumberType, upperNumberType;
|
||||
|
||||
if (lower != null && lower.getValue() != null) {
|
||||
lowerNumberType = getNumericDataType(lower.getValue());
|
||||
} else {
|
||||
lowerNumberType = null;
|
||||
}
|
||||
|
||||
if (upper != null && upper.getValue() != null) {
|
||||
upperNumberType = getNumericDataType(upper.getValue());
|
||||
} else {
|
||||
upperNumberType = null;
|
||||
}
|
||||
|
||||
if (lowerNumberType != null
|
||||
&& !lowerNumberType.equals(numericConfig.getType())) {
|
||||
throw new IllegalArgumentException(
|
||||
"lower value's type should be the same as numericConfig type: "
|
||||
+ lowerNumberType + " != " + numericConfig.getType());
|
||||
}
|
||||
|
||||
if (upperNumberType != null
|
||||
&& !upperNumberType.equals(numericConfig.getType())) {
|
||||
throw new IllegalArgumentException(
|
||||
"upper value's type should be the same as numericConfig type: "
|
||||
+ upperNumberType + " != " + numericConfig.getType());
|
||||
}
|
||||
|
||||
super.setBounds(lower, upper, lowerInclusive, upperInclusive);
|
||||
this.numericConfig = numericConfig;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link LegacyNumericConfig} associated with the lower and upper bounds.
|
||||
*
|
||||
* @return the {@link LegacyNumericConfig} associated with the lower and upper bounds
|
||||
*/
|
||||
public LegacyNumericConfig getNumericConfig() {
|
||||
return this.numericConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder("<numericRange lowerInclusive='");
|
||||
|
||||
sb.append(isLowerInclusive()).append("' upperInclusive='").append(
|
||||
isUpperInclusive()).append(
|
||||
"' precisionStep='" + numericConfig.getPrecisionStep()).append(
|
||||
"' type='" + numericConfig.getType()).append("'>\n");
|
||||
|
||||
sb.append(getLowerBound()).append('\n');
|
||||
sb.append(getUpperBound()).append('\n');
|
||||
sb.append("</numericRange>");
|
||||
|
||||
return sb.toString();
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,154 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.processors;
|
||||
|
||||
import java.text.NumberFormat;
|
||||
import java.text.ParseException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
|
||||
import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
|
||||
|
||||
/**
|
||||
* This processor is used to convert {@link FieldQueryNode}s to
|
||||
* {@link LegacyNumericRangeQueryNode}s. It looks for
|
||||
* {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} set in the {@link FieldConfig} of
|
||||
* every {@link FieldQueryNode} found. If
|
||||
* {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} is found, it considers that
|
||||
* {@link FieldQueryNode} to be a numeric query and convert it to
|
||||
* {@link LegacyNumericRangeQueryNode} with upper and lower inclusive and lower and
|
||||
* upper equals to the value represented by the {@link FieldQueryNode} converted
|
||||
* to {@link Number}. It means that <b>field:1</b> is converted to <b>field:[1
|
||||
* TO 1]</b>. <br>
|
||||
* <br>
|
||||
* Note that {@link FieldQueryNode}s children of a
|
||||
* {@link RangeQueryNode} are ignored.
|
||||
*
|
||||
* @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
|
||||
* @see FieldQueryNode
|
||||
* @see LegacyNumericConfig
|
||||
* @see LegacyNumericQueryNode
|
||||
* @deprecated Index with points and use {@link PointQueryNodeProcessor} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericQueryNodeProcessor extends QueryNodeProcessorImpl {
|
||||
|
||||
/**
|
||||
* Constructs a {@link LegacyNumericQueryNodeProcessor} object.
|
||||
*/
|
||||
public LegacyNumericQueryNodeProcessor() {
|
||||
// empty constructor
|
||||
}
|
||||
|
||||
@Override
|
||||
protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
|
||||
|
||||
if (node instanceof FieldQueryNode
|
||||
&& !(node.getParent() instanceof RangeQueryNode)) {
|
||||
|
||||
QueryConfigHandler config = getQueryConfigHandler();
|
||||
|
||||
if (config != null) {
|
||||
FieldQueryNode fieldNode = (FieldQueryNode) node;
|
||||
FieldConfig fieldConfig = config.getFieldConfig(fieldNode
|
||||
.getFieldAsString());
|
||||
|
||||
if (fieldConfig != null) {
|
||||
LegacyNumericConfig numericConfig = fieldConfig
|
||||
.get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG);
|
||||
|
||||
if (numericConfig != null) {
|
||||
|
||||
NumberFormat numberFormat = numericConfig.getNumberFormat();
|
||||
String text = fieldNode.getTextAsString();
|
||||
Number number = null;
|
||||
|
||||
if (text.length() > 0) {
|
||||
|
||||
try {
|
||||
number = numberFormat.parse(text);
|
||||
|
||||
} catch (ParseException e) {
|
||||
throw new QueryNodeParseException(new MessageImpl(
|
||||
QueryParserMessages.COULD_NOT_PARSE_NUMBER, fieldNode
|
||||
.getTextAsString(), numberFormat.getClass()
|
||||
.getCanonicalName()), e);
|
||||
}
|
||||
|
||||
switch (numericConfig.getType()) {
|
||||
case LONG:
|
||||
number = number.longValue();
|
||||
break;
|
||||
case INT:
|
||||
number = number.intValue();
|
||||
break;
|
||||
case DOUBLE:
|
||||
number = number.doubleValue();
|
||||
break;
|
||||
case FLOAT:
|
||||
number = number.floatValue();
|
||||
}
|
||||
|
||||
} else {
|
||||
throw new QueryNodeParseException(new MessageImpl(
|
||||
QueryParserMessages.NUMERIC_CANNOT_BE_EMPTY, fieldNode.getFieldAsString()));
|
||||
}
|
||||
|
||||
LegacyNumericQueryNode lowerNode = new LegacyNumericQueryNode(fieldNode
|
||||
.getField(), number, numberFormat);
|
||||
LegacyNumericQueryNode upperNode = new LegacyNumericQueryNode(fieldNode
|
||||
.getField(), number, numberFormat);
|
||||
|
||||
return new LegacyNumericRangeQueryNode(lowerNode, upperNode, true, true,
|
||||
numericConfig);
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return node;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
|
||||
return node;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
|
||||
throws QueryNodeException {
|
||||
return children;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,170 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard.processors;
|
||||
|
||||
import java.text.NumberFormat;
|
||||
import java.text.ParseException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
|
||||
import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
|
||||
import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
|
||||
import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
|
||||
import org.apache.lucene.queryparser.flexible.standard.nodes.TermRangeQueryNode;
|
||||
|
||||
/**
|
||||
* This processor is used to convert {@link TermRangeQueryNode}s to
|
||||
* {@link LegacyNumericRangeQueryNode}s. It looks for
|
||||
* {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} set in the {@link FieldConfig} of
|
||||
* every {@link TermRangeQueryNode} found. If
|
||||
* {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} is found, it considers that
|
||||
* {@link TermRangeQueryNode} to be a numeric range query and convert it to
|
||||
* {@link LegacyNumericRangeQueryNode}.
|
||||
*
|
||||
* @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
|
||||
* @see TermRangeQueryNode
|
||||
* @see LegacyNumericConfig
|
||||
* @see LegacyNumericRangeQueryNode
|
||||
* @deprecated Index with points and use {@link PointRangeQueryNodeProcessor} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public class LegacyNumericRangeQueryNodeProcessor extends QueryNodeProcessorImpl {
|
||||
|
||||
/**
|
||||
* Constructs an empty {@link LegacyNumericRangeQueryNode} object.
|
||||
*/
|
||||
public LegacyNumericRangeQueryNodeProcessor() {
|
||||
// empty constructor
|
||||
}
|
||||
|
||||
@Override
|
||||
protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
|
||||
|
||||
if (node instanceof TermRangeQueryNode) {
|
||||
QueryConfigHandler config = getQueryConfigHandler();
|
||||
|
||||
if (config != null) {
|
||||
TermRangeQueryNode termRangeNode = (TermRangeQueryNode) node;
|
||||
FieldConfig fieldConfig = config.getFieldConfig(StringUtils
|
||||
.toString(termRangeNode.getField()));
|
||||
|
||||
if (fieldConfig != null) {
|
||||
|
||||
LegacyNumericConfig numericConfig = fieldConfig
|
||||
.get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG);
|
||||
|
||||
if (numericConfig != null) {
|
||||
|
||||
FieldQueryNode lower = termRangeNode.getLowerBound();
|
||||
FieldQueryNode upper = termRangeNode.getUpperBound();
|
||||
|
||||
String lowerText = lower.getTextAsString();
|
||||
String upperText = upper.getTextAsString();
|
||||
NumberFormat numberFormat = numericConfig.getNumberFormat();
|
||||
Number lowerNumber = null, upperNumber = null;
|
||||
|
||||
if (lowerText.length() > 0) {
|
||||
|
||||
try {
|
||||
lowerNumber = numberFormat.parse(lowerText);
|
||||
|
||||
} catch (ParseException e) {
|
||||
throw new QueryNodeParseException(new MessageImpl(
|
||||
QueryParserMessages.COULD_NOT_PARSE_NUMBER, lower
|
||||
.getTextAsString(), numberFormat.getClass()
|
||||
.getCanonicalName()), e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (upperText.length() > 0) {
|
||||
|
||||
try {
|
||||
upperNumber = numberFormat.parse(upperText);
|
||||
|
||||
} catch (ParseException e) {
|
||||
throw new QueryNodeParseException(new MessageImpl(
|
||||
QueryParserMessages.COULD_NOT_PARSE_NUMBER, upper
|
||||
.getTextAsString(), numberFormat.getClass()
|
||||
.getCanonicalName()), e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
switch (numericConfig.getType()) {
|
||||
case LONG:
|
||||
if (upperNumber != null) upperNumber = upperNumber.longValue();
|
||||
if (lowerNumber != null) lowerNumber = lowerNumber.longValue();
|
||||
break;
|
||||
case INT:
|
||||
if (upperNumber != null) upperNumber = upperNumber.intValue();
|
||||
if (lowerNumber != null) lowerNumber = lowerNumber.intValue();
|
||||
break;
|
||||
case DOUBLE:
|
||||
if (upperNumber != null) upperNumber = upperNumber.doubleValue();
|
||||
if (lowerNumber != null) lowerNumber = lowerNumber.doubleValue();
|
||||
break;
|
||||
case FLOAT:
|
||||
if (upperNumber != null) upperNumber = upperNumber.floatValue();
|
||||
if (lowerNumber != null) lowerNumber = lowerNumber.floatValue();
|
||||
}
|
||||
|
||||
LegacyNumericQueryNode lowerNode = new LegacyNumericQueryNode(
|
||||
termRangeNode.getField(), lowerNumber, numberFormat);
|
||||
LegacyNumericQueryNode upperNode = new LegacyNumericQueryNode(
|
||||
termRangeNode.getField(), upperNumber, numberFormat);
|
||||
|
||||
boolean lowerInclusive = termRangeNode.isLowerInclusive();
|
||||
boolean upperInclusive = termRangeNode.isUpperInclusive();
|
||||
|
||||
return new LegacyNumericRangeQueryNode(lowerNode, upperNode,
|
||||
lowerInclusive, upperInclusive, numericConfig);
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return node;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
|
||||
return node;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
|
||||
throws QueryNodeException {
|
||||
return children;
|
||||
}
|
||||
|
||||
}
|
|
@ -53,8 +53,6 @@ public class StandardQueryNodeProcessorPipeline extends
|
|||
add(new RegexpQueryNodeProcessor());
|
||||
add(new MatchAllDocsQueryNodeProcessor());
|
||||
add(new OpenRangeQueryNodeProcessor());
|
||||
add(new LegacyNumericQueryNodeProcessor());
|
||||
add(new LegacyNumericRangeQueryNodeProcessor());
|
||||
add(new PointQueryNodeProcessor());
|
||||
add(new PointRangeQueryNodeProcessor());
|
||||
add(new TermRangeQueryNodeProcessor());
|
||||
|
|
|
@ -71,7 +71,6 @@ public class CoreParser implements QueryBuilder {
|
|||
queryFactory.addBuilder("TermsQuery", new TermsQueryBuilder(analyzer));
|
||||
queryFactory.addBuilder("MatchAllDocsQuery", new MatchAllDocsQueryBuilder());
|
||||
queryFactory.addBuilder("BooleanQuery", new BooleanQueryBuilder(queryFactory));
|
||||
queryFactory.addBuilder("LegacyNumericRangeQuery", new LegacyNumericRangeQueryBuilder());
|
||||
queryFactory.addBuilder("PointRangeQuery", new PointRangeQueryBuilder());
|
||||
queryFactory.addBuilder("RangeQuery", new RangeQueryBuilder());
|
||||
queryFactory.addBuilder("DisjunctionMaxQuery", new DisjunctionMaxQueryBuilder(queryFactory));
|
||||
|
|
|
@ -1,535 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.flexible.standard;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.DateFormat;
|
||||
import java.text.NumberFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.legacy.LegacyDoubleField;
|
||||
import org.apache.lucene.legacy.LegacyFieldType;
|
||||
import org.apache.lucene.legacy.LegacyFloatField;
|
||||
import org.apache.lucene.legacy.LegacyIntField;
|
||||
import org.apache.lucene.legacy.LegacyLongField;
|
||||
import org.apache.lucene.legacy.LegacyNumericType;
|
||||
import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
|
||||
import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.NumberDateFormat;
|
||||
import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
|
||||
import org.apache.lucene.queryparser.flexible.standard.parser.EscapeQuerySyntaxImpl;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestLegacyNumericQueryParser extends LuceneTestCase {
|
||||
|
||||
private static enum NumberType {
|
||||
NEGATIVE, ZERO, POSITIVE;
|
||||
}
|
||||
|
||||
final private static int[] DATE_STYLES = {DateFormat.FULL, DateFormat.LONG,
|
||||
DateFormat.MEDIUM, DateFormat.SHORT};
|
||||
|
||||
final private static int PRECISION_STEP = 8;
|
||||
final private static String FIELD_NAME = "field";
|
||||
private static Locale LOCALE;
|
||||
private static TimeZone TIMEZONE;
|
||||
private static Map<String,Number> RANDOM_NUMBER_MAP;
|
||||
private static EscapeQuerySyntax ESCAPER = new EscapeQuerySyntaxImpl();
|
||||
final private static String DATE_FIELD_NAME = "date";
|
||||
private static int DATE_STYLE;
|
||||
private static int TIME_STYLE;
|
||||
|
||||
private static Analyzer ANALYZER;
|
||||
|
||||
private static NumberFormat NUMBER_FORMAT;
|
||||
|
||||
private static StandardQueryParser qp;
|
||||
|
||||
private static NumberDateFormat DATE_FORMAT;
|
||||
|
||||
private static Directory directory = null;
|
||||
private static IndexReader reader = null;
|
||||
private static IndexSearcher searcher = null;
|
||||
|
||||
private static boolean checkDateFormatSanity(DateFormat dateFormat, long date) {
|
||||
try {
|
||||
return date == dateFormat.parse(dateFormat.format(new Date(date)))
|
||||
.getTime();
|
||||
} catch (ParseException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
ANALYZER = new MockAnalyzer(random());
|
||||
|
||||
qp = new StandardQueryParser(ANALYZER);
|
||||
|
||||
final HashMap<String,Number> randomNumberMap = new HashMap<>();
|
||||
|
||||
SimpleDateFormat dateFormat;
|
||||
long randomDate;
|
||||
boolean dateFormatSanityCheckPass;
|
||||
int count = 0;
|
||||
do {
|
||||
if (count > 100) {
|
||||
fail("This test has problems to find a sane random DateFormat/NumberFormat. Stopped trying after 100 iterations.");
|
||||
}
|
||||
|
||||
dateFormatSanityCheckPass = true;
|
||||
LOCALE = randomLocale(random());
|
||||
TIMEZONE = randomTimeZone(random());
|
||||
DATE_STYLE = randomDateStyle(random());
|
||||
TIME_STYLE = randomDateStyle(random());
|
||||
|
||||
// assumes localized date pattern will have at least year, month, day,
|
||||
// hour, minute
|
||||
dateFormat = (SimpleDateFormat) DateFormat.getDateTimeInstance(
|
||||
DATE_STYLE, TIME_STYLE, LOCALE);
|
||||
|
||||
// not all date patterns includes era, full year, timezone and second,
|
||||
// so we add them here
|
||||
dateFormat.applyPattern(dateFormat.toPattern() + " G s Z yyyy");
|
||||
dateFormat.setTimeZone(TIMEZONE);
|
||||
|
||||
DATE_FORMAT = new NumberDateFormat(dateFormat);
|
||||
|
||||
do {
|
||||
randomDate = random().nextLong();
|
||||
|
||||
// prune date value so it doesn't pass in insane values to some
|
||||
// calendars.
|
||||
randomDate = randomDate % 3400000000000l;
|
||||
|
||||
// truncate to second
|
||||
randomDate = (randomDate / 1000L) * 1000L;
|
||||
|
||||
// only positive values
|
||||
randomDate = Math.abs(randomDate);
|
||||
} while (randomDate == 0L);
|
||||
|
||||
dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat, randomDate);
|
||||
|
||||
dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat, 0);
|
||||
|
||||
dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat,
|
||||
-randomDate);
|
||||
|
||||
count++;
|
||||
} while (!dateFormatSanityCheckPass);
|
||||
|
||||
NUMBER_FORMAT = NumberFormat.getNumberInstance(LOCALE);
|
||||
NUMBER_FORMAT.setMaximumFractionDigits((random().nextInt() & 20) + 1);
|
||||
NUMBER_FORMAT.setMinimumFractionDigits((random().nextInt() & 20) + 1);
|
||||
NUMBER_FORMAT.setMaximumIntegerDigits((random().nextInt() & 20) + 1);
|
||||
NUMBER_FORMAT.setMinimumIntegerDigits((random().nextInt() & 20) + 1);
|
||||
|
||||
double randomDouble;
|
||||
long randomLong;
|
||||
int randomInt;
|
||||
float randomFloat;
|
||||
|
||||
while ((randomLong = normalizeNumber(Math.abs(random().nextLong()))
|
||||
.longValue()) == 0L)
|
||||
;
|
||||
while ((randomDouble = normalizeNumber(Math.abs(random().nextDouble()))
|
||||
.doubleValue()) == 0.0)
|
||||
;
|
||||
while ((randomFloat = normalizeNumber(Math.abs(random().nextFloat()))
|
||||
.floatValue()) == 0.0f)
|
||||
;
|
||||
while ((randomInt = normalizeNumber(Math.abs(random().nextInt())).intValue()) == 0)
|
||||
;
|
||||
|
||||
randomNumberMap.put(LegacyNumericType.LONG.name(), randomLong);
|
||||
randomNumberMap.put(LegacyNumericType.INT.name(), randomInt);
|
||||
randomNumberMap.put(LegacyNumericType.FLOAT.name(), randomFloat);
|
||||
randomNumberMap.put(LegacyNumericType.DOUBLE.name(), randomDouble);
|
||||
randomNumberMap.put(DATE_FIELD_NAME, randomDate);
|
||||
|
||||
RANDOM_NUMBER_MAP = Collections.unmodifiableMap(randomNumberMap);
|
||||
|
||||
directory = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
|
||||
newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000))
|
||||
.setMergePolicy(newLogMergePolicy()));
|
||||
|
||||
Document doc = new Document();
|
||||
HashMap<String,LegacyNumericConfig> numericConfigMap = new HashMap<>();
|
||||
HashMap<String,Field> numericFieldMap = new HashMap<>();
|
||||
qp.setLegacyNumericConfigMap(numericConfigMap);
|
||||
|
||||
for (LegacyNumericType type : LegacyNumericType.values()) {
|
||||
numericConfigMap.put(type.name(), new LegacyNumericConfig(PRECISION_STEP,
|
||||
NUMBER_FORMAT, type));
|
||||
|
||||
LegacyFieldType ft = new LegacyFieldType(LegacyIntField.TYPE_NOT_STORED);
|
||||
ft.setNumericType(type);
|
||||
ft.setStored(true);
|
||||
ft.setNumericPrecisionStep(PRECISION_STEP);
|
||||
ft.freeze();
|
||||
final Field field;
|
||||
|
||||
switch(type) {
|
||||
case INT:
|
||||
field = new LegacyIntField(type.name(), 0, ft);
|
||||
break;
|
||||
case FLOAT:
|
||||
field = new LegacyFloatField(type.name(), 0.0f, ft);
|
||||
break;
|
||||
case LONG:
|
||||
field = new LegacyLongField(type.name(), 0l, ft);
|
||||
break;
|
||||
case DOUBLE:
|
||||
field = new LegacyDoubleField(type.name(), 0.0, ft);
|
||||
break;
|
||||
default:
|
||||
fail();
|
||||
field = null;
|
||||
}
|
||||
numericFieldMap.put(type.name(), field);
|
||||
doc.add(field);
|
||||
}
|
||||
|
||||
numericConfigMap.put(DATE_FIELD_NAME, new LegacyNumericConfig(PRECISION_STEP,
|
||||
DATE_FORMAT, LegacyNumericType.LONG));
|
||||
LegacyFieldType ft = new LegacyFieldType(LegacyLongField.TYPE_NOT_STORED);
|
||||
ft.setStored(true);
|
||||
ft.setNumericPrecisionStep(PRECISION_STEP);
|
||||
LegacyLongField dateField = new LegacyLongField(DATE_FIELD_NAME, 0l, ft);
|
||||
numericFieldMap.put(DATE_FIELD_NAME, dateField);
|
||||
doc.add(dateField);
|
||||
|
||||
for (NumberType numberType : NumberType.values()) {
|
||||
setFieldValues(numberType, numericFieldMap);
|
||||
if (VERBOSE) System.out.println("Indexing document: " + doc);
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
reader = writer.getReader();
|
||||
searcher = newSearcher(reader);
|
||||
writer.close();
|
||||
|
||||
}
|
||||
|
||||
private static Number getNumberType(NumberType numberType, String fieldName) {
|
||||
|
||||
if (numberType == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch (numberType) {
|
||||
|
||||
case POSITIVE:
|
||||
return RANDOM_NUMBER_MAP.get(fieldName);
|
||||
|
||||
case NEGATIVE:
|
||||
Number number = RANDOM_NUMBER_MAP.get(fieldName);
|
||||
|
||||
if (LegacyNumericType.LONG.name().equals(fieldName)
|
||||
|| DATE_FIELD_NAME.equals(fieldName)) {
|
||||
number = -number.longValue();
|
||||
|
||||
} else if (LegacyNumericType.DOUBLE.name().equals(fieldName)) {
|
||||
number = -number.doubleValue();
|
||||
|
||||
} else if (LegacyNumericType.FLOAT.name().equals(fieldName)) {
|
||||
number = -number.floatValue();
|
||||
|
||||
} else if (LegacyNumericType.INT.name().equals(fieldName)) {
|
||||
number = -number.intValue();
|
||||
|
||||
} else {
|
||||
throw new IllegalArgumentException("field name not found: "
|
||||
+ fieldName);
|
||||
}
|
||||
|
||||
return number;
|
||||
|
||||
default:
|
||||
return 0;
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static void setFieldValues(NumberType numberType,
|
||||
HashMap<String,Field> numericFieldMap) {
|
||||
|
||||
Number number = getNumberType(numberType, LegacyNumericType.DOUBLE
|
||||
.name());
|
||||
numericFieldMap.get(LegacyNumericType.DOUBLE.name()).setDoubleValue(
|
||||
number.doubleValue());
|
||||
|
||||
number = getNumberType(numberType, LegacyNumericType.INT.name());
|
||||
numericFieldMap.get(LegacyNumericType.INT.name()).setIntValue(
|
||||
number.intValue());
|
||||
|
||||
number = getNumberType(numberType, LegacyNumericType.LONG.name());
|
||||
numericFieldMap.get(LegacyNumericType.LONG.name()).setLongValue(
|
||||
number.longValue());
|
||||
|
||||
number = getNumberType(numberType, LegacyNumericType.FLOAT.name());
|
||||
numericFieldMap.get(LegacyNumericType.FLOAT.name()).setFloatValue(
|
||||
number.floatValue());
|
||||
|
||||
number = getNumberType(numberType, DATE_FIELD_NAME);
|
||||
numericFieldMap.get(DATE_FIELD_NAME).setLongValue(number.longValue());
|
||||
}
|
||||
|
||||
private static int randomDateStyle(Random random) {
|
||||
return DATE_STYLES[random.nextInt(DATE_STYLES.length)];
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInclusiveNumericRange() throws Exception {
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.ZERO, true, true, 1);
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, true, true, 2);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, true, true, 2);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, true, true, 3);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, true, true, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
// test disabled since standard syntax parser does not work with inclusive and
|
||||
// exclusive at the same time
|
||||
public void testInclusiveLowerNumericRange() throws Exception {
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, false, true, 1);
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, false, true, 1);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, false, true, 2);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, false, true, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
// test disabled since standard syntax parser does not work with inclusive and
|
||||
// exclusive at the same time
|
||||
public void testInclusiveUpperNumericRange() throws Exception {
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, true, false, 1);
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, true, false, 1);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, true, false, 2);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, true, false, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExclusiveNumericRange() throws Exception {
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.ZERO, false, false, 0);
|
||||
assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, false, false, 0);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, false, false, 0);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, false, false, 1);
|
||||
assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, false, false, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOpenRangeNumericQuery() throws Exception {
|
||||
assertOpenRangeQuery(NumberType.ZERO, "<", 1);
|
||||
assertOpenRangeQuery(NumberType.POSITIVE, "<", 2);
|
||||
assertOpenRangeQuery(NumberType.NEGATIVE, "<", 0);
|
||||
|
||||
assertOpenRangeQuery(NumberType.ZERO, "<=", 2);
|
||||
assertOpenRangeQuery(NumberType.POSITIVE, "<=", 3);
|
||||
assertOpenRangeQuery(NumberType.NEGATIVE, "<=", 1);
|
||||
|
||||
assertOpenRangeQuery(NumberType.ZERO, ">", 1);
|
||||
assertOpenRangeQuery(NumberType.POSITIVE, ">", 0);
|
||||
assertOpenRangeQuery(NumberType.NEGATIVE, ">", 2);
|
||||
|
||||
assertOpenRangeQuery(NumberType.ZERO, ">=", 2);
|
||||
assertOpenRangeQuery(NumberType.POSITIVE, ">=", 1);
|
||||
assertOpenRangeQuery(NumberType.NEGATIVE, ">=", 3);
|
||||
|
||||
assertOpenRangeQuery(NumberType.NEGATIVE, "=", 1);
|
||||
assertOpenRangeQuery(NumberType.ZERO, "=", 1);
|
||||
assertOpenRangeQuery(NumberType.POSITIVE, "=", 1);
|
||||
|
||||
assertRangeQuery(NumberType.NEGATIVE, null, true, true, 3);
|
||||
assertRangeQuery(NumberType.NEGATIVE, null, false, true, 2);
|
||||
assertRangeQuery(NumberType.POSITIVE, null, true, false, 1);
|
||||
assertRangeQuery(NumberType.ZERO, null, false, false, 1);
|
||||
|
||||
assertRangeQuery(null, NumberType.POSITIVE, true, true, 3);
|
||||
assertRangeQuery(null, NumberType.POSITIVE, true, false, 2);
|
||||
assertRangeQuery(null, NumberType.NEGATIVE, false, true, 1);
|
||||
assertRangeQuery(null, NumberType.ZERO, false, false, 1);
|
||||
|
||||
assertRangeQuery(null, null, false, false, 3);
|
||||
assertRangeQuery(null, null, true, true, 3);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleNumericQuery() throws Exception {
|
||||
assertSimpleQuery(NumberType.ZERO, 1);
|
||||
assertSimpleQuery(NumberType.POSITIVE, 1);
|
||||
assertSimpleQuery(NumberType.NEGATIVE, 1);
|
||||
}
|
||||
|
||||
public void assertRangeQuery(NumberType lowerType, NumberType upperType,
|
||||
boolean lowerInclusive, boolean upperInclusive, int expectedDocCount)
|
||||
throws QueryNodeException, IOException {
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
String lowerInclusiveStr = (lowerInclusive ? "[" : "{");
|
||||
String upperInclusiveStr = (upperInclusive ? "]" : "}");
|
||||
|
||||
for (LegacyNumericType type : LegacyNumericType.values()) {
|
||||
String lowerStr = numberToString(getNumberType(lowerType, type.name()));
|
||||
String upperStr = numberToString(getNumberType(upperType, type.name()));
|
||||
|
||||
sb.append("+").append(type.name()).append(':').append(lowerInclusiveStr)
|
||||
.append('"').append(lowerStr).append("\" TO \"").append(upperStr)
|
||||
.append('"').append(upperInclusiveStr).append(' ');
|
||||
}
|
||||
|
||||
Number lowerDateNumber = getNumberType(lowerType, DATE_FIELD_NAME);
|
||||
Number upperDateNumber = getNumberType(upperType, DATE_FIELD_NAME);
|
||||
String lowerDateStr;
|
||||
String upperDateStr;
|
||||
|
||||
if (lowerDateNumber != null) {
|
||||
lowerDateStr = ESCAPER.escape(
|
||||
DATE_FORMAT.format(new Date(lowerDateNumber.longValue())), LOCALE,
|
||||
EscapeQuerySyntax.Type.STRING).toString();
|
||||
|
||||
} else {
|
||||
lowerDateStr = "*";
|
||||
}
|
||||
|
||||
if (upperDateNumber != null) {
|
||||
upperDateStr = ESCAPER.escape(
|
||||
DATE_FORMAT.format(new Date(upperDateNumber.longValue())), LOCALE,
|
||||
EscapeQuerySyntax.Type.STRING).toString();
|
||||
|
||||
} else {
|
||||
upperDateStr = "*";
|
||||
}
|
||||
|
||||
sb.append("+").append(DATE_FIELD_NAME).append(':')
|
||||
.append(lowerInclusiveStr).append('"').append(lowerDateStr).append(
|
||||
"\" TO \"").append(upperDateStr).append('"').append(
|
||||
upperInclusiveStr);
|
||||
|
||||
testQuery(sb.toString(), expectedDocCount);
|
||||
|
||||
}
|
||||
|
||||
public void assertOpenRangeQuery(NumberType boundType, String operator, int expectedDocCount)
|
||||
throws QueryNodeException, IOException {
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
for (LegacyNumericType type : LegacyNumericType.values()) {
|
||||
String boundStr = numberToString(getNumberType(boundType, type.name()));
|
||||
|
||||
sb.append("+").append(type.name()).append(operator).append('"').append(boundStr).append('"').append(' ');
|
||||
}
|
||||
|
||||
String boundDateStr = ESCAPER.escape(
|
||||
DATE_FORMAT.format(new Date(getNumberType(boundType, DATE_FIELD_NAME)
|
||||
.longValue())), LOCALE, EscapeQuerySyntax.Type.STRING).toString();
|
||||
|
||||
sb.append("+").append(DATE_FIELD_NAME).append(operator).append('"').append(boundDateStr).append('"');
|
||||
|
||||
testQuery(sb.toString(), expectedDocCount);
|
||||
}
|
||||
|
||||
public void assertSimpleQuery(NumberType numberType, int expectedDocCount)
|
||||
throws QueryNodeException, IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
for (LegacyNumericType type : LegacyNumericType.values()) {
|
||||
String numberStr = numberToString(getNumberType(numberType, type.name()));
|
||||
sb.append('+').append(type.name()).append(":\"").append(numberStr)
|
||||
.append("\" ");
|
||||
}
|
||||
|
||||
String dateStr = ESCAPER.escape(
|
||||
DATE_FORMAT.format(new Date(getNumberType(numberType, DATE_FIELD_NAME)
|
||||
.longValue())), LOCALE, EscapeQuerySyntax.Type.STRING).toString();
|
||||
|
||||
sb.append('+').append(DATE_FIELD_NAME).append(":\"").append(dateStr)
|
||||
.append('"');
|
||||
|
||||
testQuery(sb.toString(), expectedDocCount);
|
||||
|
||||
}
|
||||
|
||||
private void testQuery(String queryStr, int expectedDocCount)
|
||||
throws QueryNodeException, IOException {
|
||||
if (VERBOSE) System.out.println("Parsing: " + queryStr);
|
||||
|
||||
Query query = qp.parse(queryStr, FIELD_NAME);
|
||||
if (VERBOSE) System.out.println("Querying: " + query);
|
||||
TopDocs topDocs = searcher.search(query, 1000);
|
||||
|
||||
String msg = "Query <" + queryStr + "> retrieved " + topDocs.totalHits
|
||||
+ " document(s), " + expectedDocCount + " document(s) expected.";
|
||||
|
||||
if (VERBOSE) System.out.println(msg);
|
||||
|
||||
assertEquals(msg, expectedDocCount, topDocs.totalHits);
|
||||
}
|
||||
|
||||
private static String numberToString(Number number) {
|
||||
return number == null ? "*" : ESCAPER.escape(NUMBER_FORMAT.format(number),
|
||||
LOCALE, EscapeQuerySyntax.Type.STRING).toString();
|
||||
}
|
||||
|
||||
private static Number normalizeNumber(Number number) throws ParseException {
|
||||
return NUMBER_FORMAT.parse(NUMBER_FORMAT.format(number));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
searcher = null;
|
||||
reader.close();
|
||||
reader = null;
|
||||
directory.close();
|
||||
directory = null;
|
||||
qp = null;
|
||||
LOCALE = null;
|
||||
TIMEZONE = null;
|
||||
NUMBER_FORMAT = null;
|
||||
DATE_FORMAT = null;
|
||||
ESCAPER = null;
|
||||
}
|
||||
|
||||
}
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.document.IntPoint;
|
|||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.legacy.LegacyIntField;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -53,7 +52,6 @@ class CoreParserTestIndexData implements Closeable {
|
|||
Document doc = new Document();
|
||||
doc.add(LuceneTestCase.newTextField("date", date, Field.Store.YES));
|
||||
doc.add(LuceneTestCase.newTextField("contents", content, Field.Store.YES));
|
||||
doc.add(new LegacyIntField("date2", Integer.valueOf(date), Field.Store.NO));
|
||||
doc.add(new IntPoint("date3", Integer.valueOf(date)));
|
||||
writer.addDocument(doc);
|
||||
line = d.readLine();
|
||||
|
|
|
@ -1,31 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<BooleanQuery fieldName="contents">
|
||||
<Clause occurs="should">
|
||||
<TermQuery>merger</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="mustnot">
|
||||
<TermQuery >sumitomo</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<TermQuery>bank</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<LegacyNumericRangeQuery fieldName="date2" lowerTerm="19870409" upperTerm="19870412"/>
|
||||
</Clause>
|
||||
</BooleanQuery>
|
|
@ -1,31 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<BooleanQuery fieldName="contents">
|
||||
<Clause occurs="should">
|
||||
<TermQuery>merger</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="mustnot">
|
||||
<TermQuery >sumitomo</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<TermQuery>bank</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<LegacyNumericRangeQuery fieldName="date2" upperTerm="19870412"/>
|
||||
</Clause>
|
||||
</BooleanQuery>
|
|
@ -1,31 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<BooleanQuery fieldName="contents">
|
||||
<Clause occurs="should">
|
||||
<TermQuery>merger</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="mustnot">
|
||||
<TermQuery >sumitomo</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<TermQuery>bank</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<LegacyNumericRangeQuery fieldName="date2"/>
|
||||
</Clause>
|
||||
</BooleanQuery>
|
|
@ -1,31 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<BooleanQuery fieldName="contents">
|
||||
<Clause occurs="should">
|
||||
<TermQuery>merger</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="mustnot">
|
||||
<TermQuery >sumitomo</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<TermQuery>bank</TermQuery>
|
||||
</Clause>
|
||||
<Clause occurs="must">
|
||||
<LegacyNumericRangeQuery fieldName="date2" lowerTerm="19870409"/>
|
||||
</Clause>
|
||||
</BooleanQuery>
|
|
@ -133,26 +133,6 @@ public class TestCoreParser extends LuceneTestCase {
|
|||
dumpResults("Nested Boolean query", q, 5);
|
||||
}
|
||||
|
||||
public void testNumericRangeQueryXML() throws ParserException, IOException {
|
||||
Query q = parse("LegacyNumericRangeQuery.xml");
|
||||
dumpResults("LegacyNumericRangeQuery", q, 5);
|
||||
}
|
||||
|
||||
public void testNumericRangeQueryXMLWithoutLowerTerm() throws ParserException, IOException {
|
||||
Query q = parse("LegacyNumericRangeQueryWithoutLowerTerm.xml");
|
||||
dumpResults("LegacyNumericRangeQueryWithoutLowerTerm", q, 5);
|
||||
}
|
||||
|
||||
public void testNumericRangeQueryXMLWithoutUpperTerm() throws ParserException, IOException {
|
||||
Query q = parse("LegacyNumericRangeQueryWithoutUpperTerm.xml");
|
||||
dumpResults("LegacyNumericRangeQueryWithoutUpperTerm", q, 5);
|
||||
}
|
||||
|
||||
public void testNumericRangeQueryXMLWithoutRange() throws ParserException, IOException {
|
||||
Query q = parse("LegacyNumericRangeQueryWithoutRange.xml");
|
||||
dumpResults("LegacyNumericRangeQueryWithoutRange", q, 5);
|
||||
}
|
||||
|
||||
public void testPointRangeQuery() throws ParserException, IOException {
|
||||
Query q = parse("PointRangeQuery.xml");
|
||||
dumpResults("PointRangeQuery", q, 5);
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.StringHelper;
|
|||
* <li>{@link #newExactQuery(String, InetAddress)} for matching an exact network address.
|
||||
* <li>{@link #newPrefixQuery(String, InetAddress, int)} for matching a network based on CIDR prefix.
|
||||
* <li>{@link #newRangeQuery(String, InetAddress, InetAddress)} for matching arbitrary network address ranges.
|
||||
* <li>{@link #newSetQuery(String, InetAddress...)} for matching a set of 1D values.
|
||||
* <li>{@link #newSetQuery(String, InetAddress...)} for matching a set of network addresses.
|
||||
* </ul>
|
||||
* <p>
|
||||
* This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
|
||||
|
|
|
@ -290,36 +290,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testStringSet() throws Exception {
|
||||
Directory dir = getDirectory(createTempDir("testStringSet"));
|
||||
IndexOutput output = dir.createOutput("stringset", newIOContext(random()));
|
||||
output.writeStringSet(asSet("test1", "test2"));
|
||||
output.close();
|
||||
|
||||
IndexInput input = dir.openInput("stringset", newIOContext(random()));
|
||||
assertEquals(16, input.length());
|
||||
assertEquals(asSet("test1", "test2"), input.readStringSet());
|
||||
input.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testStringMap() throws Exception {
|
||||
Map<String,String> m = new HashMap<>();
|
||||
m.put("test1", "value1");
|
||||
m.put("test2", "value2");
|
||||
|
||||
Directory dir = getDirectory(createTempDir("testStringMap"));
|
||||
IndexOutput output = dir.createOutput("stringmap", newIOContext(random()));
|
||||
output.writeStringStringMap(m);
|
||||
output.close();
|
||||
|
||||
IndexInput input = dir.openInput("stringmap", newIOContext(random()));
|
||||
assertEquals(30, input.length());
|
||||
assertEquals(m, input.readStringStringMap());
|
||||
input.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testSetOfStrings() throws Exception {
|
||||
Directory dir = getDirectory(createTempDir("testSetOfStrings"));
|
||||
|
||||
|
|
|
@ -177,12 +177,6 @@ public class MockIndexInputWrapper extends IndexInput {
|
|||
return delegate.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String,String> readStringStringMap() throws IOException {
|
||||
ensureOpen();
|
||||
return delegate.readStringStringMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readVInt() throws IOException {
|
||||
ensureOpen();
|
||||
|
@ -207,18 +201,24 @@ public class MockIndexInputWrapper extends IndexInput {
|
|||
return delegate.readZLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> readStringSet() throws IOException {
|
||||
ensureOpen();
|
||||
return delegate.readStringSet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void skipBytes(long numBytes) throws IOException {
|
||||
ensureOpen();
|
||||
super.skipBytes(numBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String,String> readMapOfStrings() throws IOException {
|
||||
ensureOpen();
|
||||
return delegate.readMapOfStrings();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> readSetOfStrings() throws IOException {
|
||||
ensureOpen();
|
||||
return delegate.readSetOfStrings();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MockIndexInputWrapper(" + delegate + ")";
|
||||
|
|
|
@ -32,6 +32,7 @@ grant {
|
|||
permission java.io.FilePermission "${junit4.childvm.cwd}${/}jacoco.db", "write";
|
||||
permission java.io.FilePermission "${junit4.tempDir}${/}*", "read,execute,write,delete";
|
||||
permission java.io.FilePermission "${clover.db.dir}${/}-", "read,execute,write,delete";
|
||||
permission java.io.FilePermission "${tests.linedocsfile}", "read";
|
||||
|
||||
// all possibilities of accepting/binding connections on localhost with ports >=1024:
|
||||
permission java.net.SocketPermission "localhost:1024-", "accept,listen";
|
||||
|
|
|
@ -34,6 +34,7 @@ grant {
|
|||
permission java.io.FilePermission "${junit4.childvm.cwd}${/}jacoco.db", "write";
|
||||
permission java.io.FilePermission "${junit4.tempDir}${/}*", "read,write,delete";
|
||||
permission java.io.FilePermission "${clover.db.dir}${/}-", "read,write,delete";
|
||||
permission java.io.FilePermission "${tests.linedocsfile}", "read";
|
||||
|
||||
// misc HardlinkCopyDirectoryWrapper needs this to test if hardlinks can be created
|
||||
permission java.nio.file.LinkPermission "hard";
|
||||
|
|
|
@ -63,9 +63,29 @@ Apache UIMA 2.3.1
|
|||
Apache ZooKeeper 3.4.6
|
||||
Jetty 9.3.8.v20160314
|
||||
|
||||
Detailed Change List
|
||||
----------------------
|
||||
|
||||
(No Changes)
|
||||
New Features
|
||||
----------------------
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
* SOLR-9310: PeerSync fails on a node restart due to IndexFingerPrint mismatch (Pushkar Raste, noble)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
* SOLR-7362: Fix TestReqParamsAPI test failures (noble, Varun Thacker)
|
||||
|
||||
* SOLR-9412: Add failOnMissingParams option to MacroExpander, add TestMacroExpander class.
|
||||
(Jon Dorando, Christine Poerschke)
|
||||
|
||||
* SOLR-9406: SolrSuggester should selectively register close hook (Gethin James, Joel Bernstein)
|
||||
|
||||
================== 6.2.0 ==================
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.Map;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.solr.cloud.ZkSolrResourceLoader;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.params.MapSolrParams;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -156,10 +157,8 @@ public class RequestParams implements MapSerializable {
|
|||
requestParams = new RequestParams((Map) o[0], (Integer) o[1]);
|
||||
log.info("request params refreshed to version {}", requestParams.getZnodeVersion());
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (KeeperException | InterruptedException e) {
|
||||
SolrZkClient.checkInterrupted(e);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
}
|
||||
|
||||
|
|
|
@ -549,6 +549,24 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
// protect via synchronized(SolrCore.class)
|
||||
private static Set<String> dirs = new HashSet<>();
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the index in the named directory is
|
||||
* currently locked.
|
||||
* @param directory the directory to check for a lock
|
||||
* @throws IOException if there is a low-level IO error
|
||||
* @deprecated Use of this method can only lead to race conditions. Try
|
||||
* to actually obtain a lock instead.
|
||||
*/
|
||||
@Deprecated
|
||||
private static boolean isWriterLocked(Directory directory) throws IOException {
|
||||
try {
|
||||
directory.obtainLock(IndexWriter.WRITE_LOCK_NAME).close();
|
||||
return false;
|
||||
} catch (LockObtainFailedException failed) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
void initIndex(boolean reload) throws IOException {
|
||||
|
||||
String indexDir = getNewIndexDir();
|
||||
|
@ -564,7 +582,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
final String lockType = getSolrConfig().indexConfig.lockType;
|
||||
Directory dir = directoryFactory.get(indexDir, DirContext.DEFAULT, lockType);
|
||||
try {
|
||||
if (IndexWriter.isLocked(dir)) {
|
||||
if (isWriterLocked(dir)) {
|
||||
log.error(logid + "Solr index directory '{}' is locked (lockType={}). Throwing exception.",
|
||||
indexDir, lockType);
|
||||
throw new LockObtainFailedException
|
||||
|
|
|
@ -188,12 +188,12 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
|
|||
resp.add(ZNODEVER, Utils.makeMap(
|
||||
ConfigOverlay.NAME, req.getCore().getSolrConfig().getOverlay().getZnodeVersion(),
|
||||
RequestParams.NAME, req.getCore().getSolrConfig().getRequestParams().getZnodeVersion()));
|
||||
boolean checkStale = false;
|
||||
boolean isStale = false;
|
||||
int expectedVersion = req.getParams().getInt(ConfigOverlay.NAME, -1);
|
||||
int actualVersion = req.getCore().getSolrConfig().getOverlay().getZnodeVersion();
|
||||
if (expectedVersion > actualVersion) {
|
||||
log.info("expecting overlay version {} but my version is {}", expectedVersion, actualVersion);
|
||||
checkStale = true;
|
||||
isStale = true;
|
||||
} else if (expectedVersion != -1) {
|
||||
log.info("I already have the expected version {} of config", expectedVersion);
|
||||
}
|
||||
|
@ -201,11 +201,11 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
|
|||
actualVersion = req.getCore().getSolrConfig().getRequestParams().getZnodeVersion();
|
||||
if (expectedVersion > actualVersion) {
|
||||
log.info("expecting params version {} but my version is {}", expectedVersion, actualVersion);
|
||||
checkStale = true;
|
||||
isStale = true;
|
||||
} else if (expectedVersion != -1) {
|
||||
log.info("I already have the expected version {} of params", expectedVersion);
|
||||
}
|
||||
if (checkStale && req.getCore().getResourceLoader() instanceof ZkSolrResourceLoader) {
|
||||
if (isStale && req.getCore().getResourceLoader() instanceof ZkSolrResourceLoader) {
|
||||
new Thread(() -> {
|
||||
if (!reloadLock.tryLock()) {
|
||||
log.info("Another reload is in progress . Not doing anything");
|
||||
|
@ -221,7 +221,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
|
|||
}
|
||||
}, SolrConfigHandler.class.getSimpleName() + "-refreshconf").start();
|
||||
} else {
|
||||
log.info("checkStale {} , resourceloader {}", checkStale, req.getCore().getResourceLoader().getClass().getName());
|
||||
log.info("isStale {} , resourceloader {}", isStale, req.getCore().getResourceLoader().getClass().getName());
|
||||
}
|
||||
|
||||
} else {
|
||||
|
@ -287,7 +287,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
|
|||
|
||||
for (Map.Entry<String, Object> entry : map.entrySet()) {
|
||||
|
||||
Map val = null;
|
||||
Map val;
|
||||
String key = entry.getKey();
|
||||
if (isNullOrEmpty(key)) {
|
||||
op.addError("null key ");
|
||||
|
@ -354,16 +354,13 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
|
|||
if (ops.isEmpty()) {
|
||||
ZkController.touchConfDir(zkLoader);
|
||||
} else {
|
||||
log.info("persisting params data : {}", Utils.toJSONString(params.toMap()));
|
||||
log.debug("persisting params data : {}", Utils.toJSONString(params.toMap()));
|
||||
int latestVersion = ZkController.persistConfigResourceToZooKeeper(zkLoader,
|
||||
params.getZnodeVersion(),
|
||||
RequestParams.RESOURCE,
|
||||
params.toByteArray(), true);
|
||||
log.info("persisted to version : {} ", latestVersion);
|
||||
params.getZnodeVersion(), RequestParams.RESOURCE, params.toByteArray(), true);
|
||||
|
||||
log.debug("persisted to version : {} ", latestVersion);
|
||||
waitForAllReplicasState(req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName(),
|
||||
req.getCore().getCoreDescriptor().getCoreContainer().getZkController(),
|
||||
RequestParams.NAME,
|
||||
latestVersion, 30);
|
||||
req.getCore().getCoreDescriptor().getCoreContainer().getZkController(), RequestParams.NAME, latestVersion, 30);
|
||||
}
|
||||
|
||||
} else {
|
||||
|
|
|
@ -627,14 +627,17 @@ public class RealTimeGetComponent extends SearchComponent
|
|||
UpdateLog ulog = req.getCore().getUpdateHandler().getUpdateLog();
|
||||
if (ulog == null) return;
|
||||
|
||||
try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
|
||||
rb.rsp.add("versions", recentUpdates.getVersions(nVersions));
|
||||
}
|
||||
|
||||
// get fingerprint first as it will cause a soft commit
|
||||
// and would avoid mismatch if documents are being actively index especially during PeerSync
|
||||
if (doFingerprint) {
|
||||
IndexFingerprint fingerprint = IndexFingerprint.getFingerprint(req.getCore(), Long.MAX_VALUE);
|
||||
rb.rsp.add("fingerprint", fingerprint.toObject());
|
||||
}
|
||||
|
||||
try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
|
||||
List<Long> versions = recentUpdates.getVersions(nVersions);
|
||||
rb.rsp.add("versions", versions);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -687,6 +690,13 @@ public class RealTimeGetComponent extends SearchComponent
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
// find fingerprint for max version for which updates are requested
|
||||
boolean doFingerprint = params.getBool("fingerprint", false);
|
||||
if (doFingerprint) {
|
||||
long maxVersionForUpdate = Collections.min(versions, PeerSync.absComparator);
|
||||
IndexFingerprint fingerprint = IndexFingerprint.getFingerprint(req.getCore(), Math.abs(maxVersionForUpdate));
|
||||
rb.rsp.add("fingerprint", fingerprint.toObject());
|
||||
}
|
||||
|
||||
List<Object> updates = new ArrayList<>(versions.size());
|
||||
|
||||
|
|
|
@ -34,10 +34,16 @@ public class MacroExpander {
|
|||
private String macroStart = MACRO_START;
|
||||
private char escape = '\\';
|
||||
private int level;
|
||||
private final boolean failOnMissingParams;
|
||||
|
||||
|
||||
public MacroExpander(Map<String,String[]> orig) {
|
||||
this(orig, false);
|
||||
}
|
||||
|
||||
public MacroExpander(Map<String,String[]> orig, boolean failOnMissingParams) {
|
||||
this.orig = orig;
|
||||
this.failOnMissingParams = failOnMissingParams;
|
||||
}
|
||||
|
||||
public static Map<String,String[]> expand(Map<String,String[]> params) {
|
||||
|
@ -163,8 +169,14 @@ public class MacroExpander {
|
|||
String replacement = replacementList!=null ? replacementList[0] : defVal;
|
||||
if (replacement != null) {
|
||||
String expandedReplacement = expand(replacement);
|
||||
if (failOnMissingParams && expandedReplacement == null) {
|
||||
return null;
|
||||
}
|
||||
sb.append(expandedReplacement);
|
||||
}
|
||||
else if (failOnMissingParams) {
|
||||
return null;
|
||||
}
|
||||
|
||||
} catch (SyntaxError syntaxError) {
|
||||
// append the part we would have skipped
|
||||
|
|
|
@ -84,7 +84,7 @@ public abstract class ManagedResourceStorage {
|
|||
* whether the core is running in cloud mode as well as initArgs.
|
||||
*/
|
||||
public static StorageIO newStorageIO(String collection, SolrResourceLoader resourceLoader, NamedList<String> initArgs) {
|
||||
StorageIO storageIO = null;
|
||||
StorageIO storageIO;
|
||||
|
||||
SolrZkClient zkClient = null;
|
||||
String zkConfigName = null;
|
||||
|
@ -94,8 +94,9 @@ public abstract class ManagedResourceStorage {
|
|||
zkConfigName = ((ZkSolrResourceLoader)resourceLoader).getZkController().
|
||||
getZkStateReader().readConfigName(collection);
|
||||
} catch (Exception e) {
|
||||
log.error("Failed to get config name for collection {} due to: {}",
|
||||
collection, e.toString());
|
||||
log.error("Failed to get config name due to", e);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Failed to load config name for collection:" + collection + " due to: ", e);
|
||||
}
|
||||
if (zkConfigName == null) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.xml.builders;
|
||||
package org.apache.solr.search;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.legacy.LegacyNumericRangeQuery;
|
||||
|
@ -22,6 +22,7 @@ import org.apache.lucene.legacy.LegacyNumericUtils;
|
|||
import org.apache.lucene.queryparser.xml.DOMUtils;
|
||||
import org.apache.lucene.queryparser.xml.ParserException;
|
||||
import org.apache.lucene.queryparser.xml.QueryBuilder;
|
||||
import org.apache.lucene.queryparser.xml.builders.PointRangeQueryBuilder;
|
||||
import org.w3c.dom.Element;
|
||||
|
||||
/**
|
|
@ -21,7 +21,6 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.queryparser.xml.CoreParser;
|
||||
import org.apache.lucene.queryparser.xml.QueryBuilder;
|
||||
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.core.SolrResourceLoader;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
|
@ -38,6 +37,7 @@ public class SolrCoreParser extends CoreParser implements NamedListInitializedPl
|
|||
public SolrCoreParser(String defaultField, Analyzer analyzer,
|
||||
SolrQueryRequest req) {
|
||||
super(defaultField, analyzer);
|
||||
queryFactory.addBuilder("LegacyNumericRangeQuery", new LegacyNumericRangeQueryBuilder());
|
||||
this.req = req;
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,7 @@ import java.util.LinkedList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
|
@ -147,8 +148,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
|
|||
private final String path;
|
||||
private boolean releaseDirectory;
|
||||
|
||||
private volatile IndexFingerprint fingerprint;
|
||||
private final Object fingerprintLock = new Object();
|
||||
private final Map<Long, IndexFingerprint> maxVersionFingerprintCache = new ConcurrentHashMap<>();
|
||||
|
||||
private static DirectoryReader getReader(SolrCore core, SolrIndexConfig config, DirectoryFactory directoryFactory,
|
||||
String path) throws IOException {
|
||||
|
@ -2379,15 +2379,17 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
|
|||
* gets a cached version of the IndexFingerprint for this searcher
|
||||
**/
|
||||
public IndexFingerprint getIndexFingerprint(long maxVersion) throws IOException {
|
||||
IndexFingerprint fingerprint = maxVersionFingerprintCache.get(maxVersion);
|
||||
if (fingerprint != null) return fingerprint;
|
||||
// possibly expensive, so prevent more than one thread from calculating it for this searcher
|
||||
synchronized (fingerprintLock) {
|
||||
if (fingerprint == null) {
|
||||
synchronized (maxVersionFingerprintCache) {
|
||||
fingerprint = maxVersionFingerprintCache.get(maxVersionFingerprintCache);
|
||||
if (fingerprint != null) return fingerprint;
|
||||
fingerprint = IndexFingerprint.getFingerprint(this, maxVersion);
|
||||
}
|
||||
}
|
||||
|
||||
maxVersionFingerprintCache.put(maxVersion, fingerprint);
|
||||
return fingerprint;
|
||||
}
|
||||
}
|
||||
|
||||
/////////////////////////////////////////////////////////////////////
|
||||
// SolrInfoMBean stuff: Statistics and Module Info
|
||||
|
|
|
@ -24,38 +24,45 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.schema.FieldType;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
|
||||
|
||||
public class FacetField extends FacetRequest {
|
||||
String field;
|
||||
// Any type of facet request that generates a variable number of buckets
|
||||
// and the ability to sort by those generated buckets.
|
||||
abstract class FacetRequestSorted extends FacetRequest {
|
||||
long offset;
|
||||
long limit = 10;
|
||||
long mincount = 1;
|
||||
long limit;
|
||||
long mincount;
|
||||
String sortVariable;
|
||||
SortDirection sortDirection;
|
||||
RefineMethod refine; // null, NONE, or SIMPLE
|
||||
|
||||
@Override
|
||||
public RefineMethod getRefineMethod() {
|
||||
return refine;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean returnsPartial() {
|
||||
return limit > 0;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
public class FacetField extends FacetRequestSorted {
|
||||
String field;
|
||||
boolean missing;
|
||||
boolean allBuckets; // show cumulative stats across all buckets (this can be different than non-bucketed stats across all docs because of multi-valued docs)
|
||||
boolean numBuckets;
|
||||
String prefix;
|
||||
String sortVariable;
|
||||
SortDirection sortDirection;
|
||||
FacetMethod method;
|
||||
int cacheDf; // 0 means "default", -1 means "never cache"
|
||||
|
||||
// experimental - force perSeg collection when using dv method, currently for testing purposes only.
|
||||
Boolean perSeg;
|
||||
|
||||
// TODO: put this somewhere more generic?
|
||||
public enum SortDirection {
|
||||
asc(-1) ,
|
||||
desc(1);
|
||||
|
||||
private final int multiplier;
|
||||
private SortDirection(int multiplier) {
|
||||
this.multiplier = multiplier;
|
||||
}
|
||||
|
||||
// asc==-1, desc==1
|
||||
public int getMultiplier() {
|
||||
return multiplier;
|
||||
}
|
||||
{
|
||||
// defaults for FacetRequestSorted
|
||||
mincount = 1;
|
||||
limit = 10;
|
||||
}
|
||||
|
||||
public enum FacetMethod {
|
||||
|
|
|
@ -0,0 +1,211 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
|
||||
|
||||
// TODO: refactor more out to base class
|
||||
public class FacetFieldMerger extends FacetRequestSortedMerger<FacetField> {
|
||||
FacetBucket missingBucket;
|
||||
FacetBucket allBuckets;
|
||||
FacetMerger numBuckets;
|
||||
int[] numReturnedPerShard;
|
||||
|
||||
// LinkedHashMap<Object,FacetBucket> buckets = new LinkedHashMap<>();
|
||||
// List<FacetBucket> sortedBuckets;
|
||||
int numReturnedBuckets; // the number of buckets in the bucket lists returned from all of the shards
|
||||
|
||||
|
||||
public FacetFieldMerger(FacetField freq) {
|
||||
super(freq);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
if (numReturnedPerShard == null) {
|
||||
numReturnedPerShard = new int[mcontext.numShards];
|
||||
}
|
||||
merge((SimpleOrderedMap)facetResult, mcontext);
|
||||
}
|
||||
|
||||
protected void merge(SimpleOrderedMap facetResult, Context mcontext) {
|
||||
if (freq.missing) {
|
||||
Object o = facetResult.get("missing");
|
||||
if (o != null) {
|
||||
if (missingBucket == null) {
|
||||
missingBucket = newBucket(null, mcontext);
|
||||
}
|
||||
missingBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
if (freq.allBuckets) {
|
||||
Object o = facetResult.get("allBuckets");
|
||||
if (o != null) {
|
||||
if (allBuckets == null) {
|
||||
allBuckets = newBucket(null, mcontext);
|
||||
}
|
||||
allBuckets.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
|
||||
numReturnedPerShard[mcontext.shardNum] = bucketList.size();
|
||||
numReturnedBuckets += bucketList.size();
|
||||
mergeBucketList(bucketList , mcontext);
|
||||
|
||||
if (freq.numBuckets) {
|
||||
Object nb = facetResult.get("numBuckets");
|
||||
if (nb != null) {
|
||||
if (numBuckets == null) {
|
||||
numBuckets = new FacetNumBucketsMerger();
|
||||
}
|
||||
numBuckets.merge(nb , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
SimpleOrderedMap result = new SimpleOrderedMap();
|
||||
|
||||
if (numBuckets != null) {
|
||||
int removed = 0;
|
||||
if (freq.mincount > 1) {
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
if (bucket.count < freq.mincount) removed++;
|
||||
}
|
||||
}
|
||||
result.add("numBuckets", ((Number)numBuckets.getMergedResult()).longValue() - removed);
|
||||
|
||||
// TODO: we can further increase this estimate.
|
||||
// If not sorting by count, use a simple ratio to scale
|
||||
// If sorting by count desc, then add up the highest_possible_missing_count from each shard
|
||||
}
|
||||
|
||||
sortBuckets();
|
||||
|
||||
int first = (int)freq.offset;
|
||||
int end = freq.limit >=0 ? first + (int) freq.limit : Integer.MAX_VALUE;
|
||||
int last = Math.min(sortedBuckets.size(), end);
|
||||
|
||||
List<SimpleOrderedMap> resultBuckets = new ArrayList<>(Math.max(0, (last - first)));
|
||||
|
||||
/** this only works if there are no filters (like mincount)
|
||||
for (int i=first; i<last; i++) {
|
||||
FacetBucket bucket = sortedBuckets.get(i);
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
}
|
||||
***/
|
||||
|
||||
// TODO: change effective offsets + limits at shards...
|
||||
|
||||
int off = (int)freq.offset;
|
||||
int lim = freq.limit >= 0 ? (int)freq.limit : Integer.MAX_VALUE;
|
||||
for (FacetBucket bucket : sortedBuckets) {
|
||||
if (bucket.getCount() < freq.mincount) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (off > 0) {
|
||||
--off;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (resultBuckets.size() >= lim) {
|
||||
break;
|
||||
}
|
||||
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
}
|
||||
|
||||
|
||||
result.add("buckets", resultBuckets);
|
||||
if (missingBucket != null) {
|
||||
result.add("missing", missingBucket.getMergedBucket());
|
||||
}
|
||||
if (allBuckets != null) {
|
||||
result.add("allBuckets", allBuckets.getMergedBucket());
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void finish(Context mcontext) {
|
||||
// TODO: check refine of subs?
|
||||
// TODO: call subs each time with a shard/shardnum that is missing a bucket at this level?
|
||||
// or pass a bit vector of shards w/ value???
|
||||
|
||||
// build up data structure and only then call the context (or whatever) to do the refinement?
|
||||
// basically , only do at the top-level facet?
|
||||
}
|
||||
|
||||
|
||||
|
||||
private class FacetNumBucketsMerger extends FacetMerger {
|
||||
long sumBuckets;
|
||||
long shardsMissingSum;
|
||||
long shardsTruncatedSum;
|
||||
Set<Object> values;
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
|
||||
long numBuckets = ((Number)map.get("numBuckets")).longValue();
|
||||
sumBuckets += numBuckets;
|
||||
|
||||
List vals = (List)map.get("vals");
|
||||
if (vals != null) {
|
||||
if (values == null) {
|
||||
values = new HashSet<>(vals.size()*4);
|
||||
}
|
||||
values.addAll(vals);
|
||||
if (numBuckets > values.size()) {
|
||||
shardsTruncatedSum += numBuckets - values.size();
|
||||
}
|
||||
} else {
|
||||
shardsMissingSum += numBuckets;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(Context mcontext) {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
long exactCount = values == null ? 0 : values.size();
|
||||
return exactCount + shardsMissingSum + shardsTruncatedSum;
|
||||
// TODO: reduce count by (at least) number of buckets that fail to hit mincount (after merging)
|
||||
// that should make things match for most of the small tests at least
|
||||
}
|
||||
}
|
||||
}
|
|
@ -16,17 +16,131 @@
|
|||
*/
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
//
|
||||
// The FacetMerger code is in the prototype stage, and this is the reason that
|
||||
// many implementations are all in this file. They can be moved to separate
|
||||
// files after the interfaces are locked down more.
|
||||
//
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.solr.search.facet.FacetRequest.RefineMethod.SIMPLE;
|
||||
|
||||
|
||||
public abstract class FacetMerger {
|
||||
public abstract void merge(Object facetResult, Context mcontext);
|
||||
|
||||
// FIXME
|
||||
// public abstract Map<String,Object> getRefinement(Context mcontext);
|
||||
public Map<String,Object> getRefinement(Context mcontext) {
|
||||
return null;
|
||||
}
|
||||
public abstract void finish(Context mcontext);
|
||||
public abstract Object getMergedResult();
|
||||
|
||||
// This class lets mergers know overall context such as what shard is being merged
|
||||
// and what buckets have been seen by what shard.
|
||||
public static class Context {
|
||||
// FacetComponentState state; // todo: is this needed?
|
||||
Object root;
|
||||
final int numShards;
|
||||
private final BitSet sawShard = new BitSet(); // [bucket0_shard0, bucket0_shard1, bucket0_shard2, bucket1_shard0, bucket1_shard1, bucket1_shard2]
|
||||
private Map<String,Integer> shardmap = new HashMap<>();
|
||||
|
||||
public Context(int numShards) {
|
||||
this.numShards = numShards;
|
||||
}
|
||||
|
||||
Object root; // per-shard response
|
||||
int maxBucket; // the current max bucket across all bucket types... incremented as we encounter more
|
||||
int shardNum = -1; // TODO: keep same mapping across multiple phases...
|
||||
boolean bucketWasMissing;
|
||||
|
||||
public void newShard(String shard) {
|
||||
Integer prev = shardmap.put(shard, ++shardNum);
|
||||
assert prev == null;
|
||||
this.bucketWasMissing = false;
|
||||
}
|
||||
|
||||
public void setShard(String shard) {
|
||||
this.shardNum = shardmap.get(shard);
|
||||
}
|
||||
|
||||
public int getNewBucketNumber() {
|
||||
return maxBucket++;
|
||||
}
|
||||
|
||||
public void setShardFlag(int bucketNum) {
|
||||
// rely on normal bitset expansion (uses a doubling strategy)
|
||||
sawShard.set( bucketNum * numShards + shardNum );
|
||||
}
|
||||
|
||||
public boolean getShardFlag(int bucketNum) {
|
||||
return sawShard.get( bucketNum * numShards + shardNum );
|
||||
}
|
||||
|
||||
public boolean bucketWasMissing() {
|
||||
return bucketWasMissing;
|
||||
}
|
||||
|
||||
public boolean setBucketWasMissing(boolean newVal) {
|
||||
boolean oldVal = bucketWasMissing();
|
||||
bucketWasMissing = newVal;
|
||||
return oldVal;
|
||||
}
|
||||
|
||||
private Map<FacetRequest, Collection<String>> refineSubMap = new IdentityHashMap<>(4);
|
||||
public Collection<String> getSubsWithRefinement(FacetRequest freq) {
|
||||
if (freq.getSubFacets().isEmpty()) return Collections.emptyList();
|
||||
Collection<String> subs = refineSubMap.get(freq);
|
||||
if (subs != null) return subs;
|
||||
|
||||
for (Map.Entry<String,FacetRequest> entry : freq.subFacets.entrySet()) {
|
||||
Collection<String> childSubs = getSubsWithRefinement(entry.getValue());
|
||||
if (childSubs.size() > 0 || entry.getValue().getRefineMethod() == SIMPLE) {
|
||||
if (subs == null) {
|
||||
subs = new ArrayList<>(freq.getSubFacets().size());
|
||||
}
|
||||
subs.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
if (subs == null) {
|
||||
subs = Collections.emptyList();
|
||||
}
|
||||
refineSubMap.put(freq, subs);
|
||||
return subs;
|
||||
}
|
||||
|
||||
|
||||
private Map<FacetRequest, Collection<String>> partialSubsMap = new IdentityHashMap<>(4);
|
||||
public Collection<String> getSubsWithPartial(FacetRequest freq) {
|
||||
if (freq.getSubFacets().isEmpty()) return Collections.emptyList();
|
||||
Collection<String> subs = partialSubsMap.get(freq);
|
||||
if (subs != null) return subs;
|
||||
|
||||
subs = null;
|
||||
for (Map.Entry<String,FacetRequest> entry : freq.subFacets.entrySet()) {
|
||||
Collection<String> childSubs = getSubsWithPartial(entry.getValue());
|
||||
if (childSubs.size() > 0 || entry.getValue().returnsPartial()) {
|
||||
if (subs == null) {
|
||||
subs = new ArrayList<>(freq.getSubFacets().size());
|
||||
}
|
||||
subs.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
if (subs == null) {
|
||||
subs = Collections.emptyList();
|
||||
}
|
||||
partialSubsMap.put(freq, subs);
|
||||
return subs;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -17,19 +17,16 @@
|
|||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.solr.client.solrj.SolrResponse;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.FacetParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.ShardParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
|
@ -41,6 +38,7 @@ import org.apache.solr.handler.component.ShardResponse;
|
|||
import org.apache.solr.search.QueryContext;
|
||||
import org.apache.solr.search.SyntaxError;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.noggit.JSONUtil;
|
||||
|
||||
public class FacetModule extends SearchComponent {
|
||||
|
||||
|
@ -55,6 +53,7 @@ public class FacetModule extends SearchComponent {
|
|||
|
||||
// Internal information passed down from the top level to shards for distributed faceting.
|
||||
private final static String FACET_STATE = "_facet_";
|
||||
private final static String FACET_REFINE = "refine";
|
||||
|
||||
|
||||
public FacetComponentState getFacetComponentState(ResponseBuilder rb) {
|
||||
|
@ -149,12 +148,90 @@ public class FacetModule extends SearchComponent {
|
|||
}
|
||||
|
||||
|
||||
private void clearFaceting(List<ShardRequest> outgoing) {
|
||||
// turn off faceting for requests not marked as being for faceting refinements
|
||||
for (ShardRequest sreq : outgoing) {
|
||||
if ((sreq.purpose & PURPOSE_REFINE_JSON_FACETS) != 0) continue;
|
||||
sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_STATE really control the faceting
|
||||
sreq.params.remove(FACET_STATE);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int distributedProcess(ResponseBuilder rb) throws IOException {
|
||||
FacetComponentState facetState = getFacetComponentState(rb);
|
||||
if (facetState == null) return ResponseBuilder.STAGE_DONE;
|
||||
|
||||
if (rb.stage != ResponseBuilder.STAGE_GET_FIELDS) {
|
||||
return ResponseBuilder.STAGE_DONE;
|
||||
}
|
||||
|
||||
// Check if there are any refinements possible
|
||||
if (facetState.mcontext.getSubsWithRefinement(facetState.facetRequest).isEmpty()) {
|
||||
clearFaceting(rb.outgoing);
|
||||
return ResponseBuilder.STAGE_DONE;
|
||||
}
|
||||
|
||||
// Overlap facet refinement requests (those shards that we need a count
|
||||
// for particular facet values from), where possible, with
|
||||
// the requests to get fields (because we know that is the
|
||||
// only other required phase).
|
||||
// We do this in distributedProcess so we can look at all of the
|
||||
// requests in the outgoing queue at once.
|
||||
|
||||
assert rb.shards.length == facetState.mcontext.numShards;
|
||||
for (String shard : rb.shards) {
|
||||
facetState.mcontext.setShard(shard);
|
||||
|
||||
// shard-specific refinement
|
||||
Map<String,Object> refinement = facetState.merger.getRefinement(facetState.mcontext);
|
||||
if (refinement == null) continue;
|
||||
|
||||
boolean newRequest = false;
|
||||
ShardRequest shardsRefineRequest = null;
|
||||
|
||||
// try to find a request that is already going out to that shard.
|
||||
// If nshards becomes too great, we may want to move to hashing for
|
||||
// better scalability.
|
||||
for (ShardRequest sreq : rb.outgoing) {
|
||||
if ( (sreq.purpose & (ShardRequest.PURPOSE_GET_FIELDS|ShardRequest.PURPOSE_REFINE_FACETS|ShardRequest.PURPOSE_REFINE_PIVOT_FACETS)) != 0
|
||||
&& sreq.shards != null
|
||||
&& sreq.shards.length == 1
|
||||
&& sreq.shards[0].equals(shard))
|
||||
{
|
||||
shardsRefineRequest = sreq;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (shardsRefineRequest == null) {
|
||||
// we didn't find any other suitable requests going out to that shard,
|
||||
// so create one ourselves.
|
||||
newRequest = true;
|
||||
shardsRefineRequest = new ShardRequest();
|
||||
shardsRefineRequest.shards = new String[] { shard };
|
||||
shardsRefineRequest.params = new ModifiableSolrParams(rb.req.getParams());
|
||||
// don't request any documents
|
||||
shardsRefineRequest.params.remove(CommonParams.START);
|
||||
shardsRefineRequest.params.set(CommonParams.ROWS, "0");
|
||||
shardsRefineRequest.params.set(CommonParams.ROWS, "0");
|
||||
shardsRefineRequest.params.set(FacetParams.FACET, false);
|
||||
}
|
||||
|
||||
shardsRefineRequest.purpose |= PURPOSE_REFINE_JSON_FACETS;
|
||||
|
||||
Map<String,Object> fstate = new HashMap<>(1);
|
||||
fstate.put(FACET_REFINE, refinement);
|
||||
String fstateString = JSONUtil.toJSON(fstate);
|
||||
shardsRefineRequest.params.add(FACET_STATE, fstateString);
|
||||
|
||||
if (newRequest) {
|
||||
rb.addRequest(this, shardsRefineRequest);
|
||||
}
|
||||
}
|
||||
|
||||
// clearFaceting(rb.outgoing);
|
||||
return ResponseBuilder.STAGE_DONE;
|
||||
}
|
||||
|
||||
|
@ -165,11 +242,13 @@ public class FacetModule extends SearchComponent {
|
|||
|
||||
if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) {
|
||||
sreq.purpose |= FacetModule.PURPOSE_GET_JSON_FACETS;
|
||||
sreq.params.set(FACET_STATE, "{}");
|
||||
sreq.params.set(FACET_STATE, "{}"); // The presence of FACET_STATE (_facet_) turns on json faceting
|
||||
} else {
|
||||
// turn off faceting on other requests
|
||||
sreq.params.remove("json.facet");
|
||||
/*** distributedProcess will need to use other requests for refinement
|
||||
sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_STATE really control the faceting
|
||||
sreq.params.remove(FACET_STATE);
|
||||
**/
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -186,8 +265,11 @@ public class FacetModule extends SearchComponent {
|
|||
if (facet == null) continue;
|
||||
if (facetState.merger == null) {
|
||||
facetState.merger = facetState.facetRequest.createFacetMerger(facet);
|
||||
facetState.mcontext = new FacetMerger.Context( sreq.responses.size() );
|
||||
}
|
||||
facetState.merger.merge(facet , new FacetMerger.Context());
|
||||
facetState.mcontext.root = facet;
|
||||
facetState.mcontext.newShard(shardRsp.getShard());
|
||||
facetState.merger.merge(facet , facetState.mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -199,6 +281,7 @@ public class FacetModule extends SearchComponent {
|
|||
if (facetState == null) return;
|
||||
|
||||
if (facetState.merger != null) {
|
||||
// TODO: merge any refinements
|
||||
rb.rsp.add("facets", facetState.merger.getMergedResult());
|
||||
}
|
||||
}
|
||||
|
@ -226,17 +309,23 @@ class FacetComponentState {
|
|||
// Only used for distributed search
|
||||
//
|
||||
FacetMerger merger;
|
||||
FacetMerger.Context mcontext;
|
||||
}
|
||||
|
||||
|
||||
// base class for facet functions that can be used in a sort
|
||||
abstract class FacetSortableMerger extends FacetMerger {
|
||||
public void prepareSort() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(Context mcontext) {
|
||||
// nothing to do for simple stats...
|
||||
}
|
||||
|
||||
/** Return the normal comparison sort order. The sort direction is only to be used in special circumstances (such as making NaN sort
|
||||
* last regardless of sort order.) Normal sorters do not need to pay attention to direction.
|
||||
*/
|
||||
public abstract int compareTo(FacetSortableMerger other, FacetField.SortDirection direction);
|
||||
public abstract int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction);
|
||||
}
|
||||
|
||||
abstract class FacetDoubleMerger extends FacetSortableMerger {
|
||||
|
@ -252,12 +341,12 @@ abstract class FacetDoubleMerger extends FacetSortableMerger {
|
|||
|
||||
|
||||
@Override
|
||||
public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
|
||||
public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
|
||||
return compare(getDouble(), ((FacetDoubleMerger)other).getDouble(), direction);
|
||||
}
|
||||
|
||||
|
||||
public static int compare(double a, double b, FacetField.SortDirection direction) {
|
||||
public static int compare(double a, double b, FacetRequest.SortDirection direction) {
|
||||
if (a < b) return -1;
|
||||
if (a > b) return 1;
|
||||
|
||||
|
@ -295,7 +384,7 @@ class FacetLongMerger extends FacetSortableMerger {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
|
||||
public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
|
||||
return Long.compare(val, ((FacetLongMerger)other).val);
|
||||
}
|
||||
}
|
||||
|
@ -304,15 +393,20 @@ class FacetLongMerger extends FacetSortableMerger {
|
|||
// base class for facets that create buckets (and can hence have sub-facets)
|
||||
abstract class FacetBucketMerger<FacetRequestT extends FacetRequest> extends FacetMerger {
|
||||
FacetRequestT freq;
|
||||
int bucketNumber;
|
||||
|
||||
public FacetBucketMerger(FacetRequestT freq) {
|
||||
this.freq = freq;
|
||||
}
|
||||
|
||||
/** Bucketval is the representative value for the bucket. Only applicable to terms and range queries to distinguish buckets. */
|
||||
FacetBucket newBucket(Comparable bucketVal) {
|
||||
return new FacetBucket(this, bucketVal, bucketNumber++);
|
||||
FacetBucket newBucket(Comparable bucketVal, Context mcontext) {
|
||||
return new FacetBucket(this, bucketVal, mcontext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getRefinement(Context mcontext) {
|
||||
Collection<String> refineTags = mcontext.getSubsWithRefinement(freq);
|
||||
return null; // FIXME
|
||||
}
|
||||
|
||||
// do subs...
|
||||
|
@ -334,6 +428,7 @@ abstract class FacetBucketMerger<FacetRequestT extends FacetRequest> extends Fac
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
class FacetQueryMerger extends FacetBucketMerger<FacetQuery> {
|
||||
FacetBucket bucket;
|
||||
|
||||
|
@ -344,11 +439,32 @@ class FacetQueryMerger extends FacetBucketMerger<FacetQuery> {
|
|||
@Override
|
||||
public void merge(Object facet, Context mcontext) {
|
||||
if (bucket == null) {
|
||||
bucket = newBucket(null);
|
||||
bucket = newBucket(null, mcontext);
|
||||
}
|
||||
bucket.mergeBucket((SimpleOrderedMap) facet, mcontext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getRefinement(Context mcontext) {
|
||||
Collection<String> tags;
|
||||
if (mcontext.bucketWasMissing()) {
|
||||
// if this bucket was missing, we need to get all subfacets that have partials (that need to list values for refinement)
|
||||
tags = mcontext.getSubsWithPartial(freq);
|
||||
} else {
|
||||
tags = mcontext.getSubsWithRefinement(freq);
|
||||
}
|
||||
|
||||
Map<String,Object> refinement = bucket.getRefinement(mcontext, tags);
|
||||
|
||||
return refinement;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void finish(Context mcontext) {
|
||||
// FIXME we need to propagate!!!
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
return bucket.getMergedBucket();
|
||||
|
@ -360,15 +476,15 @@ class FacetQueryMerger extends FacetBucketMerger<FacetQuery> {
|
|||
class FacetBucket {
|
||||
final FacetBucketMerger parent;
|
||||
final Comparable bucketValue;
|
||||
final int bucketNumber; // this is just for internal correlation (the first bucket created is bucket 0, the next bucket 1, etc)
|
||||
final int bucketNumber; // this is just for internal correlation (the first bucket created is bucket 0, the next bucket 1, across all field buckets)
|
||||
|
||||
long count;
|
||||
Map<String, FacetMerger> subs;
|
||||
|
||||
public FacetBucket(FacetBucketMerger parent, Comparable bucketValue, int bucketNumber) {
|
||||
public FacetBucket(FacetBucketMerger parent, Comparable bucketValue, FacetMerger.Context mcontext) {
|
||||
this.parent = parent;
|
||||
this.bucketValue = bucketValue;
|
||||
this.bucketNumber = bucketNumber;
|
||||
this.bucketNumber = mcontext.getNewBucketNumber(); // TODO: we don't need bucket numbers for all buckets...
|
||||
}
|
||||
|
||||
public long getCount() {
|
||||
|
@ -403,6 +519,8 @@ class FacetBucket {
|
|||
public void mergeBucket(SimpleOrderedMap bucket, FacetMerger.Context mcontext) {
|
||||
// todo: for refinements, we want to recurse, but not re-do stats for intermediate buckets
|
||||
|
||||
mcontext.setShardFlag(bucketNumber);
|
||||
|
||||
// drive merging off the received bucket?
|
||||
for (int i=0; i<bucket.size(); i++) {
|
||||
String key = bucket.getName(i);
|
||||
|
@ -440,368 +558,84 @@ class FacetBucket {
|
|||
|
||||
return out;
|
||||
}
|
||||
|
||||
public Map<String, Object> getRefinement(FacetMerger.Context mcontext, Collection<String> refineTags) {
|
||||
if (subs == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
|
||||
class FacetFieldMerger extends FacetBucketMerger<FacetField> {
|
||||
FacetBucket missingBucket;
|
||||
FacetBucket allBuckets;
|
||||
FacetMerger numBuckets;
|
||||
|
||||
LinkedHashMap<Object,FacetBucket> buckets = new LinkedHashMap<>();
|
||||
List<FacetBucket> sortedBuckets;
|
||||
int numReturnedBuckets; // the number of buckets in the bucket lists returned from all of the shards
|
||||
|
||||
private static class SortVal implements Comparable<SortVal> {
|
||||
FacetBucket bucket;
|
||||
FacetSortableMerger merger; // make this class inner and access merger , direction in parent?
|
||||
FacetField.SortDirection direction;
|
||||
|
||||
@Override
|
||||
public int compareTo(SortVal o) {
|
||||
int c = -merger.compareTo(o.merger, direction) * direction.getMultiplier();
|
||||
return c == 0 ? bucket.bucketValue.compareTo(o.bucket.bucketValue) : c;
|
||||
Map<String,Object> refinement = null;
|
||||
for (String tag : refineTags) {
|
||||
FacetMerger subMerger = subs.get(tag);
|
||||
if (subMerger != null) {
|
||||
Map<String,Object> subRef = subMerger.getRefinement(mcontext);
|
||||
if (subRef != null) {
|
||||
if (refinement == null) {
|
||||
refinement = new HashMap<>(refineTags.size());
|
||||
}
|
||||
refinement.put(tag, subRef);
|
||||
}
|
||||
}
|
||||
|
||||
public FacetFieldMerger(FacetField freq) {
|
||||
super(freq);
|
||||
}
|
||||
return refinement;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
merge((SimpleOrderedMap)facetResult, mcontext);
|
||||
}
|
||||
public Map<String, Object> getRefinement2(FacetMerger.Context mcontext, Collection<String> refineTags) {
|
||||
// TODO - partial results should turn off refining!!!
|
||||
|
||||
protected void merge(SimpleOrderedMap facetResult, Context mcontext) {
|
||||
if (freq.missing) {
|
||||
Object o = facetResult.get("missing");
|
||||
if (o != null) {
|
||||
if (missingBucket == null) {
|
||||
missingBucket = newBucket(null);
|
||||
}
|
||||
missingBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
boolean parentMissing = mcontext.bucketWasMissing();
|
||||
|
||||
if (freq.allBuckets) {
|
||||
Object o = facetResult.get("allBuckets");
|
||||
if (o != null) {
|
||||
if (allBuckets == null) {
|
||||
allBuckets = newBucket(null);
|
||||
}
|
||||
allBuckets.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
|
||||
numReturnedBuckets += bucketList.size();
|
||||
mergeBucketList(bucketList , mcontext);
|
||||
|
||||
if (freq.numBuckets) {
|
||||
Object nb = facetResult.get("numBuckets");
|
||||
if (nb != null) {
|
||||
if (numBuckets == null) {
|
||||
numBuckets = new FacetNumBucketsMerger();
|
||||
}
|
||||
numBuckets.merge(nb , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void mergeBucketList(List<SimpleOrderedMap> bucketList, Context mcontext) {
|
||||
for (SimpleOrderedMap bucketRes : bucketList) {
|
||||
Comparable bucketVal = (Comparable)bucketRes.get("val");
|
||||
FacetBucket bucket = buckets.get(bucketVal);
|
||||
if (bucket == null) {
|
||||
bucket = newBucket(bucketVal);
|
||||
buckets.put(bucketVal, bucket);
|
||||
}
|
||||
bucket.mergeBucket( bucketRes , mcontext );
|
||||
}
|
||||
}
|
||||
|
||||
public void sortBuckets() {
|
||||
sortedBuckets = new ArrayList<>( buckets.values() );
|
||||
|
||||
Comparator<FacetBucket> comparator = null;
|
||||
|
||||
final FacetField.SortDirection direction = freq.sortDirection;
|
||||
final int sortMul = direction.getMultiplier();
|
||||
|
||||
if ("count".equals(freq.sortVariable)) {
|
||||
comparator = (o1, o2) -> {
|
||||
int v = -Long.compare(o1.count, o2.count) * sortMul;
|
||||
return v == 0 ? o1.bucketValue.compareTo(o2.bucketValue) : v;
|
||||
};
|
||||
Collections.sort(sortedBuckets, comparator);
|
||||
} else if ("index".equals(freq.sortVariable)) {
|
||||
comparator = (o1, o2) -> -o1.bucketValue.compareTo(o2.bucketValue) * sortMul;
|
||||
Collections.sort(sortedBuckets, comparator);
|
||||
} else {
|
||||
final String key = freq.sortVariable;
|
||||
|
||||
/**
|
||||
final FacetSortableMerger[] arr = new FacetSortableMerger[buckets.size()];
|
||||
final int[] index = new int[arr.length];
|
||||
int start = 0;
|
||||
int nullStart = index.length;
|
||||
int i=0;
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
FacetMerger merger = bucket.getExistingMerger(key);
|
||||
if (merger == null) {
|
||||
index[--nullStart] = i;
|
||||
}
|
||||
if (merger != null) {
|
||||
arr[start] = (FacetSortableMerger)merger;
|
||||
index[start] = i;
|
||||
start++;
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
PrimUtils.sort(0, nullStart, index, new PrimUtils.IntComparator() {
|
||||
@Override
|
||||
public int compare(int a, int b) {
|
||||
return arr[index[a]].compareTo(arr[index[b]], direction);
|
||||
}
|
||||
});
|
||||
**/
|
||||
|
||||
// timsort may do better here given that the lists may be partially sorted.
|
||||
|
||||
List<SortVal> lst = new ArrayList<SortVal>(buckets.size());
|
||||
List<FacetBucket> nulls = new ArrayList<FacetBucket>(buckets.size()>>1);
|
||||
for (int i=0; i<sortedBuckets.size(); i++) {
|
||||
FacetBucket bucket = sortedBuckets.get(i);
|
||||
FacetMerger merger = bucket.getExistingMerger(key);
|
||||
if (merger == null) {
|
||||
nulls.add(bucket);
|
||||
}
|
||||
if (merger != null) {
|
||||
SortVal sv = new SortVal();
|
||||
sv.bucket = bucket;
|
||||
sv.merger = (FacetSortableMerger)merger;
|
||||
sv.direction = direction;
|
||||
// sv.pos = i; // if we need position in the future...
|
||||
lst.add(sv);
|
||||
}
|
||||
}
|
||||
Collections.sort(lst);
|
||||
Collections.sort(nulls, (o1, o2) -> o1.bucketValue.compareTo(o2.bucketValue));
|
||||
|
||||
ArrayList<FacetBucket> out = new ArrayList<>(buckets.size());
|
||||
for (SortVal sv : lst) {
|
||||
out.add( sv.bucket );
|
||||
}
|
||||
out.addAll(nulls);
|
||||
sortedBuckets = out;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
SimpleOrderedMap result = new SimpleOrderedMap();
|
||||
|
||||
if (numBuckets != null) {
|
||||
int removed = 0;
|
||||
if (freq.mincount > 1) {
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
if (bucket.count < freq.mincount) removed++;
|
||||
}
|
||||
}
|
||||
result.add("numBuckets", ((Number)numBuckets.getMergedResult()).longValue() - removed);
|
||||
|
||||
// TODO: we can further increase this estimate.
|
||||
// If not sorting by count, use a simple ratio to scale
|
||||
// If sorting by count desc, then add up the highest_possible_missing_count from each shard
|
||||
}
|
||||
|
||||
sortBuckets();
|
||||
|
||||
int first = (int)freq.offset;
|
||||
int end = freq.limit >=0 ? first + (int) freq.limit : Integer.MAX_VALUE;
|
||||
int last = Math.min(sortedBuckets.size(), end);
|
||||
|
||||
List<SimpleOrderedMap> resultBuckets = new ArrayList<>(Math.max(0, (last - first)));
|
||||
|
||||
/** this only works if there are no filters (like mincount)
|
||||
for (int i=first; i<last; i++) {
|
||||
FacetBucket bucket = sortedBuckets.get(i);
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
}
|
||||
***/
|
||||
|
||||
// TODO: change effective offsets + limits at shards...
|
||||
|
||||
int off = (int)freq.offset;
|
||||
int lim = freq.limit >= 0 ? (int)freq.limit : Integer.MAX_VALUE;
|
||||
for (FacetBucket bucket : sortedBuckets) {
|
||||
if (bucket.getCount() < freq.mincount) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (off > 0) {
|
||||
--off;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (resultBuckets.size() >= lim) {
|
||||
break;
|
||||
}
|
||||
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
}
|
||||
|
||||
|
||||
result.add("buckets", resultBuckets);
|
||||
if (missingBucket != null) {
|
||||
result.add("missing", missingBucket.getMergedBucket());
|
||||
}
|
||||
if (allBuckets != null) {
|
||||
result.add("allBuckets", allBuckets.getMergedBucket());
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
private class FacetNumBucketsMerger extends FacetMerger {
|
||||
long sumBuckets;
|
||||
long shardsMissingSum;
|
||||
long shardsTruncatedSum;
|
||||
Set<Object> values;
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
|
||||
long numBuckets = ((Number)map.get("numBuckets")).longValue();
|
||||
sumBuckets += numBuckets;
|
||||
|
||||
List vals = (List)map.get("vals");
|
||||
if (vals != null) {
|
||||
if (values == null) {
|
||||
values = new HashSet<>(vals.size()*4);
|
||||
}
|
||||
values.addAll(vals);
|
||||
if (numBuckets > values.size()) {
|
||||
shardsTruncatedSum += numBuckets - values.size();
|
||||
// TODO: this is a redundant check for many types of facets... only do on field faceting
|
||||
if (!parentMissing) {
|
||||
// if parent bucket wasn't missing, check if this bucket was.
|
||||
// this really only needs checking on certain buckets... (like terms facet)
|
||||
boolean sawThisBucket = mcontext.getShardFlag(bucketNumber);
|
||||
if (!sawThisBucket) {
|
||||
mcontext.setBucketWasMissing(true);
|
||||
}
|
||||
} else {
|
||||
shardsMissingSum += numBuckets;
|
||||
}
|
||||
// if parent bucket was missing, then we should be too
|
||||
assert !mcontext.getShardFlag(bucketNumber);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
long exactCount = values == null ? 0 : values.size();
|
||||
return exactCount + shardsMissingSum + shardsTruncatedSum;
|
||||
// TODO: reduce count by (at least) number of buckets that fail to hit mincount (after merging)
|
||||
// that should make things match for most of the small tests at least
|
||||
Map<String,Object> refinement = null;
|
||||
|
||||
if (!mcontext.bucketWasMissing()) {
|
||||
// this is just a pass-through bucket... see if there is anything to do at all
|
||||
if (subs == null || refineTags.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
} else {
|
||||
// for missing bucket, go over all sub-facts
|
||||
refineTags = null;
|
||||
refinement = new HashMap<>(4);
|
||||
if (bucketValue != null) {
|
||||
refinement.put("_v", bucketValue);
|
||||
}
|
||||
refinement.put("_m",1);
|
||||
}
|
||||
|
||||
// TODO: listing things like sub-facets that have no field facets are redundant
|
||||
// (we only need facet that have variable values)
|
||||
|
||||
class FacetRangeMerger extends FacetBucketMerger<FacetRange> {
|
||||
FacetBucket beforeBucket;
|
||||
FacetBucket afterBucket;
|
||||
FacetBucket betweenBucket;
|
||||
|
||||
LinkedHashMap<Object, FacetBucket> buckets = new LinkedHashMap<Object, FacetBucket>();
|
||||
|
||||
|
||||
public FacetRangeMerger(FacetRange freq) {
|
||||
super(freq);
|
||||
}
|
||||
|
||||
@Override
|
||||
FacetMerger createFacetMerger(String key, Object val) {
|
||||
return super.createFacetMerger(key, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
merge((SimpleOrderedMap) facetResult , mcontext);
|
||||
}
|
||||
|
||||
public void merge(SimpleOrderedMap facetResult, Context mcontext) {
|
||||
boolean all = freq.others.contains(FacetParams.FacetRangeOther.ALL);
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.BEFORE)) {
|
||||
Object o = facetResult.get("before");
|
||||
if (o != null) {
|
||||
if (beforeBucket == null) {
|
||||
beforeBucket = newBucket(null);
|
||||
}
|
||||
beforeBucket.mergeBucket((SimpleOrderedMap)o, mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.AFTER)) {
|
||||
Object o = facetResult.get("after");
|
||||
if (o != null) {
|
||||
if (afterBucket == null) {
|
||||
afterBucket = newBucket(null);
|
||||
}
|
||||
afterBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.BETWEEN)) {
|
||||
Object o = facetResult.get("between");
|
||||
if (o != null) {
|
||||
if (betweenBucket == null) {
|
||||
betweenBucket = newBucket(null);
|
||||
}
|
||||
betweenBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
|
||||
mergeBucketList(bucketList , mcontext);
|
||||
}
|
||||
|
||||
// TODO: share more merging with field faceting
|
||||
public void mergeBucketList(List<SimpleOrderedMap> bucketList, Context mcontext) {
|
||||
for (SimpleOrderedMap bucketRes : bucketList) {
|
||||
Comparable bucketVal = (Comparable)bucketRes.get("val");
|
||||
FacetBucket bucket = buckets.get(bucketVal);
|
||||
if (bucket == null) {
|
||||
bucket = newBucket(bucketVal);
|
||||
buckets.put(bucketVal, bucket);
|
||||
}
|
||||
bucket.mergeBucket( bucketRes , mcontext );
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
SimpleOrderedMap result = new SimpleOrderedMap(4);
|
||||
|
||||
List<SimpleOrderedMap> resultBuckets = new ArrayList<>(buckets.size());
|
||||
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
if (bucket.getCount() < freq.mincount) {
|
||||
for (Map.Entry<String,FacetMerger> sub : subs.entrySet()) {
|
||||
if (refineTags != null && !refineTags.contains(sub.getKey())) {
|
||||
continue;
|
||||
}
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
Map<String,Object> subRef = sub.getValue().getRefinement(mcontext);
|
||||
if (subRef != null) {
|
||||
if (refinement == null) {
|
||||
refinement = new HashMap<>(4);
|
||||
}
|
||||
refinement.put(sub.getKey(), subRef);
|
||||
}
|
||||
}
|
||||
|
||||
result.add("buckets", resultBuckets);
|
||||
|
||||
if (beforeBucket != null) {
|
||||
result.add("before", beforeBucket.getMergedBucket());
|
||||
// reset the "bucketMissing" flag on the way back out.
|
||||
mcontext.setBucketWasMissing(parentMissing);
|
||||
return refinement;
|
||||
}
|
||||
if (afterBucket != null) {
|
||||
result.add("after", afterBucket.getMergedBucket());
|
||||
}
|
||||
if (betweenBucket != null) {
|
||||
result.add("between", betweenBucket.getMergedBucket());
|
||||
}
|
||||
return result;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.solr.schema.TrieField;
|
|||
import org.apache.solr.search.DocSet;
|
||||
import org.apache.solr.util.DateMathParser;
|
||||
|
||||
public class FacetRange extends FacetRequest {
|
||||
public class FacetRange extends FacetRequestSorted {
|
||||
String field;
|
||||
Object start;
|
||||
Object end;
|
||||
|
@ -44,8 +44,12 @@ public class FacetRange extends FacetRequest {
|
|||
boolean hardend = false;
|
||||
EnumSet<FacetParams.FacetRangeInclude> include;
|
||||
EnumSet<FacetParams.FacetRangeOther> others;
|
||||
long mincount = 0;
|
||||
|
||||
{
|
||||
// defaults
|
||||
mincount = 0;
|
||||
limit = -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FacetProcessor createFacetProcessor(FacetContext fcontext) {
|
||||
|
|
|
@ -0,0 +1,123 @@
|
|||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.solr.common.params.FacetParams;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
|
||||
public class FacetRangeMerger extends FacetRequestSortedMerger<FacetRange> {
|
||||
FacetBucket beforeBucket;
|
||||
FacetBucket afterBucket;
|
||||
FacetBucket betweenBucket;
|
||||
|
||||
public FacetRangeMerger(FacetRange freq) {
|
||||
super(freq);
|
||||
}
|
||||
|
||||
@Override
|
||||
FacetMerger createFacetMerger(String key, Object val) {
|
||||
return super.createFacetMerger(key, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(Object facetResult, Context mcontext) {
|
||||
merge((SimpleOrderedMap) facetResult , mcontext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sortBuckets() {
|
||||
// TODO: mincount>0 will mess up order?
|
||||
sortedBuckets = new ArrayList<>( buckets.values() );
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(Context mcontext) {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
public void merge(SimpleOrderedMap facetResult, Context mcontext) {
|
||||
boolean all = freq.others.contains(FacetParams.FacetRangeOther.ALL);
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.BEFORE)) {
|
||||
Object o = facetResult.get("before");
|
||||
if (o != null) {
|
||||
if (beforeBucket == null) {
|
||||
beforeBucket = newBucket(null, mcontext);
|
||||
}
|
||||
beforeBucket.mergeBucket((SimpleOrderedMap)o, mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.AFTER)) {
|
||||
Object o = facetResult.get("after");
|
||||
if (o != null) {
|
||||
if (afterBucket == null) {
|
||||
afterBucket = newBucket(null, mcontext);
|
||||
}
|
||||
afterBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
if (all || freq.others.contains(FacetParams.FacetRangeOther.BETWEEN)) {
|
||||
Object o = facetResult.get("between");
|
||||
if (o != null) {
|
||||
if (betweenBucket == null) {
|
||||
betweenBucket = newBucket(null, mcontext);
|
||||
}
|
||||
betweenBucket.mergeBucket((SimpleOrderedMap)o , mcontext);
|
||||
}
|
||||
}
|
||||
|
||||
List<SimpleOrderedMap> bucketList = (List<SimpleOrderedMap>) facetResult.get("buckets");
|
||||
mergeBucketList(bucketList , mcontext);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Object getMergedResult() {
|
||||
// TODO: use sortedBuckets
|
||||
SimpleOrderedMap result = new SimpleOrderedMap(4);
|
||||
|
||||
List<SimpleOrderedMap> resultBuckets = new ArrayList<>(buckets.size());
|
||||
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
if (bucket.getCount() < freq.mincount) {
|
||||
continue;
|
||||
}
|
||||
resultBuckets.add( bucket.getMergedBucket() );
|
||||
}
|
||||
|
||||
result.add("buckets", resultBuckets);
|
||||
|
||||
if (beforeBucket != null) {
|
||||
result.add("before", beforeBucket.getMergedBucket());
|
||||
}
|
||||
if (afterBucket != null) {
|
||||
result.add("after", afterBucket.getMergedBucket());
|
||||
}
|
||||
if (betweenBucket != null) {
|
||||
result.add("between", betweenBucket.getMergedBucket());
|
||||
}
|
||||
return result;
|
||||
|
||||
}
|
||||
}
|
|
@ -35,10 +35,48 @@ import org.apache.solr.search.QueryContext;
|
|||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.SyntaxError;
|
||||
|
||||
import static org.apache.solr.search.facet.FacetRequest.RefineMethod.NONE;
|
||||
|
||||
|
||||
public abstract class FacetRequest {
|
||||
|
||||
public static enum SortDirection {
|
||||
asc(-1) ,
|
||||
desc(1);
|
||||
|
||||
private final int multiplier;
|
||||
private SortDirection(int multiplier) {
|
||||
this.multiplier = multiplier;
|
||||
}
|
||||
|
||||
// asc==-1, desc==1
|
||||
public int getMultiplier() {
|
||||
return multiplier;
|
||||
}
|
||||
}
|
||||
|
||||
public static enum RefineMethod {
|
||||
NONE,
|
||||
SIMPLE;
|
||||
// NONE is distinct from null since we may want to know if refinement was explicitly turned off.
|
||||
public static FacetRequest.RefineMethod fromObj(Object method) {
|
||||
if (method == null) return null;
|
||||
if (method instanceof Boolean) {
|
||||
return ((Boolean)method) ? SIMPLE : NONE;
|
||||
}
|
||||
if ("simple".equals(method)) {
|
||||
return SIMPLE;
|
||||
} else if ("none".equals(method)) {
|
||||
return NONE;
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown RefineMethod method " + method);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected Map<String,AggValueSource> facetStats; // per-bucket statistics
|
||||
protected Map<String,FacetRequest> subFacets; // list of facets
|
||||
protected Map<String,FacetRequest> subFacets; // per-bucket sub-facets
|
||||
protected List<String> filters;
|
||||
protected boolean processEmpty;
|
||||
protected Domain domain;
|
||||
|
@ -64,6 +102,22 @@ public abstract class FacetRequest {
|
|||
return subFacets;
|
||||
}
|
||||
|
||||
/** Returns null if unset */
|
||||
public RefineMethod getRefineMethod() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public boolean doRefine() {
|
||||
return !(getRefineMethod()==null || getRefineMethod()==NONE);
|
||||
}
|
||||
|
||||
/** Returns true if this facet can return just some of the facet buckets that match all the criteria.
|
||||
* This is normally true only for facets with a limit.
|
||||
*/
|
||||
public boolean returnsPartial() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public void addStat(String key, AggValueSource stat) {
|
||||
facetStats.put(key, stat);
|
||||
}
|
||||
|
@ -541,6 +595,9 @@ class FacetFieldParser extends FacetParser<FacetField> {
|
|||
facet.method = FacetField.FacetMethod.fromString(getString(m, "method", null));
|
||||
facet.cacheDf = (int)getLong(m, "cacheDf", facet.cacheDf);
|
||||
|
||||
// TODO: pull up to higher level?
|
||||
facet.refine = FacetField.RefineMethod.fromObj(m.get("refine"));
|
||||
|
||||
facet.perSeg = (Boolean)m.get("perSeg");
|
||||
|
||||
// facet.sort may depend on a facet stat...
|
||||
|
@ -562,18 +619,18 @@ class FacetFieldParser extends FacetParser<FacetField> {
|
|||
private void parseSort(Object sort) {
|
||||
if (sort == null) {
|
||||
facet.sortVariable = "count";
|
||||
facet.sortDirection = FacetField.SortDirection.desc;
|
||||
facet.sortDirection = FacetRequest.SortDirection.desc;
|
||||
} else if (sort instanceof String) {
|
||||
String sortStr = (String)sort;
|
||||
if (sortStr.endsWith(" asc")) {
|
||||
facet.sortVariable = sortStr.substring(0, sortStr.length()-" asc".length());
|
||||
facet.sortDirection = FacetField.SortDirection.asc;
|
||||
facet.sortDirection = FacetRequest.SortDirection.asc;
|
||||
} else if (sortStr.endsWith(" desc")) {
|
||||
facet.sortVariable = sortStr.substring(0, sortStr.length()-" desc".length());
|
||||
facet.sortDirection = FacetField.SortDirection.desc;
|
||||
facet.sortDirection = FacetRequest.SortDirection.desc;
|
||||
} else {
|
||||
facet.sortVariable = sortStr;
|
||||
facet.sortDirection = "index".equals(facet.sortVariable) ? FacetField.SortDirection.asc : FacetField.SortDirection.desc; // default direction for "index" is ascending
|
||||
facet.sortDirection = "index".equals(facet.sortVariable) ? FacetRequest.SortDirection.asc : FacetRequest.SortDirection.desc; // default direction for "index" is ascending
|
||||
}
|
||||
} else {
|
||||
// sort : { myvar : 'desc' }
|
||||
|
@ -583,7 +640,7 @@ class FacetFieldParser extends FacetParser<FacetField> {
|
|||
String k = entry.getKey();
|
||||
Object v = entry.getValue();
|
||||
facet.sortVariable = k;
|
||||
facet.sortDirection = FacetField.SortDirection.valueOf(v.toString());
|
||||
facet.sortDirection = FacetRequest.SortDirection.valueOf(v.toString());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,234 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.solr.search.facet;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
|
||||
// base class for facets that create a list of buckets that can be sorted
|
||||
abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted> extends FacetBucketMerger<FacetRequestT> {
|
||||
LinkedHashMap<Object,FacetBucket> buckets = new LinkedHashMap<>();
|
||||
List<FacetBucket> sortedBuckets;
|
||||
|
||||
public FacetRequestSortedMerger(FacetRequestT freq) {
|
||||
super(freq);
|
||||
}
|
||||
|
||||
private static class SortVal implements Comparable<SortVal> {
|
||||
FacetBucket bucket;
|
||||
FacetSortableMerger merger; // make this class inner and access merger , direction in parent?
|
||||
FacetRequest.SortDirection direction;
|
||||
|
||||
@Override
|
||||
public int compareTo(SortVal o) {
|
||||
int c = -merger.compareTo(o.merger, direction) * direction.getMultiplier();
|
||||
return c == 0 ? bucket.bucketValue.compareTo(o.bucket.bucketValue) : c;
|
||||
}
|
||||
}
|
||||
|
||||
public void mergeBucketList(List<SimpleOrderedMap> bucketList, Context mcontext) {
|
||||
for (SimpleOrderedMap bucketRes : bucketList) {
|
||||
Comparable bucketVal = (Comparable)bucketRes.get("val");
|
||||
FacetBucket bucket = buckets.get(bucketVal);
|
||||
if (bucket == null) {
|
||||
bucket = newBucket(bucketVal, mcontext);
|
||||
buckets.put(bucketVal, bucket);
|
||||
}
|
||||
bucket.mergeBucket( bucketRes , mcontext );
|
||||
}
|
||||
}
|
||||
|
||||
public void sortBuckets() {
|
||||
sortedBuckets = new ArrayList<>( buckets.values() );
|
||||
|
||||
Comparator<FacetBucket> comparator = null;
|
||||
|
||||
final FacetRequest.SortDirection direction = freq.sortDirection;
|
||||
final int sortMul = direction.getMultiplier();
|
||||
|
||||
if ("count".equals(freq.sortVariable)) {
|
||||
comparator = (o1, o2) -> {
|
||||
int v = -Long.compare(o1.count, o2.count) * sortMul;
|
||||
return v == 0 ? o1.bucketValue.compareTo(o2.bucketValue) : v;
|
||||
};
|
||||
Collections.sort(sortedBuckets, comparator);
|
||||
} else if ("index".equals(freq.sortVariable)) {
|
||||
comparator = (o1, o2) -> -o1.bucketValue.compareTo(o2.bucketValue) * sortMul;
|
||||
Collections.sort(sortedBuckets, comparator);
|
||||
} else {
|
||||
final String key = freq.sortVariable;
|
||||
|
||||
/**
|
||||
final FacetSortableMerger[] arr = new FacetSortableMerger[buckets.size()];
|
||||
final int[] index = new int[arr.length];
|
||||
int start = 0;
|
||||
int nullStart = index.length;
|
||||
int i=0;
|
||||
for (FacetBucket bucket : buckets.values()) {
|
||||
FacetMerger merger = bucket.getExistingMerger(key);
|
||||
if (merger == null) {
|
||||
index[--nullStart] = i;
|
||||
}
|
||||
if (merger != null) {
|
||||
arr[start] = (FacetSortableMerger)merger;
|
||||
index[start] = i;
|
||||
start++;
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
PrimUtils.sort(0, nullStart, index, new PrimUtils.IntComparator() {
|
||||
@Override
|
||||
public int compare(int a, int b) {
|
||||
return arr[index[a]].compareTo(arr[index[b]], direction);
|
||||
}
|
||||
});
|
||||
**/
|
||||
|
||||
|
||||
List<SortVal> lst = new ArrayList<>(buckets.size());
|
||||
List<FacetBucket> nulls = new ArrayList<>(buckets.size()>>1);
|
||||
for (int i=0; i<sortedBuckets.size(); i++) {
|
||||
FacetBucket bucket = sortedBuckets.get(i);
|
||||
FacetMerger merger = bucket.getExistingMerger(key);
|
||||
if (merger == null) {
|
||||
nulls.add(bucket);
|
||||
}
|
||||
if (merger != null) {
|
||||
SortVal sv = new SortVal();
|
||||
sv.bucket = bucket;
|
||||
sv.merger = (FacetSortableMerger)merger;
|
||||
sv.direction = direction;
|
||||
// sv.pos = i; // if we need position in the future...
|
||||
lst.add(sv);
|
||||
}
|
||||
}
|
||||
Collections.sort(lst);
|
||||
Collections.sort(nulls, (o1, o2) -> o1.bucketValue.compareTo(o2.bucketValue));
|
||||
|
||||
ArrayList<FacetBucket> out = new ArrayList<>(buckets.size());
|
||||
for (SortVal sv : lst) {
|
||||
out.add( sv.bucket );
|
||||
}
|
||||
out.addAll(nulls);
|
||||
sortedBuckets = out;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getRefinement(Context mcontext) {
|
||||
// step 1) If this facet request has refining, then we need to fully request top buckets that were not seen by this shard.
|
||||
// step 2) If this facet does not have refining, but some sub-facets do, we need to check/recurse those sub-facets in *every* top bucket.
|
||||
// A combination of the two is possible and makes step 2 redundant for any buckets we fully requested in step 1.
|
||||
|
||||
Map<String,Object> refinement = null;
|
||||
|
||||
Collection<String> tags = mcontext.getSubsWithRefinement(freq);
|
||||
if (tags.isEmpty() && !freq.doRefine()) {
|
||||
// we don't have refining, and neither do our subs
|
||||
return null;
|
||||
}
|
||||
|
||||
// Tags for sub facets that have partial facets somewhere in their children.
|
||||
// If we are missing a bucket for this shard, we'll need to get the specific buckets that need refining.
|
||||
Collection<String> tagsWithPartial = mcontext.getSubsWithPartial(freq);
|
||||
|
||||
boolean thisMissing = mcontext.bucketWasMissing();
|
||||
|
||||
int num = (int)(freq.offset + freq.limit);
|
||||
int numBucketsToCheck = Math.min(buckets.size(), num);
|
||||
|
||||
Collection<FacetBucket> bucketList;
|
||||
if (buckets.size() < num) {
|
||||
// no need to sort
|
||||
// todo: but we may need to filter.... simplify by always sorting?
|
||||
bucketList = buckets.values();
|
||||
} else {
|
||||
// only sort once
|
||||
if (sortedBuckets == null) {
|
||||
sortBuckets(); // todo: make sure this filters buckets as well
|
||||
}
|
||||
bucketList = sortedBuckets;
|
||||
}
|
||||
|
||||
ArrayList<Object> leafBuckets = null; // "_l" missing buckets specified by bucket value only (no need to specify anything further)
|
||||
ArrayList<Object> missingBuckets = null; // "_m" missing buckets that need to specify values for partial facets
|
||||
ArrayList<Object> skipBuckets = null; // "_s" present buckets that we need to recurse into because children facets have refinement requirements
|
||||
|
||||
for (FacetBucket bucket : bucketList) {
|
||||
if (numBucketsToCheck-- <= 0) break;
|
||||
// if this bucket is missing,
|
||||
assert thisMissing == false || thisMissing == true && mcontext.getShardFlag(bucket.bucketNumber) == false;
|
||||
boolean saw = !thisMissing && mcontext.getShardFlag(bucket.bucketNumber);
|
||||
if (!saw) {
|
||||
// we didn't see the bucket for this shard
|
||||
Map<String,Object> bucketRefinement = null;
|
||||
|
||||
// find facets that we need to fill in buckets for
|
||||
if (!tagsWithPartial.isEmpty()) {
|
||||
boolean prev = mcontext.setBucketWasMissing(true);
|
||||
bucketRefinement = bucket.getRefinement(mcontext, tagsWithPartial);
|
||||
mcontext.setBucketWasMissing(prev);
|
||||
|
||||
if (bucketRefinement != null) {
|
||||
if (missingBuckets==null) missingBuckets = new ArrayList<>();
|
||||
missingBuckets.add(bucketRefinement);
|
||||
}
|
||||
}
|
||||
|
||||
// if we didn't add to "_m" (missing), then we should add to "_l" (leaf missing)
|
||||
if (bucketRefinement == null) {
|
||||
if (leafBuckets == null) leafBuckets = new ArrayList<>();
|
||||
leafBuckets.add(bucket.bucketValue);
|
||||
}
|
||||
|
||||
} else if (!tags.isEmpty()) {
|
||||
// we had this bucket, but we need to recurse to certain children that have refinements
|
||||
Map<String,Object> bucketRefinement = bucket.getRefinement(mcontext, tagsWithPartial);
|
||||
if (bucketRefinement != null) {
|
||||
if (skipBuckets == null) skipBuckets = new ArrayList<>();
|
||||
skipBuckets.add(bucketRefinement);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// TODO: what if we don't need to refine any variable buckets, but we do need to contribute to numBuckets, missing, allBuckets, etc...
|
||||
// because we were "missing". That will be handled at a higher level (i.e. we'll be in someone's missing bucket?)
|
||||
// TODO: test with a sub-facet with a limit of 0 and something like a missing bucket
|
||||
if (leafBuckets != null || missingBuckets != null || skipBuckets != null) {
|
||||
refinement = new HashMap<>(3);
|
||||
if (leafBuckets != null) refinement.put("_l",leafBuckets);
|
||||
if (missingBuckets != null) refinement.put("_m", missingBuckets);
|
||||
if (skipBuckets != null) refinement.put("_s", skipBuckets);
|
||||
}
|
||||
|
||||
return refinement;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -99,7 +99,7 @@ public class HLLAgg extends StrAggValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
|
||||
public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
|
||||
return Long.compare( getLong(), ((Merger)other).getLong() );
|
||||
}
|
||||
}
|
||||
|
|
|
@ -207,7 +207,7 @@ public class PercentileAgg extends SimpleAggValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
|
||||
public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
|
||||
return Double.compare(getSortVal(), ((Merger) other).getSortVal());
|
||||
}
|
||||
|
||||
|
|
|
@ -113,7 +113,7 @@ public class UniqueAgg extends StrAggValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
|
||||
public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
|
||||
return Long.compare( getLong(), ((Merger)other).getLong() );
|
||||
}
|
||||
}
|
||||
|
|
|
@ -118,21 +118,23 @@ public class SolrSuggester implements Accountable {
|
|||
// initialize appropriate lookup instance
|
||||
factory = core.getResourceLoader().newInstance(lookupImpl, LookupFactory.class);
|
||||
lookup = factory.create(config, core);
|
||||
|
||||
if (lookup != null && lookup instanceof Closeable) {
|
||||
core.addCloseHook(new CloseHook() {
|
||||
@Override
|
||||
public void preClose(SolrCore core) {
|
||||
if (lookup != null && lookup instanceof Closeable) {
|
||||
try {
|
||||
((Closeable) lookup).close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Could not close the suggester lookup.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postClose(SolrCore core) {}
|
||||
public void postClose(SolrCore core) {
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// if store directory is provided make it or load up the lookup with its content
|
||||
if (store != null && !store.isEmpty()) {
|
||||
|
|
|
@ -564,9 +564,14 @@ public class PeerSync {
|
|||
private boolean compareFingerprint(SyncShardRequest sreq) {
|
||||
if (sreq.fingerprint == null) return true;
|
||||
try {
|
||||
IndexFingerprint ourFingerprint = IndexFingerprint.getFingerprint(core, Long.MAX_VALUE);
|
||||
int cmp = IndexFingerprint.compare(ourFingerprint, sreq.fingerprint);
|
||||
log.info("Fingerprint comparison: " + cmp);
|
||||
// check our fingerprint only upto the max version in the other fingerprint.
|
||||
// Otherwise for missed updates (look at missed update test in PeerSyncTest) ourFingerprint won't match with otherFingerprint
|
||||
IndexFingerprint ourFingerprint = IndexFingerprint.getFingerprint(core, sreq.fingerprint.getMaxVersionSpecified());
|
||||
int cmp = IndexFingerprint.compare(sreq.fingerprint, ourFingerprint);
|
||||
log.info("Fingerprint comparison: {}" , cmp);
|
||||
if(cmp != 0) {
|
||||
log.info("Other fingerprint: {}, Our fingerprint: {}", sreq.fingerprint , ourFingerprint);
|
||||
}
|
||||
return cmp == 0; // currently, we only check for equality...
|
||||
} catch(IOException e){
|
||||
log.error(msg() + "Error getting index fingerprint", e);
|
||||
|
@ -588,6 +593,12 @@ public class PeerSync {
|
|||
sreq.params.set("distrib", false);
|
||||
sreq.params.set("getUpdates", versionsAndRanges);
|
||||
sreq.params.set("onlyIfActive", onlyIfActive);
|
||||
|
||||
// fingerprint should really be requested only for the maxversion we are requesting updates for
|
||||
// In case updates are coming in while node is coming up after restart, node would have already
|
||||
// buffered some of the updates. fingerprint we requested with versions would reflect versions
|
||||
// in our buffer as well and will definitely cause a mismatch
|
||||
sreq.params.set("fingerprint",doFingerprint);
|
||||
sreq.responses.clear(); // needs to be zeroed for correct correlation to occur
|
||||
|
||||
shardHandler.submit(sreq, sreq.shards[0], sreq.params);
|
||||
|
@ -602,10 +613,18 @@ public class PeerSync {
|
|||
|
||||
SyncShardRequest sreq = (SyncShardRequest) srsp.getShardRequest();
|
||||
if (updates.size() < sreq.totalRequestedUpdates) {
|
||||
log.error(msg() + " Requested " + sreq.requestedUpdates.size() + " updates from " + sreq.shards[0] + " but retrieved " + updates.size());
|
||||
log.error(msg() + " Requested " + sreq.totalRequestedUpdates + " updates from " + sreq.shards[0] + " but retrieved " + updates.size());
|
||||
return false;
|
||||
}
|
||||
|
||||
// overwrite fingerprint we saved in 'handleVersions()'
|
||||
Object fingerprint = srsp.getSolrResponse().getResponse().get("fingerprint");
|
||||
|
||||
if (fingerprint != null) {
|
||||
sreq.fingerprint = IndexFingerprint.fromObject(fingerprint);
|
||||
}
|
||||
|
||||
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
params.set(DISTRIB_UPDATE_PARAM, FROMLEADER.toString());
|
||||
params.set("peersync",true); // debugging
|
||||
|
|
|
@ -944,10 +944,15 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
}
|
||||
|
||||
public List<Long> getVersions(int n){
|
||||
return getVersions(n, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
public List<Long> getVersions(int n, long maxVersion) {
|
||||
List<Long> ret = new ArrayList<>(n);
|
||||
|
||||
for (List<Update> singleList : updateList) {
|
||||
for (Update ptr : singleList) {
|
||||
if(Math.abs(ptr.version) > Math.abs(maxVersion)) continue;
|
||||
ret.add(ptr.version);
|
||||
if (--n <= 0) return ret;
|
||||
}
|
||||
|
|
|
@ -1027,7 +1027,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
|
||||
// leaders can also be in buffering state during "migrate" API call, see SOLR-5308
|
||||
if (forwardedFromCollection && ulog.getState() != UpdateLog.State.ACTIVE
|
||||
&& (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
&& isReplayOrPeersync == false) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
log.info("Leader logic applied but update log is buffering: " + cmd.getPrintableId());
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
|
@ -1055,7 +1055,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
// The leader forwarded us this update.
|
||||
cmd.setVersion(versionOnUpdate);
|
||||
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && isReplayOrPeersync == false) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
ulog.add(cmd);
|
||||
|
@ -1429,7 +1429,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
} else {
|
||||
cmd.setVersion(-versionOnUpdate);
|
||||
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && isReplayOrPeersync == false) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
ulog.deleteByQuery(cmd);
|
||||
|
@ -1542,7 +1542,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
|
||||
// leaders can also be in buffering state during "migrate" API call, see SOLR-5308
|
||||
if (forwardedFromCollection && ulog.getState() != UpdateLog.State.ACTIVE
|
||||
&& (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
&& !isReplayOrPeersync) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
log.info("Leader logic applied but update log is buffering: " + cmd.getId());
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
|
@ -1567,7 +1567,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
} else {
|
||||
cmd.setVersion(-versionOnUpdate);
|
||||
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
|
||||
if (ulog.getState() != UpdateLog.State.ACTIVE && isReplayOrPeersync == false) {
|
||||
// we're not in an active state, and this update isn't from a replay, so buffer it.
|
||||
cmd.setFlags(cmd.getFlags() | UpdateCommand.BUFFERING);
|
||||
ulog.delete(cmd);
|
||||
|
|
|
@ -48,7 +48,7 @@
|
|||
</requestHandler>
|
||||
|
||||
<peerSync>
|
||||
<useRangeVersions>${solr.peerSync.useRangeVersions:false}</useRangeVersions>
|
||||
<useRangeVersions>${solr.peerSync.useRangeVersions:true}</useRangeVersions>
|
||||
</peerSync>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
|
|
|
@ -0,0 +1,360 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
|
||||
package org.apache.solr.cloud;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.client.solrj.SolrQuery;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.request.UpdateRequest;
|
||||
import org.apache.solr.cloud.ZkTestServer.LimitViolationAction;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.handler.ReplicationHandler;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
/**
|
||||
* Test sync peer sync when a node restarts and documents are indexed when node was down.
|
||||
*
|
||||
* This test is modeled after SyncSliceTest
|
||||
*/
|
||||
@Slow
|
||||
public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
private boolean success = false;
|
||||
int docId = 0;
|
||||
|
||||
List<CloudJettyRunner> nodesDown = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public void distribTearDown() throws Exception {
|
||||
if (!success) {
|
||||
printLayoutOnTearDown = true;
|
||||
}
|
||||
System.clearProperty("solr.directoryFactory");
|
||||
System.clearProperty("solr.ulog.numRecordsToKeep");
|
||||
System.clearProperty("tests.zk.violationReportAction");
|
||||
super.distribTearDown();
|
||||
}
|
||||
|
||||
public PeerSyncReplicationTest() {
|
||||
super();
|
||||
sliceCount = 1;
|
||||
fixShardCount(3);
|
||||
}
|
||||
|
||||
protected String getCloudSolrConfig() {
|
||||
return "solrconfig-tlog.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void distribSetUp() throws Exception {
|
||||
// tlog gets deleted after node restarts if we use CachingDirectoryFactory.
|
||||
// make sure that tlog stays intact after we restart a node
|
||||
System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
|
||||
System.setProperty("solr.ulog.numRecordsToKeep", "1000");
|
||||
System.setProperty("tests.zk.violationReportAction", LimitViolationAction.IGNORE.toString());
|
||||
super.distribSetUp();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
handle.clear();
|
||||
handle.put("timestamp", SKIPVAL);
|
||||
|
||||
waitForThingsToLevelOut(30);
|
||||
|
||||
del("*:*");
|
||||
|
||||
// index enough docs and commit to establish frame of reference for PeerSync
|
||||
for (int i = 0; i < 100; i++) {
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1,
|
||||
"document number " + docId++);
|
||||
}
|
||||
commit();
|
||||
waitForThingsToLevelOut(30);
|
||||
|
||||
try {
|
||||
checkShardConsistency(false, true);
|
||||
|
||||
long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
|
||||
assertEquals(docId, cloudClientDocs);
|
||||
|
||||
CloudJettyRunner initialLeaderJetty = shardToLeaderJetty.get("shard1");
|
||||
List<CloudJettyRunner> otherJetties = getOtherAvailableJetties(initialLeaderJetty);
|
||||
CloudJettyRunner neverLeader = otherJetties.get(otherJetties.size() - 1);
|
||||
otherJetties.remove(neverLeader) ;
|
||||
|
||||
// first shutdown a node that will never be a leader
|
||||
forceNodeFailures(Arrays.asList(neverLeader));
|
||||
|
||||
// node failure and recovery via PeerSync
|
||||
log.info("Forcing PeerSync");
|
||||
CloudJettyRunner nodePeerSynced = forceNodeFailureAndDoPeerSync(false);
|
||||
|
||||
// add a few more docs
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1,
|
||||
"document number " + docId++);
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1,
|
||||
"document number " + docId++);
|
||||
commit();
|
||||
|
||||
cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
|
||||
assertEquals(docId, cloudClientDocs);
|
||||
|
||||
// now shutdown all other nodes except for 'nodeShutDownForFailure'
|
||||
otherJetties.remove(nodePeerSynced);
|
||||
forceNodeFailures(otherJetties);
|
||||
waitForThingsToLevelOut(30);
|
||||
checkShardConsistency(false, true);
|
||||
|
||||
// now shutdown the original leader
|
||||
log.info("Now shutting down initial leader");
|
||||
forceNodeFailures(Arrays.asList(initialLeaderJetty));
|
||||
log.info("Updating mappings from zk");
|
||||
Thread.sleep(15000); // sleep for a while for leader to change ...
|
||||
updateMappingsFromZk(jettys, clients, true);
|
||||
assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));
|
||||
|
||||
// bring up node that was down all along, and let it PeerSync from the node that was forced to PeerSynce
|
||||
bringUpDeadNodeAndEnsureNoReplication(shardToLeaderJetty.get("shard1"), neverLeader, false);
|
||||
waitTillNodesActive();
|
||||
|
||||
checkShardConsistency(false, true);
|
||||
|
||||
|
||||
// bring back all the nodes including initial leader
|
||||
// (commented as reports Maximum concurrent create/delete watches above limit violation and reports thread leaks)
|
||||
/*for(int i = 0 ; i < nodesDown.size(); i++) {
|
||||
bringUpDeadNodeAndEnsureNoReplication(shardToLeaderJetty.get("shard1"), neverLeader, false);
|
||||
}
|
||||
checkShardConsistency(false, true);*/
|
||||
|
||||
// make sure leader has not changed after bringing initial leader back
|
||||
assertEquals(nodePeerSynced, shardToLeaderJetty.get("shard1"));
|
||||
success = true;
|
||||
} finally {
|
||||
System.clearProperty("solr.disableFingerprint");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void indexInBackground(int numDocs) {
|
||||
new Thread(() -> {
|
||||
try {
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1, "document number " + docId);
|
||||
docId++;
|
||||
// slow down adds, to get documents indexed while in PeerSync
|
||||
Thread.sleep(100);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("Error indexing doc in background", e);
|
||||
//Throwing an error here will kill the thread
|
||||
}
|
||||
}, getClassName())
|
||||
.start();
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void forceNodeFailures(List<CloudJettyRunner> replicasToShutDown) throws Exception {
|
||||
for (CloudJettyRunner replicaToShutDown : replicasToShutDown) {
|
||||
chaosMonkey.killJetty(replicaToShutDown);
|
||||
waitForNoShardInconsistency();
|
||||
}
|
||||
|
||||
int totalDown = 0;
|
||||
|
||||
Set<CloudJettyRunner> jetties = new HashSet<>();
|
||||
jetties.addAll(shardToJetty.get("shard1"));
|
||||
|
||||
if (replicasToShutDown != null) {
|
||||
jetties.removeAll(replicasToShutDown);
|
||||
totalDown += replicasToShutDown.size();
|
||||
}
|
||||
|
||||
jetties.removeAll(nodesDown);
|
||||
totalDown += nodesDown.size();
|
||||
|
||||
assertEquals(getShardCount() - totalDown, jetties.size());
|
||||
|
||||
nodesDown.addAll(replicasToShutDown);
|
||||
|
||||
Thread.sleep(3000);
|
||||
}
|
||||
|
||||
|
||||
|
||||
private CloudJettyRunner forceNodeFailureAndDoPeerSync(boolean disableFingerprint)
|
||||
throws Exception {
|
||||
// kill non leader - new leader could have all the docs or be missing one
|
||||
CloudJettyRunner leaderJetty = shardToLeaderJetty.get("shard1");
|
||||
|
||||
List<CloudJettyRunner> nonLeaderJetties = getOtherAvailableJetties(leaderJetty);
|
||||
CloudJettyRunner replicaToShutDown = nonLeaderJetties.get(random().nextInt(nonLeaderJetties.size())); // random non leader node
|
||||
|
||||
forceNodeFailures(Arrays.asList(replicaToShutDown));
|
||||
|
||||
// two docs need to be sync'd back when replica restarts
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1,
|
||||
"document number " + docId++);
|
||||
indexDoc(id, docId, i1, 50, tlong, 50, t1,
|
||||
"document number " + docId++);
|
||||
commit();
|
||||
|
||||
bringUpDeadNodeAndEnsureNoReplication(leaderJetty, replicaToShutDown, disableFingerprint);
|
||||
|
||||
return replicaToShutDown;
|
||||
}
|
||||
|
||||
|
||||
|
||||
private void bringUpDeadNodeAndEnsureNoReplication(CloudJettyRunner leaderJetty, CloudJettyRunner nodeToBringUp,
|
||||
boolean disableFingerprint) throws Exception {
|
||||
// disable fingerprint check if needed
|
||||
System.setProperty("solr.disableFingerprint", String.valueOf(disableFingerprint));
|
||||
|
||||
long numRequestsBefore = (Long) leaderJetty.jetty
|
||||
.getCoreContainer()
|
||||
.getCores()
|
||||
.iterator()
|
||||
.next()
|
||||
.getRequestHandler(ReplicationHandler.PATH)
|
||||
.getStatistics().get("requests");
|
||||
|
||||
indexInBackground(50);
|
||||
|
||||
// bring back dead node and ensure it recovers
|
||||
ChaosMonkey.start(nodeToBringUp.jetty);
|
||||
|
||||
nodesDown.remove(nodeToBringUp);
|
||||
|
||||
waitTillNodesActive();
|
||||
waitForThingsToLevelOut(30);
|
||||
|
||||
Set<CloudJettyRunner> jetties = new HashSet<>();
|
||||
jetties.addAll(shardToJetty.get("shard1"));
|
||||
jetties.removeAll(nodesDown);
|
||||
assertEquals(getShardCount() - nodesDown.size(), jetties.size());
|
||||
|
||||
long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
|
||||
assertEquals(docId, cloudClientDocs);
|
||||
|
||||
long numRequestsAfter = (Long) leaderJetty.jetty
|
||||
.getCoreContainer()
|
||||
.getCores()
|
||||
.iterator()
|
||||
.next()
|
||||
.getRequestHandler(ReplicationHandler.PATH)
|
||||
.getStatistics().get("requests");
|
||||
|
||||
assertEquals("PeerSync failed. Had to fail back to replication", numRequestsBefore, numRequestsAfter);
|
||||
}
|
||||
|
||||
|
||||
|
||||
private void waitTillNodesActive() throws Exception {
|
||||
for (int i = 0; i < 60; i++) {
|
||||
Thread.sleep(3000);
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
DocCollection collection1 = clusterState.getCollection("collection1");
|
||||
Slice slice = collection1.getSlice("shard1");
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
boolean allActive = true;
|
||||
|
||||
Collection<Replica> replicasToCheck = null;
|
||||
replicasToCheck = replicas.stream().filter(r -> nodesDown.contains(r.getName()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (Replica replica : replicasToCheck) {
|
||||
if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
|
||||
allActive = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (allActive) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
printLayout();
|
||||
fail("timeout waiting to see all nodes active");
|
||||
}
|
||||
|
||||
|
||||
|
||||
private List<CloudJettyRunner> getOtherAvailableJetties(CloudJettyRunner leader) {
|
||||
List<CloudJettyRunner> candidates = new ArrayList<>();
|
||||
candidates.addAll(shardToJetty.get("shard1"));
|
||||
|
||||
if (leader != null) {
|
||||
candidates.remove(leader);
|
||||
}
|
||||
|
||||
candidates.removeAll(nodesDown);
|
||||
|
||||
return candidates;
|
||||
}
|
||||
|
||||
|
||||
|
||||
protected void indexDoc(Object... fields) throws IOException,
|
||||
SolrServerException {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
|
||||
addFields(doc, fields);
|
||||
addFields(doc, "rnd_s", RandomStringUtils.random(random().nextInt(100) + 100));
|
||||
|
||||
UpdateRequest ureq = new UpdateRequest();
|
||||
ureq.add(doc);
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
ureq.setParams(params);
|
||||
ureq.process(cloudClient);
|
||||
}
|
||||
|
||||
// skip the randoms - they can deadlock...
|
||||
@Override
|
||||
protected void indexr(Object... fields) throws Exception {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
addFields(doc, fields);
|
||||
addFields(doc, "rnd_b", true);
|
||||
indexDoc(doc);
|
||||
}
|
||||
|
||||
}
|
|
@ -22,9 +22,10 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.cloud.SolrCloudTestCase;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
|
@ -32,37 +33,48 @@ import org.apache.solr.common.cloud.ZkStateReader;
|
|||
import org.apache.solr.core.RequestParams;
|
||||
import org.apache.solr.core.TestSolrConfigHandler;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.apache.solr.handler.TestSolrConfigHandlerCloud.compareValues;
|
||||
|
||||
public class TestReqParamsAPI extends AbstractFullDistribZkTestBase {
|
||||
public class TestReqParamsAPI extends SolrCloudTestCase {
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private static String COLL_NAME = "collection1";
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (final SolrClient client : clients) {
|
||||
RestTestHarness harness = new RestTestHarness(() -> ((HttpSolrClient) client).getBaseURL());
|
||||
for (final JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
|
||||
RestTestHarness harness = new RestTestHarness(() -> jettySolrRunner.getBaseUrl().toString() + "/" + COLL_NAME);
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void distribTearDown() throws Exception {
|
||||
super.distribTearDown();
|
||||
for (RestTestHarness r : restTestHarnesses) {
|
||||
r.close();
|
||||
}
|
||||
@BeforeClass
|
||||
public static void createCluster() throws Exception {
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
configureCluster(2)
|
||||
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-managed").resolve("conf"))
|
||||
.configure();
|
||||
cluster.createCollection(COLL_NAME, 1, 2, "conf1", null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
try {
|
||||
setupHarnesses();
|
||||
testReqParams();
|
||||
} finally {
|
||||
for (RestTestHarness r : restTestHarnesses) {
|
||||
r.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void testReqParams() throws Exception {
|
||||
DocCollection coll = cloudClient.getZkStateReader().getClusterState().getCollection("collection1");
|
||||
CloudSolrClient cloudClient = cluster.getSolrClient();
|
||||
DocCollection coll = cloudClient.getZkStateReader().getClusterState().getCollection(COLL_NAME);
|
||||
List<String> urls = new ArrayList<>();
|
||||
for (Slice slice : coll.getSlices()) {
|
||||
for (Replica replica : slice.getReplicas())
|
||||
|
@ -70,14 +82,27 @@ public class TestReqParamsAPI extends AbstractFullDistribZkTestBase {
|
|||
}
|
||||
|
||||
RestTestHarness writeHarness = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||
|
||||
String payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/dump0', 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
|
||||
"}";
|
||||
|
||||
TestSolrConfigHandler.runConfigCommand(writeHarness, "/config?wt=json", payload);
|
||||
|
||||
payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/dump1', 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
|
||||
"}";
|
||||
TestSolrConfigHandler.runConfigCommand(writeHarness, "/config?wt=json", payload);
|
||||
|
||||
AbstractFullDistribZkTestBase.waitForRecoveriesToFinish(COLL_NAME, cloudClient.getZkStateReader(), false, true, 90);
|
||||
|
||||
payload = " {\n" +
|
||||
" 'set' : {'x': {" +
|
||||
" 'a':'A val',\n" +
|
||||
" 'b': 'B val'}\n" +
|
||||
" }\n" +
|
||||
" }";
|
||||
|
||||
|
||||
TestSolrConfigHandler.runConfigCommand(writeHarness, "/config/params?wt=json", payload);
|
||||
|
||||
Map result = TestSolrConfigHandler.testForResponseElement(null,
|
||||
|
@ -89,12 +114,6 @@ public class TestReqParamsAPI extends AbstractFullDistribZkTestBase {
|
|||
10);
|
||||
compareValues(result, "B val", asList("response", "params", "x", "b"));
|
||||
|
||||
payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/dump0', 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
|
||||
"}";
|
||||
|
||||
TestSolrConfigHandler.runConfigCommand(writeHarness, "/config?wt=json", payload);
|
||||
|
||||
TestSolrConfigHandler.testForResponseElement(null,
|
||||
urls.get(random().nextInt(urls.size())),
|
||||
"/config/overlay?wt=json",
|
||||
|
@ -120,12 +139,6 @@ public class TestReqParamsAPI extends AbstractFullDistribZkTestBase {
|
|||
"fomrequest",
|
||||
5);
|
||||
|
||||
payload = "{\n" +
|
||||
"'create-requesthandler' : { 'name' : '/dump1', 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
|
||||
"}";
|
||||
|
||||
TestSolrConfigHandler.runConfigCommand(writeHarness, "/config?wt=json", payload);
|
||||
|
||||
result = TestSolrConfigHandler.testForResponseElement(null,
|
||||
urls.get(random().nextInt(urls.size())),
|
||||
"/config/overlay?wt=json",
|
||||
|
@ -263,9 +276,5 @@ public class TestReqParamsAPI extends AbstractFullDistribZkTestBase {
|
|||
asList("response", "params", "y", "p"),
|
||||
null,
|
||||
10);
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.solr.request.macro;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* Tests for the MacroExpander
|
||||
*/
|
||||
public class TestMacroExpander extends LuceneTestCase {
|
||||
|
||||
@Test
|
||||
public void testExamples() {
|
||||
final Map<String,String[]> testParams = new HashMap<String,String[]>();
|
||||
final MacroExpander me;
|
||||
// example behavior unaffected by absence or value of failOnMissingParams
|
||||
if (random().nextBoolean()) {
|
||||
me = new MacroExpander(testParams);
|
||||
} else {
|
||||
final boolean failOnMissingParams = random().nextBoolean();
|
||||
me = new MacroExpander(testParams, failOnMissingParams);
|
||||
}
|
||||
|
||||
//default examples: https://cwiki.apache.org/confluence/display/solr/Parameter+Substitution
|
||||
// and http://yonik.com/solr-query-parameter-substitution/
|
||||
|
||||
//using params
|
||||
String[] lowParams = {"50"};
|
||||
testParams.put("low",lowParams);
|
||||
String[] highParams = {"100"};
|
||||
testParams.put("high",highParams);
|
||||
|
||||
String testQuery = "q=popularity:[ ${low} TO ${high} ]";
|
||||
|
||||
assertEquals("q=popularity:[ 50 TO 100 ]", me.expand(testQuery));
|
||||
|
||||
//using default values
|
||||
testQuery = "q=popularity:[ ${low:10} TO ${high:20} ]";
|
||||
assertEquals("q=popularity:[ 50 TO 100 ]", me.expand(testQuery));
|
||||
|
||||
testParams.clear();
|
||||
assertEquals("q=popularity:[ 10 TO 20 ]", me.expand(testQuery));
|
||||
|
||||
//multiple levels of substitutions
|
||||
testQuery = "q=${pop_query}";
|
||||
String[] popQueryParams = {"${pop_field}:[ ${low} TO ${high} ] AND inStock:true"};
|
||||
String[] popFieldParams = {"popularity"};
|
||||
testParams.put("low",lowParams);
|
||||
testParams.put("high",highParams);
|
||||
testParams.put("pop_query",popQueryParams);
|
||||
testParams.put("pop_field",popFieldParams);
|
||||
|
||||
assertEquals("q=popularity:[ 50 TO 100 ] AND inStock:true", me.expand(testQuery));
|
||||
|
||||
//end default examples
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOnMissingParams() {
|
||||
final Map<String,String[]> testParams = new HashMap<String,String[]>();
|
||||
final MacroExpander meSkipOnMissingParams = new MacroExpander(testParams);
|
||||
final MacroExpander meFailOnMissingParams = new MacroExpander(testParams, true);
|
||||
|
||||
final String low = "50";
|
||||
final String high = "100";
|
||||
testParams.put("low", new String[]{ low });
|
||||
testParams.put("high", new String[]{ high });
|
||||
|
||||
final String testQuery = "q=popularity:[ ${low} TO ${high} ]";
|
||||
|
||||
//when params all present the expansion results match
|
||||
final String expandedQuery = "q=popularity:[ "+low+" TO "+high+" ]";
|
||||
assertEquals(expandedQuery, meSkipOnMissingParams.expand(testQuery));
|
||||
assertEquals(expandedQuery, meFailOnMissingParams.expand(testQuery));
|
||||
|
||||
//when param(s) missing and have no default the expansion results differ
|
||||
final String expandedLow;
|
||||
final String expandedHigh;
|
||||
if (random().nextBoolean()) { // keep low
|
||||
expandedLow = low;
|
||||
} else {
|
||||
expandedLow = "";
|
||||
testParams.remove("low");
|
||||
}
|
||||
if (random().nextBoolean()) { // keep high
|
||||
expandedHigh = high;
|
||||
} else {
|
||||
expandedHigh = "";
|
||||
testParams.remove("high");
|
||||
}
|
||||
assertEquals("q=popularity:[ "+expandedLow+" TO "+expandedHigh+" ]",
|
||||
meSkipOnMissingParams.expand(testQuery));
|
||||
if (testParams.size() < 2) { // at least one of the two parameters missing
|
||||
assertEquals(null, meFailOnMissingParams.expand(testQuery));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.queryparser.xml.builders;
|
||||
package org.apache.solr.search;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -32,7 +32,7 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
public class TestNumericRangeQueryBuilder extends LuceneTestCase {
|
||||
public class TestLegacyNumericRangeQueryBuilder extends LuceneTestCase {
|
||||
|
||||
public void testGetFilterHandleNumericParseErrorStrict() throws Exception {
|
||||
LegacyNumericRangeQueryBuilder filterBuilder = new LegacyNumericRangeQueryBuilder();
|
|
@ -192,10 +192,8 @@ public class ZkStateReader implements Closeable {
|
|||
} else {
|
||||
throw new ZooKeeperException(ErrorCode.INVALID_STATE, "No config data found at path: " + path);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Error loading config name for collection " + collection, e);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (KeeperException| InterruptedException e) {
|
||||
SolrZkClient.checkInterrupted(e);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Error loading config name for collection " + collection, e);
|
||||
}
|
||||
|
||||
|
|
|
@ -73,6 +73,19 @@ public class JSONTestUtil {
|
|||
return match(path, input, expected, delta);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param input Object structure to parse and test against
|
||||
* @param pathAndExpected JSON path expression + '==' + expected value
|
||||
* @param delta tollerance allowed in comparing float/double values
|
||||
*/
|
||||
public static String matchObj(Object input, String pathAndExpected, double delta) throws Exception {
|
||||
int pos = pathAndExpected.indexOf("==");
|
||||
String path = pos>=0 ? pathAndExpected.substring(0,pos) : null;
|
||||
String expected = pos>=0 ? pathAndExpected.substring(pos+2) : pathAndExpected;
|
||||
Object expectObj = failRepeatedKeys ? new NoDupsObjectBuilder(new JSONParser(expected)).getVal() : ObjectBuilder.fromJSON(expected);
|
||||
return matchObj(path, input, expectObj, delta);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path JSON path expression
|
||||
* @param input JSON Structure to parse and test against
|
||||
|
|
Loading…
Reference in New Issue