mirror of https://github.com/apache/lucene.git
LUCENE-5752: merge trunk
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5752@1603492 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
8f0c046ef6
|
@ -94,6 +94,10 @@ Other
|
|||
* LUCENE-5563: Removed sep layout: which has fallen behind on features and doesn't
|
||||
perform as well as other options. (Robert Muir)
|
||||
|
||||
======================= Lucene 4.10.0 ======================
|
||||
|
||||
(No Changes)
|
||||
|
||||
======================= Lucene 4.9.0 =======================
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
|
|
@ -59,8 +59,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
private final static int PAGED_BYTES_BITS = 15;
|
||||
|
||||
// all fields share this single logical byte[]
|
||||
private final PagedBytes termBytes = new PagedBytes(PAGED_BYTES_BITS);
|
||||
private PagedBytes.Reader termBytesReader;
|
||||
private final PagedBytes.Reader termBytesReader;
|
||||
|
||||
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
|
||||
|
||||
|
@ -71,6 +70,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
|
||||
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
|
||||
throws IOException {
|
||||
final PagedBytes termBytes = new PagedBytes(PAGED_BYTES_BITS);
|
||||
|
||||
this.termComp = termComp;
|
||||
|
||||
|
@ -115,7 +115,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
throw new CorruptIndexException("invalid packedIndexStart: " + packedIndexStart + " indexStart: " + indexStart + "numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
|
||||
}
|
||||
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
|
||||
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
|
||||
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
|
||||
if (previous != null) {
|
||||
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
|
||||
}
|
||||
|
@ -238,7 +238,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
final long numIndexTerms;
|
||||
final long termsStart;
|
||||
|
||||
public FieldIndexData(IndexInput in, long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, long numIndexTerms) throws IOException {
|
||||
public FieldIndexData(IndexInput in, PagedBytes termBytes, long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, long numIndexTerms) throws IOException {
|
||||
|
||||
this.termsStart = termsStart;
|
||||
termBytesStart = termBytes.getPointer();
|
||||
|
@ -292,9 +292,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = ((termBytes!=null) ? termBytes.ramBytesUsed() : 0) +
|
||||
((termBytesReader!=null)? termBytesReader.ramBytesUsed() : 0);
|
||||
|
||||
long sizeInBytes = ((termBytesReader!=null)? termBytesReader.ramBytesUsed() : 0);
|
||||
for(FieldIndexData entry : fields.values()) {
|
||||
sizeInBytes += entry.ramBytesUsed();
|
||||
}
|
||||
|
|
|
@ -151,7 +151,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
return new BloomFilteredFieldsProducer(state);
|
||||
}
|
||||
|
||||
public class BloomFilteredFieldsProducer extends FieldsProducer {
|
||||
static class BloomFilteredFieldsProducer extends FieldsProducer {
|
||||
private FieldsProducer delegateFieldsProducer;
|
||||
HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<>();
|
||||
|
||||
|
|
|
@ -169,11 +169,16 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
|
||||
private final static class DirectField extends Terms implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DirectField.class);
|
||||
|
||||
private static abstract class TermAndSkip implements Accountable {
|
||||
public int[] skips;
|
||||
}
|
||||
|
||||
private static final class LowFreqTerm extends TermAndSkip {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(HighFreqTerm.class);
|
||||
|
||||
public final int[] postings;
|
||||
public final byte[] payloads;
|
||||
public final int docFreq;
|
||||
|
@ -188,13 +193,17 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
|
||||
return BASE_RAM_BYTES_USED +
|
||||
((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
|
||||
((payloads!=null) ? RamUsageEstimator.sizeOf(payloads) : 0);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: maybe specialize into prx/no-prx/no-frq cases?
|
||||
private static final class HighFreqTerm extends TermAndSkip {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(HighFreqTerm.class);
|
||||
|
||||
public final long totalTermFreq;
|
||||
public final int[] docIDs;
|
||||
public final int[] freqs;
|
||||
|
@ -211,19 +220,22 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
long sizeInBytes = BASE_RAM_BYTES_USED;
|
||||
sizeInBytes += (docIDs!=null)? RamUsageEstimator.sizeOf(docIDs) : 0;
|
||||
sizeInBytes += (freqs!=null)? RamUsageEstimator.sizeOf(freqs) : 0;
|
||||
|
||||
if(positions != null) {
|
||||
sizeInBytes += RamUsageEstimator.shallowSizeOf(positions);
|
||||
for(int[] position : positions) {
|
||||
sizeInBytes += (position!=null) ? RamUsageEstimator.sizeOf(position) : 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (payloads != null) {
|
||||
sizeInBytes += RamUsageEstimator.shallowSizeOf(payloads);
|
||||
for(byte[][] payload : payloads) {
|
||||
if(payload != null) {
|
||||
sizeInBytes += RamUsageEstimator.shallowSizeOf(payload);
|
||||
for(byte[] pload : payload) {
|
||||
sizeInBytes += (pload!=null) ? RamUsageEstimator.sizeOf(pload) : 0;
|
||||
}
|
||||
|
@ -494,7 +506,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
long sizeInBytes = BASE_RAM_BYTES_USED;
|
||||
sizeInBytes += ((termBytes!=null) ? RamUsageEstimator.sizeOf(termBytes) : 0);
|
||||
sizeInBytes += ((termOffsets!=null) ? RamUsageEstimator.sizeOf(termOffsets) : 0);
|
||||
sizeInBytes += ((skips!=null) ? RamUsageEstimator.sizeOf(skips) : 0);
|
||||
|
@ -502,6 +514,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
sizeInBytes += ((sameCounts!=null) ? RamUsageEstimator.sizeOf(sameCounts) : 0);
|
||||
|
||||
if(terms!=null) {
|
||||
sizeInBytes += RamUsageEstimator.shallowSizeOf(terms);
|
||||
for(TermAndSkip termAndSkip : terms) {
|
||||
sizeInBytes += (termAndSkip!=null) ? termAndSkip.ramBytesUsed() : 0;
|
||||
}
|
||||
|
|
|
@ -827,7 +827,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long ramBytesUsed = 0;
|
||||
long ramBytesUsed = postingsReader.ramBytesUsed();
|
||||
for (TermsReader r : fields.values()) {
|
||||
if (r.index != null) {
|
||||
ramBytesUsed += r.index.ramBytesUsed();
|
||||
|
|
|
@ -24,6 +24,8 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.Outputs;
|
||||
|
||||
/**
|
||||
|
@ -46,7 +48,8 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
|
|||
* On an FST, only long[] part is 'shared' and pushed towards root.
|
||||
* byte[] and term stats will be kept on deeper arcs.
|
||||
*/
|
||||
static class TermData {
|
||||
static class TermData implements Accountable {
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermData.class);
|
||||
long[] longs;
|
||||
byte[] bytes;
|
||||
int docFreq;
|
||||
|
@ -64,6 +67,18 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
|
|||
this.totalTermFreq = totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long ramBytesUsed = BASE_RAM_BYTES_USED;
|
||||
if (longs != null) {
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(longs);
|
||||
}
|
||||
if (bytes != null) {
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(bytes);
|
||||
}
|
||||
return ramBytesUsed;
|
||||
}
|
||||
|
||||
// NOTE: actually, FST nodes are seldom
|
||||
// identical when outputs on their arcs
|
||||
// aren't NO_OUTPUTs.
|
||||
|
@ -112,6 +127,11 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
|
|||
this.longsSize = longsSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed(TermData output) {
|
||||
return output.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
//
|
||||
// The return value will be the smaller one, when these two are
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.automaton.ByteRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -165,7 +166,9 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
final class TermsReader extends Terms {
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsReader.class);
|
||||
final class TermsReader extends Terms implements Accountable {
|
||||
|
||||
final FieldInfo fieldInfo;
|
||||
final long numTerms;
|
||||
final long sumTotalTermFreq;
|
||||
|
@ -184,6 +187,11 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
this.dict = new FST<>(in, new FSTTermOutputs(fieldInfo, longsSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return BASE_RAM_BYTES_USED + dict.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasFreqs() {
|
||||
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
|
||||
|
@ -732,9 +740,9 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long ramBytesUsed = 0;
|
||||
long ramBytesUsed = postingsReader.ramBytesUsed();
|
||||
for (TermsReader r : fields.values()) {
|
||||
ramBytesUsed += r.dict == null ? 0 : r.dict.ramBytesUsed();
|
||||
ramBytesUsed += r.ramBytesUsed();
|
||||
}
|
||||
return ramBytesUsed;
|
||||
}
|
||||
|
|
|
@ -55,11 +55,17 @@ import org.apache.lucene.store.ChecksumIndexInput;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
class SimpleTextDocValuesReader extends DocValuesProducer {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SimpleTextDocValuesReader.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
|
||||
|
||||
static class OneField {
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OneField.class);
|
||||
long dataStartFilePointer;
|
||||
String pattern;
|
||||
String ordPattern;
|
||||
|
@ -500,7 +506,8 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(scratch.bytes)
|
||||
+ fields.size() * (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2L + OneField.BASE_RAM_BYTES_USED);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.lucene.util.CharsRef;
|
|||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
|
@ -65,6 +66,11 @@ import org.apache.lucene.util.fst.PositiveIntOutputs;
|
|||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
class SimpleTextFieldsReader extends FieldsProducer {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SimpleTextFieldsReader.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(TreeMap.class);
|
||||
|
||||
private final TreeMap<String,Long> fields;
|
||||
private final IndexInput in;
|
||||
private final FieldInfos fieldInfos;
|
||||
|
@ -497,6 +503,10 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
|
||||
private static final long TERMS_BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SimpleTextTerms.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
|
||||
private class SimpleTextTerms extends Terms implements Accountable {
|
||||
private final long termsStart;
|
||||
private final FieldInfo fieldInfo;
|
||||
|
@ -581,7 +591,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return (fst!=null) ? fst.ramBytesUsed() : 0;
|
||||
return TERMS_BASE_RAM_BYTES_USED + (fst!=null ? fst.ramBytesUsed() : 0)
|
||||
+ RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -643,14 +654,14 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
synchronized public Terms terms(String field) throws IOException {
|
||||
Terms terms = termsCache.get(field);
|
||||
SimpleTextTerms terms = termsCache.get(field);
|
||||
if (terms == null) {
|
||||
Long fp = fields.get(field);
|
||||
if (fp == null) {
|
||||
return null;
|
||||
} else {
|
||||
terms = new SimpleTextTerms(field, fp, maxDoc);
|
||||
termsCache.put(field, (SimpleTextTerms) terms);
|
||||
termsCache.put(field, terms);
|
||||
}
|
||||
}
|
||||
return terms;
|
||||
|
@ -667,8 +678,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
public synchronized long ramBytesUsed() {
|
||||
long sizeInBytes = BASE_RAM_BYTES_USED + fields.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
for(SimpleTextTerms simpleTextTerms : termsCache.values()) {
|
||||
sizeInBytes += (simpleTextTerms!=null) ? simpleTextTerms.ramBytesUsed() : 0;
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
|
@ -48,6 +49,12 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextStoredFieldsWriter.*
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SimpleTextStoredFieldsReader.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
|
||||
|
||||
private long offsets[]; /* docid -> offset in .fld file */
|
||||
private IndexInput in;
|
||||
private BytesRef scratch = new BytesRef();
|
||||
|
@ -199,7 +206,8 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets)
|
||||
+ RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -43,8 +43,10 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
|
||||
|
||||
/**
|
||||
|
@ -54,6 +56,12 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SimpleTextTermVectorsReader.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
|
||||
|
||||
private long offsets[]; /* docid -> offset in .vec file */
|
||||
private IndexInput in;
|
||||
private BytesRef scratch = new BytesRef();
|
||||
|
@ -539,7 +547,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Basic tests of a PF using FixedGap terms dictionary
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.codecs.memory;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests DirectPostingsFormat
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.codecs.memory;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests MemoryPostingsFormat
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.store.ByteArrayDataInput;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.fst.ByteSequenceOutputs;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -34,6 +35,11 @@ import org.apache.lucene.util.fst.FST;
|
|||
/** BlockTree's implementation of {@link Terms}. */
|
||||
// public for CheckIndex:
|
||||
public final class FieldReader extends Terms implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(FieldReader.class)
|
||||
+ 3 * RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
|
||||
|
||||
final long numTerms;
|
||||
final FieldInfo fieldInfo;
|
||||
final long sumTotalTermFreq;
|
||||
|
@ -172,6 +178,6 @@ public final class FieldReader extends Terms implements Accountable {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((index!=null)? index.ramBytesUsed() : 0);
|
||||
return BASE_RAM_BYTES_USED + ((index!=null)? index.ramBytesUsed() : 0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,6 +36,8 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
*/
|
||||
public final class CompressingStoredFieldsIndexReader implements Cloneable, Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(CompressingStoredFieldsIndexReader.class);
|
||||
|
||||
final int maxDoc;
|
||||
final int[] docBases;
|
||||
final long[] startPointers;
|
||||
|
@ -163,12 +165,14 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long res = 0;
|
||||
long res = BASE_RAM_BYTES_USED;
|
||||
|
||||
for(PackedInts.Reader r : docBasesDeltas) {
|
||||
res += RamUsageEstimator.shallowSizeOf(docBasesDeltas);
|
||||
for (PackedInts.Reader r : docBasesDeltas) {
|
||||
res += r.ramBytesUsed();
|
||||
}
|
||||
for(PackedInts.Reader r : startPointersDeltas) {
|
||||
res += RamUsageEstimator.shallowSizeOf(startPointersDeltas);
|
||||
for (PackedInts.Reader r : startPointersDeltas) {
|
||||
res += r.ramBytesUsed();
|
||||
}
|
||||
|
||||
|
|
|
@ -328,7 +328,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||
CodecUtil.checkEOF(input);
|
||||
success = true;
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
||||
return new BinaryDocValues() {
|
||||
|
||||
@Override
|
||||
|
@ -370,7 +370,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
CodecUtil.checkEOF(data);
|
||||
CodecUtil.checkEOF(index);
|
||||
success = true;
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
public BytesRef get(int docID) {
|
||||
|
@ -414,7 +414,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
final PackedInts.Reader reader = PackedInts.getReader(index);
|
||||
CodecUtil.checkEOF(data);
|
||||
CodecUtil.checkEOF(index);
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
|
||||
success = true;
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
|
@ -457,7 +457,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
final PackedInts.Reader reader = PackedInts.getReader(index);
|
||||
CodecUtil.checkEOF(data);
|
||||
CodecUtil.checkEOF(index);
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
|
||||
success = true;
|
||||
return new BinaryDocValues() {
|
||||
|
||||
|
@ -540,7 +540,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
bytes.copy(data, fixedLength * (long) valueCount);
|
||||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||
final PackedInts.Reader reader = PackedInts.getReader(index);
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
|
||||
|
||||
return correctBuggyOrds(new SortedDocValues() {
|
||||
@Override
|
||||
|
@ -578,7 +578,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
final PackedInts.Reader ordsReader = PackedInts.getReader(index);
|
||||
|
||||
final int valueCount = addressReader.size() - 1;
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
|
||||
|
||||
return correctBuggyOrds(new SortedDocValues() {
|
||||
@Override
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
@ -47,6 +48,9 @@ import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*;
|
|||
* @lucene.internal
|
||||
*/
|
||||
public final class Lucene40StoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
|
||||
|
||||
private static final long RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene40StoredFieldsReader.class);
|
||||
|
||||
private final FieldInfos fieldInfos;
|
||||
private final IndexInput fieldsStream;
|
||||
private final IndexInput indexStream;
|
||||
|
@ -248,7 +252,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
return RAM_BYTES_USED;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Concrete class that reads docId(maybe frq,pos,offset,payloads) list
|
||||
|
@ -53,6 +54,8 @@ import org.apache.lucene.util.IOUtils;
|
|||
*/
|
||||
public final class Lucene41PostingsReader extends PostingsReaderBase {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41PostingsReader.class);
|
||||
|
||||
private final IndexInput docIn;
|
||||
private final IndexInput posIn;
|
||||
private final IndexInput payIn;
|
||||
|
@ -1543,7 +1546,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
return BASE_RAM_BYTES_USED;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -291,7 +291,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||
if (entry.minLength == entry.maxLength) {
|
||||
final int fixedLength = entry.minLength;
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
||||
return new BinaryDocValues() {
|
||||
@Override
|
||||
public BytesRef get(int docID) {
|
||||
|
@ -302,7 +302,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
};
|
||||
} else {
|
||||
final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
||||
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addresses.ramBytesUsed());
|
||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
|
||||
return new BinaryDocValues() {
|
||||
|
||||
@Override
|
||||
|
|
|
@ -199,7 +199,9 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
private class FieldsReader extends FieldsProducer {
|
||||
private static class FieldsReader extends FieldsProducer {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FieldsReader.class);
|
||||
|
||||
private final Map<String,FieldsProducer> fields = new TreeMap<>();
|
||||
private final Map<String,FieldsProducer> formats = new HashMap<>();
|
||||
|
@ -258,12 +260,13 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
long ramBytesUsed = BASE_RAM_BYTES_USED;
|
||||
ramBytesUsed += fields.size() * 2L * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
ramBytesUsed += formats.size() * 2L * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
for(Map.Entry<String,FieldsProducer> entry: formats.entrySet()) {
|
||||
sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
|
||||
sizeInBytes += entry.getValue().ramBytesUsed();
|
||||
ramBytesUsed += entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
return ramBytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -39,11 +39,14 @@ import org.apache.lucene.store.IOContext;
|
|||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Holds core readers that are shared (unchanged) when
|
||||
* SegmentReader is cloned or reopened */
|
||||
final class SegmentCoreReaders implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(SegmentCoreReaders.class);
|
||||
|
||||
// Counts how many other readers share the core objects
|
||||
// (freqStream, proxStream, tis, etc.) of this reader;
|
||||
// when coreRef drops to 0, these core objects may be
|
||||
|
@ -213,7 +216,8 @@ final class SegmentCoreReaders implements Accountable {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
|
||||
return BASE_RAM_BYTES_USED +
|
||||
((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
|
||||
((fields!=null) ? fields.ramBytesUsed() : 0) +
|
||||
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
|
||||
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
|
@ -51,6 +52,11 @@ import org.apache.lucene.util.Version;
|
|||
*/
|
||||
public final class SegmentReader extends AtomicReader implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SegmentReader.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(SegmentDocValues.class);
|
||||
private static final long LONG_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Long.class);
|
||||
|
||||
private final SegmentCommitInfo si;
|
||||
private final Bits liveDocs;
|
||||
|
||||
|
@ -594,7 +600,10 @@ public final class SegmentReader extends AtomicReader implements Accountable {
|
|||
@Override
|
||||
public long ramBytesUsed() {
|
||||
ensureOpen();
|
||||
long ramBytesUsed = 0;
|
||||
long ramBytesUsed = BASE_RAM_BYTES_USED;
|
||||
ramBytesUsed += dvGens.size() * LONG_RAM_BYTES_USED;
|
||||
ramBytesUsed += dvProducers.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
ramBytesUsed += dvProducersByField.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
if (dvProducers != null) {
|
||||
for (DocValuesProducer producer : dvProducers) {
|
||||
ramBytesUsed += producer.ramBytesUsed();
|
||||
|
|
|
@ -18,8 +18,7 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
|
@ -35,9 +34,10 @@ import org.apache.lucene.store.IndexInput;
|
|||
// TODO: refactor this, byteblockpool, fst.bytestore, and any
|
||||
// other "shift/mask big arrays". there are too many of these classes!
|
||||
public final class PagedBytes implements Accountable {
|
||||
private final List<byte[]> blocks = new ArrayList<>();
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(PagedBytes.class);
|
||||
private byte[][] blocks = new byte[16][];
|
||||
private int numBlocks;
|
||||
// TODO: these are unused?
|
||||
private final List<Integer> blockEnd = new ArrayList<>();
|
||||
private final int blockSize;
|
||||
private final int blockBits;
|
||||
private final int blockMask;
|
||||
|
@ -54,24 +54,19 @@ public final class PagedBytes implements Accountable {
|
|||
*
|
||||
* @see #freeze */
|
||||
public final static class Reader implements Accountable {
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Reader.class);
|
||||
private final byte[][] blocks;
|
||||
private final int[] blockEnds;
|
||||
private final int blockBits;
|
||||
private final int blockMask;
|
||||
private final int blockSize;
|
||||
private final long bytesUsedPerBlock;
|
||||
|
||||
private Reader(PagedBytes pagedBytes) {
|
||||
blocks = new byte[pagedBytes.blocks.size()][];
|
||||
for(int i=0;i<blocks.length;i++) {
|
||||
blocks[i] = pagedBytes.blocks.get(i);
|
||||
}
|
||||
blockEnds = new int[blocks.length];
|
||||
for(int i=0;i< blockEnds.length;i++) {
|
||||
blockEnds[i] = pagedBytes.blockEnd.get(i);
|
||||
}
|
||||
blocks = Arrays.copyOf(pagedBytes.blocks, pagedBytes.numBlocks);
|
||||
blockBits = pagedBytes.blockBits;
|
||||
blockMask = pagedBytes.blockMask;
|
||||
blockSize = pagedBytes.blockSize;
|
||||
bytesUsedPerBlock = pagedBytes.bytesUsedPerBlock;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -132,7 +127,12 @@ public final class PagedBytes implements Accountable {
|
|||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((blocks!=null) ? (blockSize * blocks.length) : 0);
|
||||
long size = BASE_RAM_BYTES_USED + RamUsageEstimator.shallowSizeOf(blocks);
|
||||
if (blocks.length > 0) {
|
||||
size += (blocks.length - 1) * bytesUsedPerBlock;
|
||||
size += RamUsageEstimator.sizeOf(blocks[blocks.length - 1]);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -144,7 +144,15 @@ public final class PagedBytes implements Accountable {
|
|||
this.blockBits = blockBits;
|
||||
blockMask = blockSize-1;
|
||||
upto = blockSize;
|
||||
bytesUsedPerBlock = blockSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
bytesUsedPerBlock = RamUsageEstimator.alignObjectSize(blockSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER);
|
||||
numBlocks = 0;
|
||||
}
|
||||
|
||||
private void addBlock(byte[] block) {
|
||||
if (blocks.length == numBlocks) {
|
||||
blocks = Arrays.copyOf(blocks, ArrayUtil.oversize(numBlocks, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
|
||||
}
|
||||
blocks[numBlocks++] = block;
|
||||
}
|
||||
|
||||
/** Read this many bytes from in */
|
||||
|
@ -153,8 +161,7 @@ public final class PagedBytes implements Accountable {
|
|||
int left = blockSize - upto;
|
||||
if (left == 0) {
|
||||
if (currentBlock != null) {
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
}
|
||||
currentBlock = new byte[blockSize];
|
||||
upto = 0;
|
||||
|
@ -179,8 +186,7 @@ public final class PagedBytes implements Accountable {
|
|||
int left = blockSize - upto;
|
||||
if (bytes.length > left || currentBlock==null) {
|
||||
if (currentBlock != null) {
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
didSkipBytes = true;
|
||||
}
|
||||
currentBlock = new byte[blockSize];
|
||||
|
@ -214,8 +220,7 @@ public final class PagedBytes implements Accountable {
|
|||
if (currentBlock == null) {
|
||||
currentBlock = EMPTY_BYTES;
|
||||
}
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
frozen = true;
|
||||
currentBlock = null;
|
||||
return new PagedBytes.Reader(this);
|
||||
|
@ -225,13 +230,18 @@ public final class PagedBytes implements Accountable {
|
|||
if (currentBlock == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return (blocks.size() * ((long) blockSize)) + upto;
|
||||
return (numBlocks * ((long) blockSize)) + upto;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return (blocks.size() + (currentBlock != null ? 1 : 0)) * bytesUsedPerBlock;
|
||||
long size = BASE_RAM_BYTES_USED + RamUsageEstimator.shallowSizeOf(blocks)
|
||||
+ bytesUsedPerBlock * numBlocks;
|
||||
if (currentBlock != null) {
|
||||
size += RamUsageEstimator.sizeOf(currentBlock);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
/** Copy bytes in, writing the length as a 1 or 2 byte
|
||||
|
@ -247,8 +257,7 @@ public final class PagedBytes implements Accountable {
|
|||
throw new IllegalArgumentException("block size " + blockSize + " is too small to store length " + bytes.length + " bytes");
|
||||
}
|
||||
if (currentBlock != null) {
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
}
|
||||
currentBlock = new byte[blockSize];
|
||||
upto = 0;
|
||||
|
@ -274,7 +283,7 @@ public final class PagedBytes implements Accountable {
|
|||
private byte[] currentBlock;
|
||||
|
||||
PagedBytesDataInput() {
|
||||
currentBlock = blocks.get(0);
|
||||
currentBlock = blocks[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -293,7 +302,7 @@ public final class PagedBytes implements Accountable {
|
|||
* {@link #getPosition}. */
|
||||
public void setPosition(long pos) {
|
||||
currentBlockIndex = (int) (pos >> blockBits);
|
||||
currentBlock = blocks.get(currentBlockIndex);
|
||||
currentBlock = blocks[currentBlockIndex];
|
||||
currentBlockUpto = (int) (pos & blockMask);
|
||||
}
|
||||
|
||||
|
@ -332,7 +341,7 @@ public final class PagedBytes implements Accountable {
|
|||
private void nextBlock() {
|
||||
currentBlockIndex++;
|
||||
currentBlockUpto = 0;
|
||||
currentBlock = blocks.get(currentBlockIndex);
|
||||
currentBlock = blocks[currentBlockIndex];
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -341,8 +350,7 @@ public final class PagedBytes implements Accountable {
|
|||
public void writeByte(byte b) {
|
||||
if (upto == blockSize) {
|
||||
if (currentBlock != null) {
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
}
|
||||
currentBlock = new byte[blockSize];
|
||||
upto = 0;
|
||||
|
@ -359,8 +367,7 @@ public final class PagedBytes implements Accountable {
|
|||
|
||||
if (upto == blockSize) {
|
||||
if (currentBlock != null) {
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(upto);
|
||||
addBlock(currentBlock);
|
||||
}
|
||||
currentBlock = new byte[blockSize];
|
||||
upto = 0;
|
||||
|
@ -372,8 +379,7 @@ public final class PagedBytes implements Accountable {
|
|||
final int blockLeft = blockSize - upto;
|
||||
if (blockLeft < left) {
|
||||
System.arraycopy(b, offset, currentBlock, upto, blockLeft);
|
||||
blocks.add(currentBlock);
|
||||
blockEnd.add(blockSize);
|
||||
addBlock(currentBlock);
|
||||
currentBlock = new byte[blockSize];
|
||||
upto = 0;
|
||||
offset += blockLeft;
|
||||
|
|
|
@ -103,6 +103,13 @@ public enum Version {
|
|||
@Deprecated
|
||||
LUCENE_4_9,
|
||||
|
||||
/**
|
||||
* Match settings and bugs in Lucene's 4.10 release.
|
||||
* @deprecated (5.0) Use latest
|
||||
*/
|
||||
@Deprecated
|
||||
LUCENE_4_10,
|
||||
|
||||
/** Match settings and bugs in Lucene's 5.0 release.
|
||||
* <p>
|
||||
* Use this to get the latest & greatest settings, bug
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* An FST {@link Outputs} implementation where each output
|
||||
|
@ -145,4 +146,9 @@ public final class ByteSequenceOutputs extends Outputs<BytesRef> {
|
|||
public String outputToString(BytesRef output) {
|
||||
return output.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed(BytesRef output) {
|
||||
return super.ramBytesUsed(output) + RamUsageEstimator.sizeOf(output.bytes);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,11 +23,17 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
// TODO: merge with PagedBytes, except PagedBytes doesn't
|
||||
// let you read while writing which FST needs
|
||||
|
||||
class BytesStore extends DataOutput {
|
||||
class BytesStore extends DataOutput implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(BytesStore.class)
|
||||
+ RamUsageEstimator.shallowSizeOfInstance(ArrayList.class);
|
||||
|
||||
private final List<byte[]> blocks = new ArrayList<>();
|
||||
|
||||
|
@ -465,4 +471,14 @@ class BytesStore extends DataOutput {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long size = BASE_RAM_BYTES_USED;
|
||||
for (byte[] block : blocks) {
|
||||
size += RamUsageEstimator.sizeOf(block);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.lucene.util.Constants;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.Builder.UnCompiledNode;
|
||||
import org.apache.lucene.util.packed.GrowableWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
@ -69,6 +70,10 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public final class FST<T> implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FST.class);
|
||||
private static final long ARC_SHALLOW_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Arc.class);
|
||||
|
||||
/** Specifies allowed range of each int input label for
|
||||
* this FST. */
|
||||
public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
|
||||
|
@ -390,15 +395,38 @@ public final class FST<T> implements Accountable {
|
|||
return inputType;
|
||||
}
|
||||
|
||||
private long ramBytesUsed(Arc<T>[] arcs) {
|
||||
long size = 0;
|
||||
if (arcs != null) {
|
||||
size += RamUsageEstimator.shallowSizeOf(arcs);
|
||||
for (Arc<T> arc : arcs) {
|
||||
if (arc != null) {
|
||||
size += ARC_SHALLOW_RAM_BYTES_USED;
|
||||
if (arc.output != null && arc.output != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.output);
|
||||
}
|
||||
if (arc.nextFinalOutput != null && arc.nextFinalOutput != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.nextFinalOutput);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long size = bytes.getPosition();
|
||||
long size = BASE_RAM_BYTES_USED;
|
||||
size += bytes.ramBytesUsed();
|
||||
if (packed) {
|
||||
size += nodeRefToAddress.ramBytesUsed();
|
||||
} else if (nodeAddress != null) {
|
||||
size += nodeAddress.ramBytesUsed();
|
||||
size += inCounts.ramBytesUsed();
|
||||
}
|
||||
size += ramBytesUsed(cachedRootArcs);
|
||||
size += ramBytesUsed(assertingCachedRootArcs);
|
||||
size += RamUsageEstimator.sizeOf(bytesPerArc);
|
||||
return size;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Represents the outputs for an FST, providing the basic
|
||||
|
@ -95,4 +97,10 @@ public abstract class Outputs<T> {
|
|||
public T merge(T first, T second) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/** Return memory usage for the provided output.
|
||||
* @see Accountable */
|
||||
public long ramBytesUsed(T output) {
|
||||
return RamUsageEstimator.shallowSizeOf(output);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -175,4 +175,16 @@ public class PairOutputs<A,B> extends Outputs<PairOutputs.Pair<A,B>> {
|
|||
public String toString() {
|
||||
return "PairOutputs<" + outputs1 + "," + outputs2 + ">";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed(Pair<A,B> output) {
|
||||
long ramBytesUsed = super.ramBytesUsed(output);
|
||||
if (output.output1 != null) {
|
||||
ramBytesUsed += outputs1.ramBytesUsed(output.output1);
|
||||
}
|
||||
if (output.output2 != null) {
|
||||
ramBytesUsed += outputs2.ramBytesUsed(output.output2);
|
||||
}
|
||||
return ramBytesUsed;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.HashMap;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
|
||||
|
@ -37,18 +38,28 @@ import org.apache.lucene.util.TestUtil;
|
|||
|
||||
public class TestOrdinalMap extends LuceneTestCase {
|
||||
|
||||
private static final RamUsageTester.Filter ORDINAL_MAP_FILTER = new RamUsageTester.Filter() {
|
||||
@Override
|
||||
public boolean accept(Field field) {
|
||||
if (field.getDeclaringClass().equals(OrdinalMap.class) && field.getName().equals("owner")) {
|
||||
return false;
|
||||
private static final Field ORDINAL_MAP_OWNER_FIELD;
|
||||
static {
|
||||
try {
|
||||
ORDINAL_MAP_OWNER_FIELD = OrdinalMap.class.getDeclaredField("owner");
|
||||
} catch (Exception e) {
|
||||
throw new Error();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean accept(Object o) {
|
||||
return o != LongValues.IDENTITY;
|
||||
private static final RamUsageTester.Accumulator ORDINAL_MAP_ACCUMULATOR = new RamUsageTester.Accumulator() {
|
||||
|
||||
public long accumulateObject(Object o, long shallowSize, java.util.Map<Field,Object> fieldValues, java.util.Collection<Object> queue) {
|
||||
if (o == LongValues.IDENTITY) {
|
||||
return 0L;
|
||||
}
|
||||
if (o instanceof OrdinalMap) {
|
||||
fieldValues = new HashMap<>(fieldValues);
|
||||
fieldValues.remove(ORDINAL_MAP_OWNER_FIELD);
|
||||
}
|
||||
return super.accumulateObject(o, shallowSize, fieldValues, queue);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
public void testRamBytesUsed() throws IOException {
|
||||
|
@ -77,12 +88,12 @@ public class TestOrdinalMap extends LuceneTestCase {
|
|||
SortedDocValues sdv = ar.getSortedDocValues("sdv");
|
||||
if (sdv instanceof MultiSortedDocValues) {
|
||||
OrdinalMap map = ((MultiSortedDocValues) sdv).mapping;
|
||||
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_FILTER), map.ramBytesUsed());
|
||||
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_ACCUMULATOR), map.ramBytesUsed());
|
||||
}
|
||||
SortedSetDocValues ssdv = ar.getSortedSetDocValues("ssdv");
|
||||
if (ssdv instanceof MultiSortedSetDocValues) {
|
||||
OrdinalMap map = ((MultiSortedSetDocValues) ssdv).mapping;
|
||||
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_FILTER), map.ramBytesUsed());
|
||||
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_ACCUMULATOR), map.ramBytesUsed());
|
||||
}
|
||||
iw.close();
|
||||
r.close();
|
||||
|
|
|
@ -27,6 +27,7 @@ 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.util.PagedBytes.Reader;
|
||||
import org.junit.Ignore;
|
||||
|
||||
public class TestPagedBytes extends LuceneTestCase {
|
||||
|
@ -185,4 +186,17 @@ public class TestPagedBytes extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testRamBytesUsed() {
|
||||
final int blockBits = TestUtil.nextInt(random(), 4, 22);
|
||||
PagedBytes b = new PagedBytes(blockBits);
|
||||
final int totalBytes = random().nextInt(10000);
|
||||
for (long pointer = 0; pointer < totalBytes; ) {
|
||||
BytesRef bytes = new BytesRef(TestUtil.randomSimpleString(random(), 10));
|
||||
pointer = b.copyUsingLengthPrefix(bytes);
|
||||
}
|
||||
assertEquals(RamUsageTester.sizeOf(b), b.ramBytesUsed());
|
||||
final PagedBytes.Reader reader = b.freeze(random().nextBoolean());
|
||||
assertEquals(RamUsageTester.sizeOf(reader), reader.ramBytesUsed());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -30,10 +30,12 @@ import org.apache.lucene.index.Fields;
|
|||
import org.apache.lucene.index.FilterAtomicReader;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.sorter.Sorter.DocMap;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -222,6 +224,32 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
}
|
||||
}
|
||||
|
||||
private static class SortingSortedNumericDocValues extends SortedNumericDocValues {
|
||||
|
||||
private final SortedNumericDocValues in;
|
||||
private final Sorter.DocMap docMap;
|
||||
|
||||
SortingSortedNumericDocValues(SortedNumericDocValues in, DocMap docMap) {
|
||||
this.in = in;
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int count() {
|
||||
return in.count();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocument(int doc) {
|
||||
in.setDocument(docMap.newToOld(doc));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long valueAt(int index) {
|
||||
return in.valueAt(index);
|
||||
}
|
||||
}
|
||||
|
||||
private static class SortingBits implements Bits {
|
||||
|
||||
private final Bits in;
|
||||
|
@ -787,6 +815,17 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
return new SortingNumericDocValues(oldDocValues, docMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDocValues getSortedNumericDocValues(String field)
|
||||
throws IOException {
|
||||
final SortedNumericDocValues oldDocValues = in.getSortedNumericDocValues(field);
|
||||
if (oldDocValues == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new SortingSortedNumericDocValues(oldDocValues, docMap);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSortedDocValues(String field) throws IOException {
|
||||
SortedDocValues sortedDV = in.getSortedDocValues(field);
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.document.Field.Store;
|
|||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedNumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
|
@ -52,6 +53,7 @@ import org.apache.lucene.index.NumericDocValues;
|
|||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -145,6 +147,7 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
protected static final String DOC_POSITIONS_FIELD = "positions";
|
||||
protected static final String DOC_POSITIONS_TERM = "$all$";
|
||||
protected static final String NUMERIC_DV_FIELD = "numeric";
|
||||
protected static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
|
||||
protected static final String NORMS_FIELD = "norm";
|
||||
protected static final String BINARY_DV_FIELD = "binary";
|
||||
protected static final String SORTED_DV_FIELD = "sorted";
|
||||
|
@ -183,6 +186,10 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id))));
|
||||
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id + 1))));
|
||||
}
|
||||
if (defaultCodecSupportsSortedNumeric()) {
|
||||
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id));
|
||||
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
|
||||
}
|
||||
doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
|
||||
return doc;
|
||||
}
|
||||
|
@ -388,6 +395,20 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortedNumericDocValuesField() throws Exception {
|
||||
assumeTrue("default codec does not support SORTED_NUMERIC", defaultCodecSupportsSortedNumeric());
|
||||
SortedNumericDocValues dv = reader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
|
||||
int maxDoc = reader.maxDoc();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
dv.setDocument(i);
|
||||
assertEquals(2, dv.count());
|
||||
int value = sortedValues[i].intValue();
|
||||
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value, dv.valueAt(0));
|
||||
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value + 1, dv.valueAt(1));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTermVectors() throws Exception {
|
||||
int maxDoc = reader.maxDoc();
|
||||
|
|
|
@ -54,7 +54,7 @@ import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
|
|||
* <li>Only indexes Rectangles; just one per field value.</li>
|
||||
* <li>Can query only by a Rectangle.</li>
|
||||
* <li>Supports all {@link SpatialOperation}s.</li>
|
||||
* <li>Uses the FieldCache for any sorting / relevancy.</li>
|
||||
* <li>Uses the DocValues API for any sorting / relevancy.</li>
|
||||
* </ul>
|
||||
*
|
||||
* <h4>Implementation:</h4>
|
||||
|
@ -116,7 +116,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
public Field[] createIndexableFields(Shape shape) {
|
||||
if (shape instanceof Rectangle)
|
||||
return createIndexableFields((Rectangle)shape);
|
||||
throw new UnsupportedOperationException("Can only index Rectangle, not " + shape);
|
||||
throw new UnsupportedOperationException("Can only index a Rectangle, not " + shape);
|
||||
}
|
||||
|
||||
public Field[] createIndexableFields(Rectangle bbox) {
|
||||
|
@ -174,7 +174,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
throw new UnsupportedOperationException("Can only query by Rectangle, not " + shape);
|
||||
|
||||
Rectangle bbox = (Rectangle) shape;
|
||||
Query spatial = null;
|
||||
Query spatial;
|
||||
|
||||
// Useful for understanding Relations:
|
||||
// http://edndoc.esri.com/arcsde/9.1/general_topics/understand_spatial_relations.htm
|
||||
|
@ -212,10 +212,10 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinY <= queryExtent.getMinY() AND docMaxY >= queryExtent.getMaxY()
|
||||
Query qMinY = NumericRangeQuery.newDoubleRange(field_minY, precisionStep, null, bbox.getMinY(), false, true);
|
||||
Query qMaxY = NumericRangeQuery.newDoubleRange(field_maxY, precisionStep, bbox.getMaxY(), null, true, false);
|
||||
Query yConditions = this.makeQuery(new Query[]{qMinY, qMaxY}, BooleanClause.Occur.MUST);
|
||||
Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
|
||||
|
||||
// X conditions
|
||||
Query xConditions = null;
|
||||
Query xConditions;
|
||||
|
||||
// queries that do not cross the date line
|
||||
if (!bbox.getCrossesDateLine()) {
|
||||
|
@ -225,7 +225,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinX <= queryExtent.getMinX() AND docMaxX >= queryExtent.getMaxX()
|
||||
Query qMinX = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, null, bbox.getMinX(), false, true);
|
||||
Query qMaxX = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, bbox.getMaxX(), null, true, false);
|
||||
Query qMinMax = this.makeQuery(new Query[]{qMinX, qMaxX}, BooleanClause.Occur.MUST);
|
||||
Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
|
||||
Query qNonXDL = this.makeXDL(false, qMinMax);
|
||||
|
||||
// X Conditions for documents that cross the date line,
|
||||
|
@ -234,11 +234,11 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinXLeft <= queryExtent.getMinX() OR docMaxXRight >= queryExtent.getMaxX()
|
||||
Query qXDLLeft = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, null, bbox.getMinX(), false, true);
|
||||
Query qXDLRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, bbox.getMaxX(), null, true, false);
|
||||
Query qXDLLeftRight = this.makeQuery(new Query[]{qXDLLeft, qXDLRight}, BooleanClause.Occur.SHOULD);
|
||||
Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qXDLLeft, qXDLRight);
|
||||
Query qXDL = this.makeXDL(true, qXDLLeftRight);
|
||||
|
||||
// apply the non-XDL and XDL conditions
|
||||
xConditions = this.makeQuery(new Query[]{qNonXDL, qXDL}, BooleanClause.Occur.SHOULD);
|
||||
xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
|
||||
|
||||
// queries that cross the date line
|
||||
} else {
|
||||
|
@ -251,13 +251,13 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinXLeft <= queryExtent.getMinX() AND docMaxXRight >= queryExtent.getMaxX()
|
||||
Query qXDLLeft = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, null, bbox.getMinX(), false, true);
|
||||
Query qXDLRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, bbox.getMaxX(), null, true, false);
|
||||
Query qXDLLeftRight = this.makeQuery(new Query[]{qXDLLeft, qXDLRight}, BooleanClause.Occur.MUST);
|
||||
Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight);
|
||||
|
||||
xConditions = this.makeXDL(true, qXDLLeftRight);
|
||||
}
|
||||
|
||||
// both X and Y conditions must occur
|
||||
return this.makeQuery(new Query[]{xConditions, yConditions}, BooleanClause.Occur.MUST);
|
||||
return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -274,10 +274,10 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinY > queryExtent.getMaxY() OR docMaxY < queryExtent.getMinY()
|
||||
Query qMinY = NumericRangeQuery.newDoubleRange(field_minY, precisionStep, bbox.getMaxY(), null, false, false);
|
||||
Query qMaxY = NumericRangeQuery.newDoubleRange(field_maxY, precisionStep, null, bbox.getMinY(), false, false);
|
||||
Query yConditions = this.makeQuery(new Query[]{qMinY, qMaxY}, BooleanClause.Occur.SHOULD);
|
||||
Query yConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qMinY, qMaxY);
|
||||
|
||||
// X conditions
|
||||
Query xConditions = null;
|
||||
Query xConditions;
|
||||
|
||||
// queries that do not cross the date line
|
||||
if (!bbox.getCrossesDateLine()) {
|
||||
|
@ -286,7 +286,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX()
|
||||
Query qMinX = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMaxX(), null, false, false);
|
||||
Query qMaxX = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMinX(), false, false);
|
||||
Query qMinMax = this.makeQuery(new Query[]{qMinX, qMaxX}, BooleanClause.Occur.SHOULD);
|
||||
Query qMinMax = this.makeQuery(BooleanClause.Occur.SHOULD, qMinX, qMaxX);
|
||||
Query qNonXDL = this.makeXDL(false, qMinMax);
|
||||
|
||||
// X Conditions for documents that cross the date line,
|
||||
|
@ -298,11 +298,11 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// (docMinXRight > queryExtent.getMaxX()) equates to (-180.0 > queryExtent.getMaxX()) and is ignored
|
||||
Query qMinXLeft = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMaxX(), null, false, false);
|
||||
Query qMaxXRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMinX(), false, false);
|
||||
Query qLeftRight = this.makeQuery(new Query[]{qMinXLeft, qMaxXRight}, BooleanClause.Occur.MUST);
|
||||
Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXRight);
|
||||
Query qXDL = this.makeXDL(true, qLeftRight);
|
||||
|
||||
// apply the non-XDL and XDL conditions
|
||||
xConditions = this.makeQuery(new Query[]{qNonXDL, qXDL}, BooleanClause.Occur.SHOULD);
|
||||
xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
|
||||
|
||||
// queries that cross the date line
|
||||
} else {
|
||||
|
@ -315,9 +315,9 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
Query qMaxXLeft = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMinX(), false, false);
|
||||
Query qMinXRight = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMaxX(), null, false, false);
|
||||
Query qMaxXRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, -180.0, false, false);
|
||||
Query qLeft = this.makeQuery(new Query[]{qMinXLeft, qMaxXLeft}, BooleanClause.Occur.SHOULD);
|
||||
Query qRight = this.makeQuery(new Query[]{qMinXRight, qMaxXRight}, BooleanClause.Occur.SHOULD);
|
||||
Query qLeftRight = this.makeQuery(new Query[]{qLeft, qRight}, BooleanClause.Occur.MUST);
|
||||
Query qLeft = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXLeft, qMaxXLeft);
|
||||
Query qRight = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXRight, qMaxXRight);
|
||||
Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qLeft, qRight);
|
||||
|
||||
// No need to search for documents that do not cross the date line
|
||||
|
||||
|
@ -325,7 +325,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
}
|
||||
|
||||
// either X or Y conditions should occur
|
||||
return this.makeQuery(new Query[]{xConditions, yConditions}, BooleanClause.Occur.SHOULD);
|
||||
return this.makeQuery(BooleanClause.Occur.SHOULD, xConditions, yConditions);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -359,13 +359,13 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// switch to a NOT Disjoint query
|
||||
|
||||
// MUST_NOT causes a problem when it's the only clause type within a BooleanQuery,
|
||||
// to get round it we add all documents as a SHOULD
|
||||
// to get around it we add all documents as a SHOULD
|
||||
|
||||
// there must be an envelope, it must not be disjoint
|
||||
Query qDisjoint = makeDisjoint(bbox);
|
||||
Query qIsNonXDL = this.makeXDL(false);
|
||||
Query qIsXDL = this.makeXDL(true);
|
||||
Query qHasEnv = this.makeQuery(new Query[]{qIsNonXDL, qIsXDL}, BooleanClause.Occur.SHOULD);
|
||||
Query qHasEnv = this.makeQuery(BooleanClause.Occur.SHOULD, qIsNonXDL, qIsXDL);
|
||||
BooleanQuery qNotDisjoint = new BooleanQuery();
|
||||
qNotDisjoint.add(qHasEnv, BooleanClause.Occur.MUST);
|
||||
qNotDisjoint.add(qDisjoint, BooleanClause.Occur.MUST_NOT);
|
||||
|
@ -380,11 +380,11 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
/**
|
||||
* Makes a boolean query based upon a collection of queries and a logical operator.
|
||||
*
|
||||
* @param queries the query collection
|
||||
* @param occur the logical operator
|
||||
* @param queries the query collection
|
||||
* @return the query
|
||||
*/
|
||||
BooleanQuery makeQuery(Query[] queries, BooleanClause.Occur occur) {
|
||||
BooleanQuery makeQuery(BooleanClause.Occur occur, Query... queries) {
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
for (Query query : queries) {
|
||||
bq.add(query, occur);
|
||||
|
@ -406,10 +406,10 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinY >= queryExtent.getMinY() AND docMaxY <= queryExtent.getMaxY()
|
||||
Query qMinY = NumericRangeQuery.newDoubleRange(field_minY, precisionStep, bbox.getMinY(), null, true, false);
|
||||
Query qMaxY = NumericRangeQuery.newDoubleRange(field_maxY, precisionStep, null, bbox.getMaxY(), false, true);
|
||||
Query yConditions = this.makeQuery(new Query[]{qMinY, qMaxY}, BooleanClause.Occur.MUST);
|
||||
Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
|
||||
|
||||
// X conditions
|
||||
Query xConditions = null;
|
||||
Query xConditions;
|
||||
|
||||
// X Conditions for documents that cross the date line,
|
||||
// the left portion of the document must be within the left portion of the query,
|
||||
|
@ -418,7 +418,7 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// AND docMinXRight >= -180.0 AND docMaxXRight <= queryExtent.getMaxX()
|
||||
Query qXDLLeft = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMinX(), null, true, false);
|
||||
Query qXDLRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMaxX(), false, true);
|
||||
Query qXDLLeftRight = this.makeQuery(new Query[]{qXDLLeft, qXDLRight}, BooleanClause.Occur.MUST);
|
||||
Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight);
|
||||
Query qXDL = this.makeXDL(true, qXDLLeftRight);
|
||||
|
||||
// queries that do not cross the date line
|
||||
|
@ -428,12 +428,12 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinX >= queryExtent.getMinX() AND docMaxX <= queryExtent.getMaxX()
|
||||
Query qMinX = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMinX(), null, true, false);
|
||||
Query qMaxX = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMaxX(), false, true);
|
||||
Query qMinMax = this.makeQuery(new Query[]{qMinX, qMaxX}, BooleanClause.Occur.MUST);
|
||||
Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
|
||||
Query qNonXDL = this.makeXDL(false, qMinMax);
|
||||
|
||||
// apply the non-XDL or XDL X conditions
|
||||
if ((bbox.getMinX() <= -180.0) && bbox.getMaxX() >= 180.0) {
|
||||
xConditions = this.makeQuery(new Query[]{qNonXDL, qXDL}, BooleanClause.Occur.SHOULD);
|
||||
xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
|
||||
} else {
|
||||
xConditions = qNonXDL;
|
||||
}
|
||||
|
@ -447,25 +447,25 @@ public class BBoxStrategy extends SpatialStrategy {
|
|||
// docMinX >= queryExtent.getMinX() AND docMaxX <= 180.0
|
||||
Query qMinXLeft = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, bbox.getMinX(), null, true, false);
|
||||
Query qMaxXLeft = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, 180.0, false, true);
|
||||
Query qLeft = this.makeQuery(new Query[]{qMinXLeft, qMaxXLeft}, BooleanClause.Occur.MUST);
|
||||
Query qLeft = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXLeft);
|
||||
|
||||
// the document should be within the right portion of the query
|
||||
// docMinX >= -180.0 AND docMaxX <= queryExtent.getMaxX()
|
||||
Query qMinXRight = NumericRangeQuery.newDoubleRange(field_minX, precisionStep, -180.0, null, true, false);
|
||||
Query qMaxXRight = NumericRangeQuery.newDoubleRange(field_maxX, precisionStep, null, bbox.getMaxX(), false, true);
|
||||
Query qRight = this.makeQuery(new Query[]{qMinXRight, qMaxXRight}, BooleanClause.Occur.MUST);
|
||||
Query qRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXRight, qMaxXRight);
|
||||
|
||||
// either left or right conditions should occur,
|
||||
// apply the left and right conditions to documents that do not cross the date line
|
||||
Query qLeftRight = this.makeQuery(new Query[]{qLeft, qRight}, BooleanClause.Occur.SHOULD);
|
||||
Query qLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qLeft, qRight);
|
||||
Query qNonXDL = this.makeXDL(false, qLeftRight);
|
||||
|
||||
// apply the non-XDL and XDL conditions
|
||||
xConditions = this.makeQuery(new Query[]{qNonXDL, qXDL}, BooleanClause.Occur.SHOULD);
|
||||
xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
|
||||
}
|
||||
|
||||
// both X and Y conditions must occur
|
||||
return this.makeQuery(new Query[]{xConditions, yConditions}, BooleanClause.Occur.MUST);
|
||||
return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,24 +18,106 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.RamUsageTester;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Common tests to all index formats.
|
||||
*/
|
||||
abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
||||
|
||||
// metadata or Directory-level objects
|
||||
private static final Set<Class<?>> EXCLUDED_CLASSES = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
|
||||
|
||||
// Notorious singletons
|
||||
private static final Set<Object> EXCLUDED_OBJECTS = Collections.newSetFromMap(new IdentityHashMap<Object,Boolean>());
|
||||
|
||||
static {
|
||||
// Directory objects, don't take into account eg. the NIO buffers
|
||||
EXCLUDED_CLASSES.add(Directory.class);
|
||||
EXCLUDED_CLASSES.add(IndexInput.class);
|
||||
|
||||
// used for thread management, not by the index
|
||||
EXCLUDED_CLASSES.add(CloseableThreadLocal.class);
|
||||
|
||||
// don't follow references to the top-level reader
|
||||
EXCLUDED_CLASSES.add(IndexReader.class);
|
||||
EXCLUDED_CLASSES.add(IndexReaderContext.class);
|
||||
|
||||
// usually small but can bump memory usage for
|
||||
// memory-efficient things like stored fields
|
||||
EXCLUDED_CLASSES.add(FieldInfos.class);
|
||||
EXCLUDED_CLASSES.add(SegmentInfo.class);
|
||||
EXCLUDED_CLASSES.add(SegmentCommitInfo.class);
|
||||
EXCLUDED_CLASSES.add(FieldInfo.class);
|
||||
|
||||
// singletons
|
||||
EXCLUDED_CLASSES.add(Codec.class);
|
||||
EXCLUDED_CLASSES.add(Enum.class);
|
||||
for (PackedInts.Format format : PackedInts.Format.values()) {
|
||||
for (int i = 1; i <= 64; ++i) {
|
||||
if (format.isSupported(i)) {
|
||||
EXCLUDED_OBJECTS.add(PackedInts.getDecoder(format, PackedInts.VERSION_CURRENT, i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class Accumulator extends RamUsageTester.Accumulator {
|
||||
|
||||
private final Object root;
|
||||
|
||||
Accumulator(Object root) {
|
||||
this.root = root;
|
||||
}
|
||||
|
||||
public long accumulateObject(Object o, long shallowSize, java.util.Map<Field, Object> fieldValues, java.util.Collection<Object> queue) {
|
||||
if (EXCLUDED_OBJECTS.contains(o)) {
|
||||
return 0L;
|
||||
}
|
||||
for (Class<?> clazz = o.getClass(); clazz != null; clazz = clazz.getSuperclass()) {
|
||||
if (EXCLUDED_CLASSES.contains(clazz) && o != root) {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
// we have no way to estimate the size of these things in codecs although
|
||||
// something like a Collections.newSetFromMap(new HashMap<>()) uses quite
|
||||
// some memory... So for now the test ignores the overhead of such
|
||||
// collections but can we do better?
|
||||
if (o instanceof Collection) {
|
||||
Collection<?> coll = (Collection<?>) o;
|
||||
queue.addAll((Collection<?>) o);
|
||||
return (long) coll.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
} else if (o instanceof Map) {
|
||||
final Map<?, ?> map = (Map<?,?>) o;
|
||||
queue.addAll(map.keySet());
|
||||
queue.addAll(map.values());
|
||||
return 2L * map.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
}
|
||||
return super.accumulateObject(o, shallowSize, fieldValues, queue);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
/** Returns the codec to run tests against */
|
||||
protected abstract Codec getCodec();
|
||||
|
||||
|
@ -118,4 +200,38 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
|||
dir2.close();
|
||||
}
|
||||
|
||||
/** Test the accuracy of the ramBytesUsed estimations. */
|
||||
public void testRamBytesUsed() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, cfg);
|
||||
// we need to index enough documents so that constant overhead doesn't dominate
|
||||
final int numDocs = atLeast(10000);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document d = new Document();
|
||||
addRandomFields(d);
|
||||
w.addDocument(d);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
w.commit();
|
||||
w.close();
|
||||
|
||||
IndexReader reader = DirectoryReader.open(dir);
|
||||
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
final AtomicReader r = context.reader();
|
||||
// beware of lazy-loaded stuff
|
||||
new SimpleMergedSegmentWarmer(InfoStream.NO_OUTPUT).warm(r);
|
||||
final long actualBytes = RamUsageTester.sizeOf(r, new Accumulator(r));
|
||||
final long expectedBytes = ((SegmentReader) r).ramBytesUsed();
|
||||
final long absoluteError = actualBytes - expectedBytes;
|
||||
final double relativeError = (double) absoluteError / actualBytes;
|
||||
final String message = "Actual RAM usage " + actualBytes + ", but got " + expectedBytes + ", " + relativeError + "% error";
|
||||
assertTrue(message, Math.abs(relativeError) < 0.20d || Math.abs(absoluteError) < 1000);
|
||||
}
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -238,7 +238,8 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
|
|||
@Override
|
||||
protected void addRandomFields(Document doc) {
|
||||
// TODO: improve
|
||||
doc.add(new TextField("foobar", "boo", Field.Store.NO));
|
||||
doc.add(new TextField("foobar", TestUtil.randomSimpleString(random()), Field.Store.NO));
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,40 +20,40 @@ package org.apache.lucene.util;
|
|||
import java.lang.reflect.Array;
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.AbstractList;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/** Crawls object graph to collect RAM usage for testing */
|
||||
public final class RamUsageTester {
|
||||
|
||||
/**
|
||||
* A {@link Filter} that accepts all fields.
|
||||
*/
|
||||
private static final Filter DEFAULT_FILTER = new Filter() {
|
||||
/** An accumulator of object references. This class allows for customizing RAM usage estimation. */
|
||||
public static class Accumulator {
|
||||
|
||||
@Override
|
||||
public boolean accept(Field field) {
|
||||
return true;
|
||||
/** Accumulate transitive references for the provided fields of the given
|
||||
* object into <code>queue</code> and return the shallow size of this object. */
|
||||
public long accumulateObject(Object o, long shallowSize, Map<Field, Object> fieldValues, Collection<Object> queue) {
|
||||
for (Object value : fieldValues.values()) {
|
||||
queue.add(value);
|
||||
}
|
||||
return shallowSize;
|
||||
}
|
||||
|
||||
public boolean accept(Object o) {
|
||||
return true;
|
||||
/** Accumulate transitive references for the provided values of the given
|
||||
* array into <code>queue</code> and return the shallow size of this array. */
|
||||
public long accumulateArray(Object array, long shallowSize, List<Object> values, Collection<Object> queue) {
|
||||
queue.addAll(values);
|
||||
return shallowSize;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
/** A filter that allows to decide on what to take into account when measuring RAM usage. */
|
||||
public static interface Filter {
|
||||
|
||||
/** Whether the provided field should be taken into account when measuring RAM usage. */
|
||||
boolean accept(Field field);
|
||||
|
||||
/** Whether the provided field value should be taken into account when measuring RAM usage. */
|
||||
boolean accept(Object o);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -65,13 +65,13 @@ public final class RamUsageTester {
|
|||
* (it isn't side-effect free). After the method exits, this memory
|
||||
* should be GCed.</p>
|
||||
*/
|
||||
public static long sizeOf(Object obj, Filter filter) {
|
||||
return measureObjectSize(obj, filter);
|
||||
public static long sizeOf(Object obj, Accumulator accumulator) {
|
||||
return measureObjectSize(obj, accumulator);
|
||||
}
|
||||
|
||||
/** Same as calling <code>sizeOf(obj, DEFAULT_FILTER)</code>. */
|
||||
public static long sizeOf(Object obj) {
|
||||
return sizeOf(obj, DEFAULT_FILTER);
|
||||
return sizeOf(obj, new Accumulator());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -89,7 +89,7 @@ public final class RamUsageTester {
|
|||
* or complex graphs (a max. recursion depth on my machine was ~5000 objects linked in a chain
|
||||
* so not too much).
|
||||
*/
|
||||
private static long measureObjectSize(Object root, Filter filter) {
|
||||
private static long measureObjectSize(Object root, Accumulator accumulator) {
|
||||
// Objects seen so far.
|
||||
final IdentityHashSet<Object> seen = new IdentityHashSet<>();
|
||||
// Class cache with reference Field and precalculated shallow size.
|
||||
|
@ -114,25 +114,28 @@ public final class RamUsageTester {
|
|||
* Consider an array, possibly of primitive types. Push any of its references to
|
||||
* the processing stack and accumulate this array's shallow size.
|
||||
*/
|
||||
long size = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
|
||||
final long shallowSize = RamUsageEstimator.shallowSizeOf(ob);
|
||||
final int len = Array.getLength(ob);
|
||||
if (len > 0) {
|
||||
final List<Object> values;
|
||||
Class<?> componentClazz = obClazz.getComponentType();
|
||||
if (componentClazz.isPrimitive()) {
|
||||
size += (long) len * RamUsageEstimator.shallowSizeOfInstance(componentClazz);
|
||||
values = Collections.emptyList();
|
||||
} else {
|
||||
size += (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * len;
|
||||
values = new AbstractList<Object>() {
|
||||
|
||||
// Push refs for traversal later.
|
||||
for (int i = len; --i >= 0 ;) {
|
||||
final Object o = Array.get(ob, i);
|
||||
if (o != null && !seen.contains(o) && filter.accept(o)) {
|
||||
stack.add(o);
|
||||
@Override
|
||||
public Object get(int index) {
|
||||
return Array.get(ob, index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return len;
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}
|
||||
totalSize += RamUsageEstimator.alignObjectSize(size);
|
||||
totalSize += accumulator.accumulateArray(ob, shallowSize, values, stack);
|
||||
} else {
|
||||
/*
|
||||
* Consider an object. Push any references it has to the processing stack
|
||||
|
@ -144,17 +147,12 @@ public final class RamUsageTester {
|
|||
classCache.put(obClazz, cachedInfo = createCacheEntry(obClazz));
|
||||
}
|
||||
|
||||
Map<Field, Object> fieldValues = new HashMap<>();
|
||||
for (Field f : cachedInfo.referenceFields) {
|
||||
if (filter.accept(f)) {
|
||||
// Fast path to eliminate redundancies.
|
||||
final Object o = f.get(ob);
|
||||
if (o != null && !seen.contains(o) && filter.accept(o)) {
|
||||
stack.add(o);
|
||||
}
|
||||
}
|
||||
fieldValues.put(f, f.get(ob));
|
||||
}
|
||||
|
||||
totalSize += cachedInfo.alignedShallowInstanceSize;
|
||||
totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
|
||||
} catch (IllegalAccessException e) {
|
||||
// this should never happen as we enabled setAccessible().
|
||||
throw new RuntimeException("Reflective field access failed?", e);
|
||||
|
|
|
@ -86,6 +86,12 @@ Other Changes
|
|||
|
||||
* SOLR-6169: Properly remove CoreAdminHandler handleAlias action (Alan Woodward)
|
||||
|
||||
================== 4.10.0 =================
|
||||
|
||||
Bug fixes
|
||||
|
||||
* SOLR-6095 : SolrCloud cluster can end up without an overseer with overseer roles (Noble Paul, Shalin Mangar)
|
||||
|
||||
================== 4.9.0 ==================
|
||||
|
||||
Versions of Major Components
|
||||
|
@ -99,7 +105,11 @@ Apache ZooKeeper 3.4.6
|
|||
Upgrading from Solr 4.8
|
||||
----------------------
|
||||
|
||||
* placeholder
|
||||
* Support for DiskDocValuesFormat (ie: fieldTypes configured with docValuesFormat="Disk")
|
||||
has been removed due to poor performance. If you have an existing fieldTypes using
|
||||
DiskDocValuesFormat please modify your schema.xml to remove the 'docValuesFormat'
|
||||
attribute, and optimize your index to rewrite it into the default codec, prior to
|
||||
upgrading to 4.9. See LUCENE-5761 for more details.
|
||||
|
||||
Detailed Change List
|
||||
----------------------
|
||||
|
@ -187,6 +197,15 @@ Bug Fixes
|
|||
* SOLR-6175: DebugComponent throws NPE on shard exceptions when using shards.tolerant.
|
||||
(Tomás Fernández Löbbe via shalin)
|
||||
|
||||
* SOLR-6129: DateFormatTransformer doesn't resolve dateTimeFormat. (Aaron LaBella via shalin)
|
||||
|
||||
* SOLR-6164: Copy Fields Schema additions are not distributed to other nodes.
|
||||
(Gregory Chanan via Steve Rowe)
|
||||
|
||||
* SOLR-6160: An error was sometimes possible if a distributed search included grouping
|
||||
with group.facet, faceting on facet.field and either facet.range or facet.query.
|
||||
(David Smiley)
|
||||
|
||||
Other Changes
|
||||
---------------------
|
||||
|
||||
|
|
|
@ -58,6 +58,8 @@ public class DateFormatTransformer extends Transformer {
|
|||
String fmt = map.get(DATE_TIME_FMT);
|
||||
if (fmt == null)
|
||||
continue;
|
||||
VariableResolver resolver = context.getVariableResolver();
|
||||
fmt = resolver.replaceTokens(fmt);
|
||||
String column = map.get(DataImporter.COLUMN);
|
||||
String srcCol = map.get(RegexTransformer.SRC_COL_NAME);
|
||||
if (srcCol == null)
|
||||
|
|
|
@ -38,7 +38,7 @@ public class TestDateFormatTransformer extends AbstractDataImportHandlerTestCase
|
|||
fields.add(createMap(DataImporter.COLUMN, "lastModified"));
|
||||
fields.add(createMap(DataImporter.COLUMN,
|
||||
"dateAdded", RegexTransformer.SRC_COL_NAME, "lastModified",
|
||||
DateFormatTransformer.DATE_TIME_FMT, "MM/dd/yyyy"));
|
||||
DateFormatTransformer.DATE_TIME_FMT, "${xyz.myDateFormat}"));
|
||||
|
||||
SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy", Locale.ROOT);
|
||||
Date now = format.parse(format.format(new Date()));
|
||||
|
@ -47,6 +47,7 @@ public class TestDateFormatTransformer extends AbstractDataImportHandlerTestCase
|
|||
|
||||
VariableResolver resolver = new VariableResolver();
|
||||
resolver.addNamespace("e", row);
|
||||
resolver.addNamespace("xyz", createMap("myDateFormat", "MM/dd/yyyy"));
|
||||
|
||||
Context context = getContext(null, resolver,
|
||||
null, Context.FULL_DUMP, fields, null);
|
||||
|
|
|
@ -71,7 +71,7 @@ public abstract class ElectionContext {
|
|||
zkClient.delete(leaderSeqPath, -1, true);
|
||||
} catch (NoNodeException e) {
|
||||
// fine
|
||||
log.warn("cancelElection did not find election node to remove",e);
|
||||
log.warn("cancelElection did not find election node to remove {}" ,leaderSeqPath);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -80,6 +80,10 @@ public abstract class ElectionContext {
|
|||
public void checkIfIamLeaderFired() {}
|
||||
|
||||
public void joinedElectionFired() {}
|
||||
|
||||
public ElectionContext copy(){
|
||||
throw new UnsupportedOperationException("copy");
|
||||
}
|
||||
}
|
||||
|
||||
class ShardLeaderElectionContextBase extends ElectionContext {
|
||||
|
@ -530,6 +534,11 @@ final class OverseerElectionContext extends ElectionContext {
|
|||
overseer.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ElectionContext copy() {
|
||||
return new OverseerElectionContext(zkClient, overseer ,id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void joinedElectionFired() {
|
||||
overseer.close();
|
||||
|
|
|
@ -57,7 +57,7 @@ public class LeaderElector {
|
|||
|
||||
static final String ELECTION_NODE = "/election";
|
||||
|
||||
private final static Pattern LEADER_SEQ = Pattern.compile(".*?/?.*?-n_(\\d+)");
|
||||
public final static Pattern LEADER_SEQ = Pattern.compile(".*?/?.*?-n_(\\d+)");
|
||||
private final static Pattern SESSION_ID = Pattern.compile(".*?/?(.*?-.*?)-n_\\d+");
|
||||
private final static Pattern NODE_NAME = Pattern.compile(".*?/?(.*?-)(.*?)-n_\\d+");
|
||||
|
||||
|
@ -100,14 +100,34 @@ public class LeaderElector {
|
|||
return;
|
||||
}
|
||||
if (seq <= intSeqs.get(0)) {
|
||||
if(seq == intSeqs.get(0) && !context.leaderSeqPath.equals(holdElectionPath+"/"+seqs.get(0)) ) {//somebody else already became the leader with the same sequence id , not me
|
||||
log.info("was going be leader {} , seq(0) {}",context.leaderSeqPath,holdElectionPath+"/"+seqs.get(0));//but someone else jumped the line
|
||||
retryElection(context,false);//join at the tail again
|
||||
return;
|
||||
}
|
||||
// first we delete the node advertising the old leader in case the ephem is still there
|
||||
// first we delete the node advertising the old leader in case the ephem is still there
|
||||
try {
|
||||
zkClient.delete(context.leaderPath, -1, true);
|
||||
} catch(Exception e) {
|
||||
}catch (KeeperException.NoNodeException nne){
|
||||
//no problem
|
||||
}catch (InterruptedException e){
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
//failed to delete the leader node
|
||||
log.error("leader elect delete error",e);
|
||||
retryElection(context, false);
|
||||
return;
|
||||
// fine
|
||||
}
|
||||
|
||||
try {
|
||||
runIamLeaderProcess(context, replacement);
|
||||
} catch (KeeperException.NodeExistsException e) {
|
||||
log.error("node exists",e);
|
||||
retryElection(context, false);
|
||||
return;
|
||||
}
|
||||
} else {
|
||||
// I am not the leader - watch the node below me
|
||||
int i = 1;
|
||||
|
@ -124,7 +144,8 @@ public class LeaderElector {
|
|||
return;
|
||||
}
|
||||
try {
|
||||
zkClient.getData(holdElectionPath + "/" + seqs.get(index), watcher = new ElectionWatcher(context.leaderSeqPath , seq, context) , null, true);
|
||||
String watchedNode = holdElectionPath + "/" + seqs.get(index);
|
||||
zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath , watchedNode,seq, context) , null, true);
|
||||
} catch (KeeperException.SessionExpiredException e) {
|
||||
throw e;
|
||||
} catch (KeeperException e) {
|
||||
|
@ -196,6 +217,9 @@ public class LeaderElector {
|
|||
}
|
||||
return intSeqs;
|
||||
}
|
||||
public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
|
||||
return joinElection(context,replacement, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Begin participating in the election process. Gets a new sequential number
|
||||
|
@ -206,7 +230,7 @@ public class LeaderElector {
|
|||
*
|
||||
* @return sequential node number
|
||||
*/
|
||||
public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
|
||||
public int joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
|
||||
context.joinedElectionFired();
|
||||
|
||||
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
|
||||
|
@ -218,8 +242,30 @@ public class LeaderElector {
|
|||
int tries = 0;
|
||||
while (cont) {
|
||||
try {
|
||||
if(joinAtHead){
|
||||
log.info("node {} Trying to join election at the head ", id);
|
||||
List<String> nodes = OverseerCollectionProcessor.getSortedElectionNodes(zkClient);
|
||||
if(nodes.size() <2){
|
||||
leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
|
||||
CreateMode.EPHEMERAL_SEQUENTIAL, false);
|
||||
} else {
|
||||
String firstInLine = nodes.get(1);
|
||||
log.info("The current head: {}", firstInLine);
|
||||
Matcher m = LEADER_SEQ.matcher(firstInLine);
|
||||
if (!m.matches()) {
|
||||
throw new IllegalStateException("Could not find regex match in:"
|
||||
+ firstInLine);
|
||||
}
|
||||
leaderSeqPath = shardsElectZkPath + "/" + id + "-n_"+ m.group(1);
|
||||
zkClient.create(leaderSeqPath, null, CreateMode.EPHEMERAL, false);
|
||||
log.info("Joined at the head {}", leaderSeqPath );
|
||||
|
||||
}
|
||||
} else {
|
||||
leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
|
||||
CreateMode.EPHEMERAL_SEQUENTIAL, false);
|
||||
}
|
||||
|
||||
context.leaderSeqPath = leaderSeqPath;
|
||||
cont = false;
|
||||
} catch (ConnectionLossException e) {
|
||||
|
@ -270,14 +316,15 @@ public class LeaderElector {
|
|||
}
|
||||
|
||||
private class ElectionWatcher implements Watcher {
|
||||
final String leaderSeqPath;
|
||||
final String myNode,watchedNode;
|
||||
final int seq;
|
||||
final ElectionContext context;
|
||||
|
||||
private boolean canceled = false;
|
||||
|
||||
private ElectionWatcher(String leaderSeqPath, int seq, ElectionContext context) {
|
||||
this.leaderSeqPath = leaderSeqPath;
|
||||
private ElectionWatcher(String myNode, String watchedNode, int seq, ElectionContext context) {
|
||||
this.myNode = myNode;
|
||||
this.watchedNode = watchedNode;
|
||||
this.seq = seq;
|
||||
this.context = context;
|
||||
}
|
||||
|
@ -295,7 +342,14 @@ public class LeaderElector {
|
|||
return;
|
||||
}
|
||||
if(canceled) {
|
||||
log.info("This watcher is not active anymore {}", leaderSeqPath);
|
||||
log.info("This watcher is not active anymore {}", myNode);
|
||||
try {
|
||||
zkClient.delete(myNode,-1,true);
|
||||
}catch (KeeperException.NoNodeException nne) {
|
||||
//expected . don't do anything
|
||||
} catch (Exception e) {
|
||||
log.warn("My watched node still exists and can't remove "+myNode, e);
|
||||
}
|
||||
return;
|
||||
}
|
||||
try {
|
||||
|
@ -332,16 +386,19 @@ public class LeaderElector {
|
|||
|
||||
@Override
|
||||
public int compare(String o1, String o2) {
|
||||
return Integer.valueOf(getSeq(o1)).compareTo(
|
||||
int i = Integer.valueOf(getSeq(o1)).compareTo(
|
||||
Integer.valueOf(getSeq(o2)));
|
||||
return i == 0 ? o1.compareTo(o2) : i ;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void retryElection() throws KeeperException, InterruptedException, IOException {
|
||||
context.cancelElection();
|
||||
void retryElection(ElectionContext context, boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
|
||||
ElectionWatcher watcher = this.watcher;
|
||||
if(watcher!= null) watcher.cancel(context.leaderSeqPath);
|
||||
joinElection(context, true);
|
||||
ElectionContext ctx = context.copy();
|
||||
if(watcher!= null) watcher.cancel(this.context.leaderSeqPath);
|
||||
this.context.cancelElection();
|
||||
this.context = ctx;
|
||||
joinElection(ctx, true, joinAtHead);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -308,6 +308,7 @@ public class Overseer {
|
|||
log.error("could not read the data" ,e);
|
||||
return;
|
||||
}
|
||||
try {
|
||||
Map m = (Map) ZkStateReader.fromJSON(data);
|
||||
String id = (String) m.get("id");
|
||||
if(overseerCollectionProcessor.getId().equals(id)){
|
||||
|
@ -318,20 +319,21 @@ public class Overseer {
|
|||
//no problem ignore it some other Overseer has already taken over
|
||||
} catch (Exception e) {
|
||||
log.error("Could not delete my leader node ", e);
|
||||
} finally {
|
||||
try {
|
||||
if(zkController !=null && !zkController.getCoreContainer().isShutDown()){
|
||||
zkController.rejoinOverseerElection();
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
log.error("error canceling overseer election election ",e);
|
||||
}
|
||||
}
|
||||
|
||||
} else{
|
||||
log.info("somebody else has already taken up the overseer position");
|
||||
}
|
||||
} finally {
|
||||
//if I am not shutting down, Then I need to rejoin election
|
||||
try {
|
||||
if (zkController != null && !zkController.getCoreContainer().isShutDown()) {
|
||||
zkController.rejoinOverseerElection(null, false);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn("Unable to rejoinElection ",e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ClusterState processMessage(ClusterState clusterState,
|
||||
|
@ -377,9 +379,13 @@ public class Overseer {
|
|||
} else if(CLUSTERPROP.isEqual(operation)){
|
||||
handleProp(message);
|
||||
} else if( QUIT.equals(operation)){
|
||||
if(myId.equals( message.get("id"))){
|
||||
log.info("Quit command received {}", LeaderElector.getNodeName(myId));
|
||||
overseerCollectionProcessor.close();
|
||||
close();
|
||||
} else {
|
||||
log.warn("Overseer received wrong QUIT message {}", message);
|
||||
}
|
||||
} else{
|
||||
throw new RuntimeException("unknown operation:" + operation
|
||||
+ " contents:" + message.getProperties());
|
||||
|
|
|
@ -407,95 +407,42 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
}
|
||||
}
|
||||
|
||||
private void prioritizeOverseerNodes() throws KeeperException, InterruptedException {
|
||||
private synchronized void prioritizeOverseerNodes() throws KeeperException, InterruptedException {
|
||||
SolrZkClient zk = zkStateReader.getZkClient();
|
||||
if(!zk.exists(ZkStateReader.ROLES,true))return;
|
||||
Map m = (Map) ZkStateReader.fromJSON(zk.getData(ZkStateReader.ROLES, null, new Stat(), true));
|
||||
|
||||
List overseerDesignates = (List) m.get("overseer");
|
||||
if(overseerDesignates==null || overseerDesignates.isEmpty()) return;
|
||||
if(overseerDesignates.size() == 1 && overseerDesignates.contains(getLeaderNode(zk))) return;
|
||||
log.info("prioritizing overseer nodes at {}", LeaderElector.getNodeName(myId));
|
||||
log.info("overseer designates {}", overseerDesignates);
|
||||
String ldr = getLeaderNode(zk);
|
||||
if(overseerDesignates.contains(ldr)) return;
|
||||
log.info("prioritizing overseer nodes at {} overseer designates are {}", myId, overseerDesignates);
|
||||
List<String> electionNodes = getSortedElectionNodes(zk);
|
||||
if(electionNodes.size()<2) return;
|
||||
log.info("sorted nodes {}", electionNodes);
|
||||
|
||||
List<String> nodeNames = getSortedOverseerNodeNames(zk);
|
||||
if(nodeNames.size()<2) return;
|
||||
boolean designateIsInFront = overseerDesignates.contains( nodeNames.get(0));
|
||||
|
||||
ArrayList<String> nodesTobePushedBack = new ArrayList<>();
|
||||
//ensure that the node right behind the leader , i.e at position 1 is a Overseer
|
||||
List<String> availableDesignates = new ArrayList<>();
|
||||
|
||||
log.info("sorted nodes {}", nodeNames);//TODO to be removed
|
||||
for (int i = 1; i < nodeNames.size(); i++) {
|
||||
String s = nodeNames.get(i);
|
||||
|
||||
if (overseerDesignates.contains(s)) {
|
||||
availableDesignates.add(s);
|
||||
|
||||
for(int j=1;j<i;j++){
|
||||
String n = nodeNames.get(j);
|
||||
if(!overseerDesignates.contains(n)) {
|
||||
if(!nodesTobePushedBack.contains(n)) nodesTobePushedBack.add(n);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
if(availableDesignates.size()>1) break;//we don't need to line up more than 2 designates
|
||||
}
|
||||
|
||||
if(!availableDesignates.isEmpty()){
|
||||
for (String s : nodesTobePushedBack) {
|
||||
log.info("pushing back {} ", s);
|
||||
invokeOverseerOp(s, "rejoin");
|
||||
}
|
||||
|
||||
//wait for a while to ensure the designate has indeed come in front
|
||||
boolean prioritizationComplete = false;
|
||||
long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(2500, TimeUnit.MILLISECONDS);
|
||||
|
||||
while (System.nanoTime() < timeout) {
|
||||
List<String> currentNodeNames = getSortedOverseerNodeNames(zk);
|
||||
|
||||
int totalLeaders = 0;
|
||||
|
||||
for(int i=0;i<availableDesignates.size();i++) {
|
||||
if(overseerDesignates.contains(currentNodeNames.get(i))) totalLeaders++;
|
||||
}
|
||||
if(totalLeaders == availableDesignates.size()){
|
||||
prioritizationComplete = true;
|
||||
String designateNodeId = null;
|
||||
for (String electionNode : electionNodes) {
|
||||
if(overseerDesignates.contains( LeaderElector.getNodeName(electionNode))){
|
||||
designateNodeId = electionNode;
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(50);
|
||||
} catch (InterruptedException e) {
|
||||
log.warn("Thread interrupted",e);
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
if(!prioritizationComplete) {
|
||||
log.warn("available designates and current state {} {} ", availableDesignates, getSortedOverseerNodeNames(zk));
|
||||
}
|
||||
|
||||
} else if(!designateIsInFront) {
|
||||
log.warn("No overseer designates are available, overseerDesignates: {}, live nodes : {}",overseerDesignates,nodeNames);
|
||||
if(designateNodeId == null){
|
||||
log.warn("No live overseer designate ");
|
||||
return;
|
||||
}
|
||||
|
||||
String leaderNode = getLeaderNode(zkStateReader.getZkClient());
|
||||
if(leaderNode ==null) return;
|
||||
if(!overseerDesignates.contains(leaderNode) ){
|
||||
List<String> sortedNodes = getSortedOverseerNodeNames(zk);
|
||||
|
||||
if(leaderNode.equals(sortedNodes.get(0)) || // I am leader and I am in front of the queue
|
||||
overseerDesignates.contains(sortedNodes.get(0))) {// I am leader but somebody else is in the front , Screwed up leader election
|
||||
//this means there are I am not a designate and the next guy is lined up to become the leader, kill myself
|
||||
log.info("I am not an overseer designate , forcing myself out {} ", leaderNode);
|
||||
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.QUIT)));
|
||||
}
|
||||
if(!designateNodeId.equals( electionNodes.get(1))) { //checking if it is already at no:1
|
||||
log.info("asking node {} to come join election at head", designateNodeId);
|
||||
invokeOverseerOp(designateNodeId, "rejoinAtHead"); //ask designate to come first
|
||||
log.info("asking the old first in line {} to rejoin election ",electionNodes.get(1) );
|
||||
invokeOverseerOp(electionNodes.get(1), "rejoin");//ask second inline to go behind
|
||||
}
|
||||
//now ask the current leader to QUIT , so that the designate can takeover
|
||||
Overseer.getInQueue(zkStateReader.getZkClient()).offer(
|
||||
ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.QUIT,
|
||||
"id",getLeaderId(zkStateReader.getZkClient()))));
|
||||
|
||||
}
|
||||
|
||||
|
@ -513,28 +460,46 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
return nodeNames;
|
||||
}
|
||||
|
||||
public static List<String> getSortedElectionNodes(SolrZkClient zk) throws KeeperException, InterruptedException {
|
||||
List<String> children = null;
|
||||
try {
|
||||
children = zk.getChildren(OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE, null, true);
|
||||
LeaderElector.sortSeqs(children);
|
||||
return children;
|
||||
} catch (Exception e) {
|
||||
throw e;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static String getLeaderNode(SolrZkClient zkClient) throws KeeperException, InterruptedException {
|
||||
byte[] data = new byte[0];
|
||||
String id = getLeaderId(zkClient);
|
||||
return id==null ?
|
||||
null:
|
||||
LeaderElector.getNodeName( id);
|
||||
}
|
||||
|
||||
public static String getLeaderId(SolrZkClient zkClient) throws KeeperException,InterruptedException{
|
||||
byte[] data = null;
|
||||
try {
|
||||
data = zkClient.getData("/overseer_elect/leader", null, new Stat(), true);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
return null;
|
||||
}
|
||||
Map m = (Map) ZkStateReader.fromJSON(data);
|
||||
String s = (String) m.get("id");
|
||||
String nodeName = LeaderElector.getNodeName(s);
|
||||
return nodeName;
|
||||
return (String) m.get("id");
|
||||
}
|
||||
|
||||
private void invokeOverseerOp(String nodeName, String op) {
|
||||
private void invokeOverseerOp(String electionNode, String op) {
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
|
||||
params.set(CoreAdminParams.ACTION, CoreAdminAction.OVERSEEROP.toString());
|
||||
params.set("op", op);
|
||||
params.set("qt", adminPath);
|
||||
params.set("electionNode", electionNode);
|
||||
ShardRequest sreq = new ShardRequest();
|
||||
sreq.purpose = 1;
|
||||
String replica = zkStateReader.getBaseUrlForNodeName(nodeName);
|
||||
String replica = zkStateReader.getBaseUrlForNodeName(LeaderElector.getNodeName(electionNode));
|
||||
sreq.shards = new String[]{replica};
|
||||
sreq.actualShards = sreq.shards;
|
||||
sreq.params = params;
|
||||
|
|
|
@ -1691,9 +1691,33 @@ public final class ZkController {
|
|||
return out;
|
||||
}
|
||||
|
||||
public void rejoinOverseerElection() {
|
||||
public void rejoinOverseerElection(String electionNode, boolean joinAtHead) {
|
||||
try {
|
||||
overseerElector.retryElection();
|
||||
if(electionNode !=null){
|
||||
//this call is from inside the JVM . not from CoreAdminHandler
|
||||
if(overseerElector.getContext() == null || overseerElector.getContext().leaderSeqPath == null){
|
||||
overseerElector.retryElection(new OverseerElectionContext(zkClient,
|
||||
overseer, getNodeName()), joinAtHead);
|
||||
return;
|
||||
}
|
||||
if(!overseerElector.getContext().leaderSeqPath.endsWith(electionNode)){
|
||||
log.warn("Asked to rejoin with wrong election node : {}, current node is {}",electionNode, overseerElector.getContext().leaderSeqPath);
|
||||
//however delete it . This is possible when the last attempt at deleting the election node failed.
|
||||
if(electionNode.startsWith(getNodeName())){
|
||||
try {
|
||||
zkClient.delete(OverseerElectionContext.PATH+LeaderElector.ELECTION_NODE+"/"+electionNode,-1,true);
|
||||
} catch (NoNodeException e) {
|
||||
//no problem
|
||||
} catch (InterruptedException e){
|
||||
Thread.currentThread().interrupt();
|
||||
} catch(Exception e) {
|
||||
log.warn("Old election node exists , could not be removed ",e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}else {
|
||||
overseerElector.retryElection(overseerElector.getContext(), joinAtHead);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to rejoin election", e);
|
||||
}
|
||||
|
|
|
@ -272,7 +272,12 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
ZkController zkController = coreContainer.getZkController();
|
||||
if(zkController != null){
|
||||
String op = req.getParams().get("op");
|
||||
if ("rejoin".equals(op)) zkController.rejoinOverseerElection();
|
||||
String electionNode = req.getParams().get("electionNode");
|
||||
if(electionNode != null) {
|
||||
zkController.rejoinOverseerElection(electionNode, "rejoinAtHead".equals(op));
|
||||
} else {
|
||||
log.info("electionNode is required param");
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -17,33 +17,11 @@
|
|||
|
||||
package org.apache.solr.request;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.EnumSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.FutureTask;
|
||||
import java.util.concurrent.RunnableFuture;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.MultiDocsEnum;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
@ -61,7 +39,6 @@ import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
|
@ -82,7 +59,6 @@ import org.apache.solr.schema.SchemaField;
|
|||
import org.apache.solr.schema.TrieDateField;
|
||||
import org.apache.solr.schema.TrieField;
|
||||
import org.apache.solr.search.BitDocSet;
|
||||
import org.apache.solr.search.DocIterator;
|
||||
import org.apache.solr.search.DocSet;
|
||||
import org.apache.solr.search.Grouping;
|
||||
import org.apache.solr.search.HashDocSet;
|
||||
|
@ -96,7 +72,26 @@ import org.apache.solr.search.grouping.GroupingSpecification;
|
|||
import org.apache.solr.util.BoundedTreeSet;
|
||||
import org.apache.solr.util.DateMathParser;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.solr.util.LongPriorityQueue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.EnumSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.FutureTask;
|
||||
import java.util.concurrent.RunnableFuture;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* A class that generates simple Facet information for a request.
|
||||
|
@ -315,8 +310,7 @@ public class SimpleFacets {
|
|||
* @see FacetParams#FACET_QUERY
|
||||
*/
|
||||
public int getGroupedFacetQueryCount(Query facetQuery) throws IOException {
|
||||
GroupingSpecification groupingSpecification = rb.getGroupingSpec();
|
||||
String groupField = groupingSpecification != null ? groupingSpecification.getFields()[0] : null;
|
||||
String groupField = params.get(GroupParams.GROUP_FIELD);
|
||||
if (groupField == null) {
|
||||
throw new SolrException (
|
||||
SolrException.ErrorCode.BAD_REQUEST,
|
||||
|
|
|
@ -591,44 +591,7 @@ public class IndexSchema {
|
|||
// expression = "/schema/copyField";
|
||||
|
||||
dynamicCopyFields = new DynamicCopy[] {};
|
||||
expression = "//" + COPY_FIELD;
|
||||
nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
|
||||
|
||||
for (int i=0; i<nodes.getLength(); i++) {
|
||||
node = nodes.item(i);
|
||||
NamedNodeMap attrs = node.getAttributes();
|
||||
|
||||
String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
|
||||
String dest = DOMUtil.getAttr(attrs, DESTINATION, COPY_FIELD + " definition");
|
||||
String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
|
||||
int maxCharsInt = CopyField.UNLIMITED;
|
||||
if (maxChars != null) {
|
||||
try {
|
||||
maxCharsInt = Integer.parseInt(maxChars);
|
||||
} catch (NumberFormatException e) {
|
||||
log.warn("Couldn't parse " + MAX_CHARS + " attribute for " + COPY_FIELD + " from "
|
||||
+ source + " to " + dest + " as integer. The whole field will be copied.");
|
||||
}
|
||||
}
|
||||
|
||||
if (dest.equals(uniqueKeyFieldName)) {
|
||||
String msg = UNIQUE_KEY + " field ("+uniqueKeyFieldName+
|
||||
") can not be the " + DESTINATION + " of a " + COPY_FIELD + "(" + SOURCE + "=" +source+")";
|
||||
log.error(msg);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, msg);
|
||||
|
||||
}
|
||||
|
||||
registerCopyField(source, dest, maxCharsInt);
|
||||
}
|
||||
|
||||
for (Map.Entry<SchemaField, Integer> entry : copyFieldTargetCounts.entrySet()) {
|
||||
if (entry.getValue() > 1 && !entry.getKey().multiValued()) {
|
||||
log.warn("Field " + entry.getKey().name + " is not multivalued "+
|
||||
"and destination for multiple " + COPY_FIELDS + " ("+
|
||||
entry.getValue()+")");
|
||||
}
|
||||
}
|
||||
loadCopyFields(document, xpath);
|
||||
|
||||
//Run the callbacks on SchemaAware now that everything else is done
|
||||
for (SchemaAware aware : schemaAware) {
|
||||
|
@ -746,6 +709,50 @@ public class IndexSchema {
|
|||
return explicitRequiredProp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the copy fields
|
||||
*/
|
||||
protected synchronized void loadCopyFields(Document document, XPath xpath) throws XPathExpressionException {
|
||||
String expression = "//" + COPY_FIELD;
|
||||
NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
|
||||
|
||||
for (int i=0; i<nodes.getLength(); i++) {
|
||||
Node node = nodes.item(i);
|
||||
NamedNodeMap attrs = node.getAttributes();
|
||||
|
||||
String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
|
||||
String dest = DOMUtil.getAttr(attrs, DESTINATION, COPY_FIELD + " definition");
|
||||
String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
|
||||
|
||||
int maxCharsInt = CopyField.UNLIMITED;
|
||||
if (maxChars != null) {
|
||||
try {
|
||||
maxCharsInt = Integer.parseInt(maxChars);
|
||||
} catch (NumberFormatException e) {
|
||||
log.warn("Couldn't parse " + MAX_CHARS + " attribute for " + COPY_FIELD + " from "
|
||||
+ source + " to " + dest + " as integer. The whole field will be copied.");
|
||||
}
|
||||
}
|
||||
|
||||
if (dest.equals(uniqueKeyFieldName)) {
|
||||
String msg = UNIQUE_KEY + " field ("+uniqueKeyFieldName+
|
||||
") can not be the " + DESTINATION + " of a " + COPY_FIELD + "(" + SOURCE + "=" +source+")";
|
||||
log.error(msg);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, msg);
|
||||
}
|
||||
|
||||
registerCopyField(source, dest, maxCharsInt);
|
||||
}
|
||||
|
||||
for (Map.Entry<SchemaField, Integer> entry : copyFieldTargetCounts.entrySet()) {
|
||||
if (entry.getValue() > 1 && !entry.getKey().multiValued()) {
|
||||
log.warn("Field " + entry.getKey().name + " is not multivalued "+
|
||||
"and destination for multiple " + COPY_FIELDS + " ("+
|
||||
entry.getValue()+")");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a sequence of path steps into a rooted path, by inserting slashes in front of each step.
|
||||
* @param steps The steps to join with slashes to form a path
|
||||
|
|
|
@ -43,6 +43,7 @@ import java.nio.charset.StandardCharsets;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** Solr-managed schema - non-user-editable, but can be mutable via internal and external REST API requests. */
|
||||
|
@ -338,6 +339,13 @@ public final class ManagedIndexSchema extends IndexSchema {
|
|||
Document document = schemaConf.getDocument();
|
||||
final XPath xpath = schemaConf.getXPath();
|
||||
newSchema.loadFields(document, xpath);
|
||||
// let's completely rebuild the copy fields from the schema in ZK.
|
||||
// create new copyField-related objects so we don't affect the
|
||||
// old schema
|
||||
newSchema.copyFieldsMap = new HashMap<>();
|
||||
newSchema.dynamicCopyFields = null;
|
||||
newSchema.copyFieldTargetCounts = new HashMap<>();
|
||||
newSchema.loadCopyFields(document, xpath);
|
||||
if (null != uniqueKeyField) {
|
||||
newSchema.requiredFields.add(uniqueKeyField);
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.solr.cloud;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import static org.apache.solr.cloud.OverseerCollectionProcessor.MAX_SHARDS_PER_NODE;
|
||||
import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
|
||||
import static org.apache.solr.cloud.OverseerCollectionProcessor.REPLICATION_FACTOR;
|
||||
|
@ -173,80 +174,45 @@ public class OverseerRolesTest extends AbstractFullDistribZkTestBase{
|
|||
log.info("Adding another overseer designate {}", anotherOverseer);
|
||||
setOverseerRole(CollectionAction.ADDROLE, anotherOverseer);
|
||||
|
||||
timeout = System.currentTimeMillis()+10000;
|
||||
leaderchanged = false;
|
||||
for(;System.currentTimeMillis() < timeout;){
|
||||
List<String> sortedNodeNames = getSortedOverseerNodeNames(client.getZkStateReader().getZkClient());
|
||||
if(sortedNodeNames.get(1) .equals(anotherOverseer) || sortedNodeNames.get(0).equals(anotherOverseer)){
|
||||
leaderchanged =true;
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
assertTrue("New overseer not the frontrunner : "+ getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) + " expected : "+ anotherOverseer, leaderchanged);
|
||||
|
||||
|
||||
String currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
|
||||
|
||||
String killedOverseer = currentOverseer;
|
||||
|
||||
log.info("Current Overseer {}", currentOverseer);
|
||||
Pattern pattern = Pattern.compile("(.*):(\\d*)(.*)");
|
||||
Matcher m = pattern.matcher(currentOverseer);
|
||||
JettySolrRunner stoppedJetty =null;
|
||||
|
||||
String hostPort = null;
|
||||
String hostPort = currentOverseer.substring(0,currentOverseer.indexOf('_'));
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
if(m.matches()){
|
||||
hostPort = m.group(1)+":"+m.group(2);
|
||||
|
||||
//
|
||||
//
|
||||
log.info("hostPort : {}", hostPort);
|
||||
|
||||
JettySolrRunner leaderJetty = null;
|
||||
|
||||
for (JettySolrRunner jetty : jettys) {
|
||||
String s = jetty.getBaseUrl().toString();
|
||||
log.info("jetTy {}",s);
|
||||
sb.append(s).append(" , ");
|
||||
if(s.contains(hostPort)){
|
||||
log.info("leader node {}",s);
|
||||
ChaosMonkey.stop(jetty);
|
||||
stoppedJetty = jetty;
|
||||
timeout = System.currentTimeMillis()+10000;
|
||||
leaderchanged = false;
|
||||
for(;System.currentTimeMillis() < timeout;){
|
||||
currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
|
||||
if(anotherOverseer.equals(currentOverseer)){
|
||||
leaderchanged =true;
|
||||
if (s.contains(hostPort)) {
|
||||
leaderJetty = jetty;
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
assertTrue("New overseer designate has not become the overseer, expected : "+ anotherOverseer + "actual : "+ currentOverseer, leaderchanged);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
} else{
|
||||
fail("pattern didn't match for"+currentOverseer );
|
||||
}
|
||||
|
||||
if(stoppedJetty !=null) {
|
||||
ChaosMonkey.start(stoppedJetty);
|
||||
assertNotNull("Could not find a jetty2 kill", leaderJetty);
|
||||
|
||||
log.info("leader node {}", leaderJetty.getBaseUrl());
|
||||
log.info ("current election Queue", OverseerCollectionProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient()));
|
||||
ChaosMonkey.stop(leaderJetty);
|
||||
timeout = System.currentTimeMillis() + 10000;
|
||||
leaderchanged = false;
|
||||
for (; System.currentTimeMillis() < timeout; ) {
|
||||
List<String> sortedNodeNames = getSortedOverseerNodeNames(client.getZkStateReader().getZkClient());
|
||||
if (sortedNodeNames.get(1).equals(killedOverseer) || sortedNodeNames.get(0).equals(killedOverseer)) {
|
||||
currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
|
||||
if (anotherOverseer.equals(currentOverseer)) {
|
||||
leaderchanged = true;
|
||||
break;
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
assertTrue("New overseer not the frontrunner : " + getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) + " expected : " + killedOverseer, leaderchanged);
|
||||
} else {
|
||||
log.warn("The jetty where the overseer {} is running could not be located in {}",hostPort,sb);
|
||||
}
|
||||
assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(client.getZkStateReader().getZkClient()), leaderchanged);
|
||||
}
|
||||
|
||||
private void setOverseerRole(CollectionAction action, String overseerDesignate) throws Exception, IOException {
|
||||
|
@ -261,6 +227,7 @@ public class OverseerRolesTest extends AbstractFullDistribZkTestBase{
|
|||
client.request(request);
|
||||
}
|
||||
|
||||
|
||||
protected void createCollection(String COLL_NAME, CloudSolrServer client) throws Exception {
|
||||
int replicationFactor = 2;
|
||||
int numShards = 4;
|
||||
|
|
|
@ -0,0 +1,139 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RollingRestartTest extends AbstractFullDistribZkTestBase {
|
||||
public static Logger log = LoggerFactory.getLogger(ChaosMonkeyNothingIsSafeTest.class);
|
||||
|
||||
public RollingRestartTest() {
|
||||
fixShardCount = true;
|
||||
sliceCount = 2;
|
||||
shardCount = 16;
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
System.setProperty("numShards", Integer.toString(sliceCount));
|
||||
useFactory("solr.StandardDirectoryFactory");
|
||||
}
|
||||
|
||||
@Override
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
System.clearProperty("numShards");
|
||||
super.tearDown();
|
||||
resetExceptionIgnores();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
waitForRecoveriesToFinish(false);
|
||||
|
||||
restartWithRolesTest();
|
||||
|
||||
waitForRecoveriesToFinish(false);
|
||||
}
|
||||
|
||||
|
||||
public void restartWithRolesTest() throws Exception {
|
||||
String leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
assertNotNull(leader);
|
||||
log.info("Current overseer leader = {}", leader);
|
||||
|
||||
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
|
||||
|
||||
int numOverseers = 3;
|
||||
List<String> designates = new ArrayList<>();
|
||||
List<CloudJettyRunner> overseerDesignates = new ArrayList<>();
|
||||
for (int i = 0; i < numOverseers; i++) {
|
||||
int n = random().nextInt(shardCount);
|
||||
String nodeName = cloudJettys.get(n).nodeName;
|
||||
log.info("Chose {} as overseer designate", nodeName);
|
||||
invokeCollectionApi(CollectionParams.ACTION, CollectionParams.CollectionAction.ADDROLE.toLower(), "role", "overseer", "node", nodeName);
|
||||
designates.add(nodeName);
|
||||
overseerDesignates.add(cloudJettys.get(n));
|
||||
}
|
||||
|
||||
waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
|
||||
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
|
||||
|
||||
int numRestarts = 4; // 1 + random().nextInt(5);
|
||||
for (int i = 0; i < numRestarts; i++) {
|
||||
log.info("Rolling restart #{}", i + 1);
|
||||
for (CloudJettyRunner cloudJetty : overseerDesignates) {
|
||||
log.info("Restarting {}", cloudJetty);
|
||||
chaosMonkey.stopJetty(cloudJetty);
|
||||
boolean success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
if (!success) {
|
||||
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
if(leader == null) log.error("NOOVERSEER election queue is :"+ OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
fail("No overseer designate as leader found after restart #" + (i + 1) + ": " + leader);
|
||||
}
|
||||
cloudJetty.jetty.start();
|
||||
success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
if (!success) {
|
||||
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
if(leader == null) log.error("NOOVERSEER election queue is :"+ OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
fail("No overseer leader found after restart #" + (i + 1) + ": " + leader);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
assertNotNull(leader);
|
||||
log.info("Current overseer leader (after restart) = {}", leader);
|
||||
|
||||
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
|
||||
}
|
||||
|
||||
static boolean waitUntilOverseerDesignateIsLeader(SolrZkClient testZkClient, List<String> overseerDesignates, int timeoutInSeconds) throws KeeperException, InterruptedException {
|
||||
long now = System.nanoTime();
|
||||
long timeout = now + TimeUnit.NANOSECONDS.convert(timeoutInSeconds, TimeUnit.SECONDS);
|
||||
boolean firstTime = true;
|
||||
int stableCheckTimeout = 2000;
|
||||
while (System.nanoTime() < timeout) {
|
||||
String newLeader = OverseerCollectionProcessor.getLeaderNode(testZkClient);
|
||||
if (!overseerDesignates.contains(newLeader)) {
|
||||
Thread.sleep(500);
|
||||
} else {
|
||||
if (firstTime) {
|
||||
firstTime = false;
|
||||
Thread.sleep(stableCheckTimeout);
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -396,7 +396,7 @@ public class DistributedDebugComponentTest extends SolrJettyTestBase {
|
|||
assertEquals("Unexpected response size for shard", 1, badShardTrack.size());
|
||||
Entry<String, String> exception = badShardTrack.iterator().next();
|
||||
assertEquals("Expected key 'Exception' not found", "Exception", exception.getKey());
|
||||
assertTrue("Unexpected exception message", exception.getValue().contains("Server refused connection"));
|
||||
assertNotNull("Exception message should not be null", exception.getValue());
|
||||
unIgnoreException("Server refused connection");
|
||||
}
|
||||
|
||||
|
|
|
@ -1,123 +0,0 @@
|
|||
package org.apache.solr.schema;
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.util.BaseTestHarness;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.restlet.ext.servlet.ServerServlet;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class TestCloudManagedSchemaAddField extends AbstractFullDistribZkTestBase {
|
||||
private static final Logger log = LoggerFactory.getLogger(TestCloudManagedSchemaAddField.class);
|
||||
|
||||
public TestCloudManagedSchemaAddField() {
|
||||
super();
|
||||
fixShardCount = true;
|
||||
|
||||
sliceCount = 4;
|
||||
shardCount = 8;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void initSysProperties() {
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
System.setProperty("enable.update.log", "true");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCloudSolrConfig() {
|
||||
return "solrconfig-managed-schema.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedMap<ServletHolder,String> getExtraServlets() {
|
||||
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
|
||||
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
|
||||
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
|
||||
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
|
||||
return extraServlets;
|
||||
}
|
||||
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (int i = 0 ; i < clients.size() ; ++i) {
|
||||
final HttpSolrServer client = (HttpSolrServer)clients.get(i);
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return client.getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
setupHarnesses();
|
||||
|
||||
// First. add a bunch of fields, but do it fast enough
|
||||
// and verify shards' schemas after all of them are added
|
||||
int numFields = 25;
|
||||
for (int i = 1 ; i <= numFields ; ++i) {
|
||||
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
String newFieldName = "newfield" + i;
|
||||
final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
|
||||
String request = "/schema/fields/" + newFieldName + "?wt=xml";
|
||||
String response = publisher.put(request, content);
|
||||
String result = publisher.validateXPath
|
||||
(response, "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
|
||||
if (null != result) {
|
||||
fail("PUT REQUEST FAILED: xpath=" + result + " request=" + request
|
||||
+ " content=" + content + " response=" + response);
|
||||
}
|
||||
}
|
||||
|
||||
Thread.sleep(100000);
|
||||
|
||||
for (int i = 1 ; i <= numFields ; ++i) {
|
||||
String newFieldName = "newfield" + i;
|
||||
for (RestTestHarness client : restTestHarnesses) {
|
||||
String request = "/schema/fields/" + newFieldName + "?wt=xml";
|
||||
String response = client.query(request);
|
||||
String result = client.validateXPath(response,
|
||||
"/response/lst[@name='responseHeader']/int[@name='status'][.='0']",
|
||||
"/response/lst[@name='field']/str[@name='name'][.='" + newFieldName + "']");
|
||||
if (null != result) {
|
||||
if (response.contains("Field '" + newFieldName + "' not found.")) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,123 +0,0 @@
|
|||
package org.apache.solr.schema;
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.util.BaseTestHarness;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.restlet.ext.servlet.ServerServlet;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class TestCloudManagedSchemaAddFields extends AbstractFullDistribZkTestBase {
|
||||
private static final Logger log = LoggerFactory.getLogger(TestCloudManagedSchemaAddField.class);
|
||||
|
||||
public TestCloudManagedSchemaAddFields() {
|
||||
super();
|
||||
fixShardCount = true;
|
||||
|
||||
sliceCount = 4;
|
||||
shardCount = 8;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void initSysProperties() {
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
System.setProperty("enable.update.log", "true");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCloudSolrConfig() {
|
||||
return "solrconfig-managed-schema.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedMap<ServletHolder,String> getExtraServlets() {
|
||||
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
|
||||
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
|
||||
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
|
||||
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
|
||||
return extraServlets;
|
||||
}
|
||||
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (int i = 0 ; i < clients.size() ; ++i) {
|
||||
final HttpSolrServer client = (HttpSolrServer)clients.get(i);
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return client.getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
setupHarnesses();
|
||||
|
||||
// First. add a bunch of fields, but do it fast enough
|
||||
// and verify shards' schemas after all of them are added
|
||||
int numFields = 200;
|
||||
for (int i = 1 ; i <= numFields ; ++i) {
|
||||
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
String newFieldName = "newfield" + i;
|
||||
final String content = "[{\"name\":\""+newFieldName+"\",\"type\":\"text\",\"stored\":\"false\"}]";
|
||||
String request = "/schema/fields/?wt=xml";
|
||||
String response = publisher.post(request, content);
|
||||
String result = publisher.validateXPath
|
||||
(response, "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
|
||||
if (null != result) {
|
||||
fail("POST REQUEST FAILED: xpath=" + result + " request=" + request
|
||||
+ " content=" + content + " response=" + response);
|
||||
}
|
||||
}
|
||||
|
||||
Thread.sleep(100000);
|
||||
|
||||
for (int i = 1 ; i <= numFields ; ++i) {
|
||||
String newFieldName = "newfield" + i;
|
||||
for (RestTestHarness client : restTestHarnesses) {
|
||||
String request = "/schema/fields/" + newFieldName + "?wt=xml";
|
||||
String response = client.query(request);
|
||||
String result = client.validateXPath(response,
|
||||
"/response/lst[@name='responseHeader']/int[@name='status'][.='0']",
|
||||
"/response/lst[@name='field']/str[@name='name'][.='" + newFieldName + "']");
|
||||
if (null != result) {
|
||||
if (response.contains("Field '" + newFieldName + "' not found.")) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,251 @@
|
|||
package org.apache.solr.schema;
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.solr.client.solrj.SolrServer;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.util.BaseTestHarness;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.restlet.ext.servlet.ServerServlet;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class TestCloudManagedSchemaConcurrent extends AbstractFullDistribZkTestBase {
|
||||
private static final Logger log = LoggerFactory.getLogger(TestCloudManagedSchemaConcurrent.class);
|
||||
private static final String SUCCESS_XPATH = "/response/lst[@name='responseHeader']/int[@name='status'][.='0']";
|
||||
|
||||
public TestCloudManagedSchemaConcurrent() {
|
||||
super();
|
||||
fixShardCount = true;
|
||||
|
||||
sliceCount = 4;
|
||||
shardCount = 8;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void initSysProperties() {
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
System.setProperty("enable.update.log", "true");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCloudSolrConfig() {
|
||||
return "solrconfig-managed-schema.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedMap<ServletHolder,String> getExtraServlets() {
|
||||
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
|
||||
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
|
||||
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
|
||||
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
|
||||
return extraServlets;
|
||||
}
|
||||
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (final SolrServer client : clients) {
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return ((HttpSolrServer)client).getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifySuccess(String request, String response) throws Exception {
|
||||
String result = BaseTestHarness.validateXPath(response, SUCCESS_XPATH);
|
||||
if (null != result) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
}
|
||||
|
||||
private void addFieldPut(RestTestHarness publisher, String fieldName) throws Exception {
|
||||
final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
|
||||
String request = "/schema/fields/" + fieldName + "?wt=xml";
|
||||
String response = publisher.put(request, content);
|
||||
verifySuccess(request, response);
|
||||
}
|
||||
|
||||
private void addFieldPost(RestTestHarness publisher, String fieldName) throws Exception {
|
||||
final String content = "[{\"name\":\""+fieldName+"\",\"type\":\"text\",\"stored\":\"false\"}]";
|
||||
String request = "/schema/fields/?wt=xml";
|
||||
String response = publisher.post(request, content);
|
||||
verifySuccess(request, response);
|
||||
}
|
||||
|
||||
private void copyField(RestTestHarness publisher, String source, String dest) throws Exception {
|
||||
final String content = "[{\"source\":\""+source+"\",\"dest\":[\""+dest+"\"]}]";
|
||||
String request = "/schema/copyfields/?wt=xml";
|
||||
String response = publisher.post(request, content);
|
||||
verifySuccess(request, response);
|
||||
}
|
||||
|
||||
private String[] getExpectedFieldResponses(int numAddFieldPuts, int numAddFieldPosts) {
|
||||
String[] expectedAddFields = new String[1 + numAddFieldPuts + numAddFieldPosts];
|
||||
expectedAddFields[0] = SUCCESS_XPATH;
|
||||
|
||||
for (int i = 0; i < numAddFieldPuts; ++i) {
|
||||
String newFieldName = "newfieldPut" + i;
|
||||
expectedAddFields[1 + i]
|
||||
= "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
|
||||
}
|
||||
|
||||
for (int i = 0; i < numAddFieldPosts; ++i) {
|
||||
String newFieldName = "newfieldPost" + i;
|
||||
expectedAddFields[1 + numAddFieldPuts + i]
|
||||
= "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
|
||||
}
|
||||
|
||||
return expectedAddFields;
|
||||
}
|
||||
|
||||
private String[] getExpectedCopyFieldResponses(List<CopyFieldInfo> copyFields) {
|
||||
ArrayList<String> expectedCopyFields = new ArrayList<>();
|
||||
expectedCopyFields.add(SUCCESS_XPATH);
|
||||
for (CopyFieldInfo cpi : copyFields) {
|
||||
String expectedSourceName = cpi.getSourceField();
|
||||
expectedCopyFields.add
|
||||
("/response/arr[@name='copyFields']/lst/str[@name='source'][.='" + expectedSourceName + "']");
|
||||
String expectedDestName = cpi.getDestField();
|
||||
expectedCopyFields.add
|
||||
("/response/arr[@name='copyFields']/lst/str[@name='dest'][.='" + expectedDestName + "']");
|
||||
}
|
||||
|
||||
return expectedCopyFields.toArray(new String[expectedCopyFields.size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
setupHarnesses();
|
||||
|
||||
// First, add a bunch of fields via PUT and POST, as well as copyFields,
|
||||
// but do it fast enough and verify shards' schemas after all of them are added
|
||||
int numFields = 100;
|
||||
int numAddFieldPuts = 0;
|
||||
int numAddFieldPosts = 0;
|
||||
List<CopyFieldInfo> copyFields = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i <= numFields ; ++i) {
|
||||
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
|
||||
int type = random().nextInt(3);
|
||||
if (type == 0) { // send an add field via PUT
|
||||
addFieldPut(publisher, "newfieldPut" + numAddFieldPuts++);
|
||||
}
|
||||
else if (type == 1) { // send an add field via POST
|
||||
addFieldPost(publisher, "newfieldPost" + numAddFieldPosts++);
|
||||
}
|
||||
else if (type == 2) { // send a copy field
|
||||
String sourceField = null;
|
||||
String destField = null;
|
||||
|
||||
int sourceType = random().nextInt(3);
|
||||
if (sourceType == 0) { // existing
|
||||
sourceField = "name";
|
||||
} else if (sourceType == 1) { // newly created
|
||||
sourceField = "copySource" + i;
|
||||
addFieldPut(publisher, sourceField);
|
||||
} else { // dynamic
|
||||
sourceField = "*_dynamicSource" + i + "_t";
|
||||
// * only supported if both src and dst use it
|
||||
destField = "*_dynamicDest" + i + "_t";
|
||||
}
|
||||
|
||||
if (destField == null) {
|
||||
int destType = random().nextInt(2);
|
||||
if (destType == 0) { // existing
|
||||
destField = "title";
|
||||
} else { // newly created
|
||||
destField = "copyDest" + i;
|
||||
addFieldPut(publisher, destField);
|
||||
}
|
||||
}
|
||||
copyField(publisher, sourceField, destField);
|
||||
copyFields.add(new CopyFieldInfo(sourceField, destField));
|
||||
}
|
||||
}
|
||||
|
||||
String[] expectedAddFields = getExpectedFieldResponses(numAddFieldPuts, numAddFieldPosts);
|
||||
String[] expectedCopyFields = getExpectedCopyFieldResponses(copyFields);
|
||||
|
||||
boolean success = false;
|
||||
long maxTimeoutMillis = 100000;
|
||||
long startTime = System.nanoTime();
|
||||
String request = null;
|
||||
String response = null;
|
||||
String result = null;
|
||||
|
||||
while ( ! success
|
||||
&& TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutMillis) {
|
||||
Thread.sleep(100);
|
||||
|
||||
for (RestTestHarness client : restTestHarnesses) {
|
||||
// verify addFieldPuts and addFieldPosts
|
||||
request = "/schema/fields?wt=xml";
|
||||
response = client.query(request);
|
||||
result = BaseTestHarness.validateXPath(response, expectedAddFields);
|
||||
if (result != null) {
|
||||
break;
|
||||
}
|
||||
|
||||
// verify copyFields
|
||||
request = "/schema/copyfields?wt=xml";
|
||||
response = client.query(request);
|
||||
result = BaseTestHarness.validateXPath(response, expectedCopyFields);
|
||||
if (result != null) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
success = (result == null);
|
||||
}
|
||||
if ( ! success) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
}
|
||||
|
||||
private static class CopyFieldInfo {
|
||||
private String sourceField;
|
||||
private String destField;
|
||||
|
||||
public CopyFieldInfo(String sourceField, String destField) {
|
||||
this.sourceField = sourceField;
|
||||
this.destField = destField;
|
||||
}
|
||||
|
||||
public String getSourceField() { return sourceField; }
|
||||
public String getDestField() { return destField; }
|
||||
}
|
||||
}
|
|
@ -1,114 +0,0 @@
|
|||
package org.apache.solr.schema;
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.restlet.ext.servlet.ServerServlet;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class TestCloudManagedSchemaCopyFields extends AbstractFullDistribZkTestBase {
|
||||
private static final Logger log = LoggerFactory.getLogger(TestCloudManagedSchemaAddField.class);
|
||||
|
||||
public TestCloudManagedSchemaCopyFields() {
|
||||
super();
|
||||
fixShardCount = true;
|
||||
|
||||
sliceCount = 4;
|
||||
shardCount = 8;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void initSysProperties() {
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
System.setProperty("enable.update.log", "true");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCloudSolrConfig() {
|
||||
return "solrconfig-managed-schema.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedMap<ServletHolder,String> getExtraServlets() {
|
||||
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
|
||||
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
|
||||
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
|
||||
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
|
||||
return extraServlets;
|
||||
}
|
||||
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (int i = 0 ; i < clients.size() ; ++i) {
|
||||
final HttpSolrServer client = (HttpSolrServer)clients.get(i);
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return client.getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
setupHarnesses();
|
||||
|
||||
// First, add the same copy field directive a bunch of times.
|
||||
// Then verify each shard's schema has it.
|
||||
int numFields = 200;
|
||||
for (int i = 1 ; i <= numFields ; ++i) {
|
||||
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
final String content = "[{\"source\":\""+"sku1"+"\",\"dest\":[\"sku2\"]}]";
|
||||
String request = "/schema/copyfields/?wt=xml";
|
||||
String response = publisher.post(request, content);
|
||||
String result = publisher.validateXPath
|
||||
(response, "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
|
||||
if (null != result) {
|
||||
fail("POST REQUEST FAILED: xpath=" + result + " request=" + request
|
||||
+ " content=" + content + " response=" + response);
|
||||
}
|
||||
}
|
||||
|
||||
Thread.sleep(100000);
|
||||
|
||||
String request = "/schema/copyfields/?wt=xml&indent=on&source.fl=sku1";
|
||||
for (RestTestHarness client : restTestHarnesses) {
|
||||
String response = client.query(request);
|
||||
String result = client.validateXPath(response,
|
||||
"/response/lst[@name='responseHeader']/int[@name='status'][.='0']",
|
||||
"/response/arr[@name='copyFields']/lst/str[@name='dest'][.='sku2']");
|
||||
if (null != result) {
|
||||
fail("QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue