mirror of https://github.com/apache/lucene.git
LUCENE-6148: Accountable.getChildResources returns a Collection.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1649074 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
64d933d34d
commit
972e64f7c7
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.blockterms;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -854,7 +855,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (indexReader != null) {
|
||||
resources.add(Accountables.namedAccountable("term index", indexReader));
|
||||
|
|
|
@ -17,11 +17,19 @@ package org.apache.lucene.codecs.blockterms;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -29,14 +37,6 @@ import org.apache.lucene.util.IOUtils;
|
|||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
||||
/**
|
||||
* TermsIndexReader for simple every Nth terms indexes.
|
||||
*
|
||||
|
@ -262,7 +262,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (termOffsets != null) {
|
||||
resources.add(Accountables.namedAccountable("term lengths", termOffsets));
|
||||
|
@ -303,7 +303,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blockterms;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
||||
|
@ -165,7 +166,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -208,7 +209,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.blocktreeords;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -29,9 +30,9 @@ import org.apache.lucene.codecs.FieldsProducer;
|
|||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -234,7 +235,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blocktreeords;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
|
||||
|
@ -174,7 +175,7 @@ final class OrdsFieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -19,12 +19,13 @@ package org.apache.lucene.codecs.bloom;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
|
@ -407,7 +408,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", bloomsByFieldName));
|
||||
if (delegateFieldsProducer != null) {
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.memory;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -259,7 +260,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("numeric field", numericInstances));
|
||||
resources.addAll(Accountables.namedAccountables("binary field", binaryInstances));
|
||||
|
@ -661,7 +662,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (address != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", RamUsageEstimator.sizeOf(address)));
|
||||
|
@ -700,7 +701,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return docToOrd.getChildResources();
|
||||
}
|
||||
|
||||
|
@ -724,7 +725,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (docToAddress != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", docToAddress));
|
||||
|
@ -753,7 +754,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (docToOrdAddress != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", docToOrdAddress));
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.memory;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
@ -162,7 +163,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,21 +18,26 @@ package org.apache.lucene.codecs.memory;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.BlockTermState;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.TermState;
|
||||
|
@ -41,26 +46,22 @@ import org.apache.lucene.index.TermsEnum;
|
|||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
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.Accountables;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.automaton.ByteRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.Outputs;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.codecs.BlockTermState;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
|
||||
/**
|
||||
* FST-based terms dictionary reader.
|
||||
|
@ -288,7 +289,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -865,7 +866,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -20,18 +20,23 @@ package org.apache.lucene.codecs.memory;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.BlockTermState;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.TermState;
|
||||
|
@ -39,25 +44,21 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
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.Accountables;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.automaton.ByteRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.Outputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.codecs.BlockTermState;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
|
||||
/**
|
||||
* FST-based terms dictionary reader.
|
||||
|
@ -191,7 +192,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (dict == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -763,7 +764,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.memory;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -296,7 +297,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
|
||||
resources.addAll(Accountables.namedAccountables("pagedbytes field", pagedBytesInstances));
|
||||
|
@ -799,7 +800,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (addresses != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", addresses));
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.memory;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
@ -33,10 +34,10 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -963,7 +964,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -1038,7 +1039,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -17,8 +17,19 @@ package org.apache.lucene.codecs.simpletext;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -57,16 +68,6 @@ import org.apache.lucene.util.fst.PairOutputs;
|
|||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
|
||||
|
||||
class SimpleTextFieldsReader extends FieldsProducer {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
|
@ -595,7 +596,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -700,7 +701,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", termsCache);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,9 +18,8 @@ package org.apache.lucene.codecs.simpletext;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
|
@ -82,7 +81,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return impl.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.blocktree;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -281,7 +282,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blocktree;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -186,7 +187,7 @@ public final class FieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.lucene.util.BitUtil.zigZagDecode;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -189,7 +190,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
|
||||
long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
|
||||
|
|
|
@ -42,6 +42,7 @@ import java.io.EOFException;
|
|||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
|
@ -620,7 +621,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
|
||||
}
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
@ -1074,7 +1075,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("term vector index", indexReader));
|
||||
}
|
||||
|
||||
|
|
|
@ -17,29 +17,30 @@ package org.apache.lucene.codecs.lucene50;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.CONST_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.MONOTONIC_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_SINGLE_VALUED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_WITH_ADDRESSES;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
|
||||
|
||||
import java.io.Closeable; // javadocs
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -382,7 +383,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
|
||||
resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
|
||||
|
@ -872,7 +873,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", terms));
|
||||
resources.add(Accountables.namedAccountable("term addresses", termAddresses));
|
||||
|
|
|
@ -17,8 +17,19 @@ package org.apache.lucene.codecs.lucene50;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.CONST_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_BITSET;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_TABLE;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -45,16 +56,6 @@ import org.apache.lucene.util.packed.BlockPackedReader;
|
|||
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.CONST_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_BITSET;
|
||||
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED_TABLE;
|
||||
|
||||
/**
|
||||
* Reader for {@link Lucene50NormsFormat}
|
||||
*/
|
||||
|
@ -187,7 +188,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", instances);
|
||||
}
|
||||
|
||||
|
@ -255,7 +256,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(reader);
|
||||
}
|
||||
|
||||
|
@ -297,7 +298,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(ordsReader);
|
||||
}
|
||||
|
||||
|
@ -341,7 +342,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", live));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
|
@ -382,7 +383,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", set));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
|
@ -433,7 +434,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("common", ordsReader));
|
||||
children.add(Accountables.namedAccountable("uncommon", nestedInstance));
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.perfield;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
|
@ -330,7 +331,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("format", formats);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.codecs.perfield;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
|
@ -36,6 +37,7 @@ import org.apache.lucene.codecs.FieldsProducer;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterFields;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -45,8 +47,6 @@ import org.apache.lucene.util.Accountables;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
import static org.apache.lucene.index.FilterLeafReader.FilterFields;
|
||||
|
||||
/**
|
||||
* Enables per field postings support.
|
||||
* <p>
|
||||
|
@ -302,7 +302,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("format", formats);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
|
||||
|
@ -638,7 +638,7 @@ public class MultiDocValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("global ord deltas", globalOrdDeltas));
|
||||
resources.add(Accountables.namedAccountable("first segments", firstSegments));
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
|
@ -159,7 +160,7 @@ class SegmentDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
for (Accountable producer : dvProducers) {
|
||||
resources.add(Accountables.namedAccountable("delegate", producer));
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -500,7 +501,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
ensureOpen();
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("postings", core.fields));
|
||||
|
@ -516,7 +517,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
|
|||
if (getTermVectorsReader() != null) {
|
||||
resources.add(Accountables.namedAccountable("term vectors", getTermVectorsReader()));
|
||||
}
|
||||
return resources;
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ import static org.apache.lucene.search.DocIdSet.EMPTY;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -159,7 +160,7 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
// Sync to pull the current set of values:
|
||||
synchronized (cache) {
|
||||
// no need to clone, Accountable#namedAccountables already copies the data
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -70,7 +71,7 @@ public abstract class FilteredDocIdSet extends DocIdSet {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return _innerSet.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -286,7 +287,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
synchronized (this) {
|
||||
return Accountables.namedAccountables("segment", cache);
|
||||
}
|
||||
|
|
|
@ -270,7 +270,7 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("cache", cache));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -143,7 +143,7 @@ public class RAMDirectory extends BaseDirectory implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("file", fileMap);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.store;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.zip.CRC32;
|
||||
import java.util.zip.Checksum;
|
||||
|
@ -193,7 +194,7 @@ public class RAMOutputStream extends IndexOutput implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("file", file));
|
||||
}
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.util;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
|
@ -36,7 +37,7 @@ public interface Accountable {
|
|||
* The result should be a point-in-time snapshot (to avoid race conditions).
|
||||
* @see Accountables
|
||||
*/
|
||||
default Iterable<Accountable> getChildResources() {
|
||||
default Collection<Accountable> getChildResources() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
|
|
|
@ -122,7 +122,7 @@ public class Accountables {
|
|||
* that the passed in children Accountables (and all their descendants) were created
|
||||
* with one of the namedAccountable functions.
|
||||
*/
|
||||
public static Accountable namedAccountable(final String description, final Iterable<Accountable> children, final long bytes) {
|
||||
public static Accountable namedAccountable(final String description, final Collection<Accountable> children, final long bytes) {
|
||||
return new Accountable() {
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
|
@ -130,7 +130,7 @@ public class Accountables {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return children;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.io.OutputStream;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -445,7 +446,7 @@ public final class FST<T> implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (packed) {
|
||||
resources.add(Accountables.namedAccountable("node ref to address", nodeRefToAddress));
|
||||
|
|
|
@ -18,13 +18,13 @@ package org.apache.lucene.facet.taxonomy;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
@ -161,7 +161,7 @@ public class CachedOrdinalsReader extends OrdinalsReader implements Accountable
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
public synchronized Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("segment", ordsCache);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.join;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -90,7 +91,7 @@ public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter implement
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return filter.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.uninverting;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -603,12 +604,12 @@ class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", bytes));
|
||||
resources.add(Accountables.namedAccountable("ord -> term", termOrdToBytesOffset));
|
||||
resources.add(Accountables.namedAccountable("doc -> ord", docToTermOrd));
|
||||
return resources;
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -746,7 +747,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", bytes));
|
||||
resources.add(Accountables.namedAccountable("addresses", docToOffset));
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.idversion;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -232,7 +233,7 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.idversion;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -163,7 +164,7 @@ final class VersionFieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.io.IOException;
|
|||
import java.io.StringReader;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -74,7 +76,6 @@ import org.apache.lucene.search.TopFieldCollector;
|
|||
import org.apache.lucene.search.TopFieldDocs;
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.search.suggest.Lookup;
|
||||
import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -83,6 +84,7 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
// javadocs
|
||||
|
||||
// TODO:
|
||||
// - a PostingsFormat that stores super-high-freq terms as
|
||||
|
@ -604,7 +606,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
/** Override this method to customize the Object
|
||||
* representing a single highlighted suggestions; the
|
||||
* result is set on each {@link
|
||||
* LookupResult#highlightKey} member. */
|
||||
* org.apache.lucene.search.suggest.Lookup.LookupResult#highlightKey} member. */
|
||||
protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
|
||||
try (TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text))) {
|
||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
|
@ -732,7 +734,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
try {
|
||||
if (searcherMgr != null) {
|
||||
|
@ -748,7 +750,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
searcherMgr.release(searcher);
|
||||
}
|
||||
}
|
||||
return resources;
|
||||
return Collections.unmodifiableList(resources);
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
|
|
|
@ -17,11 +17,14 @@ package org.apache.lucene.search.suggest.analyzing;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
|
@ -62,8 +65,6 @@ import org.apache.lucene.util.fst.Util;
|
|||
import org.apache.lucene.util.fst.Util.Result;
|
||||
import org.apache.lucene.util.fst.Util.TopResults;
|
||||
|
||||
import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
|
||||
|
||||
/**
|
||||
* Suggester that first analyzes the surface form, adds the
|
||||
* analyzed form to a weighted FST, and then does the same
|
||||
|
@ -262,7 +263,7 @@ public class AnalyzingSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -21,11 +21,11 @@ package org.apache.lucene.search.suggest.analyzing;
|
|||
// - test w/ syns
|
||||
// - add pruning of low-freq ngrams?
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
|
@ -69,14 +69,14 @@ import org.apache.lucene.util.IOUtils;
|
|||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.FST.Arc;
|
||||
import org.apache.lucene.util.fst.FST.BytesReader;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.Outputs;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.util.fst.Util.Result;
|
||||
import org.apache.lucene.util.fst.Util.TopResults;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
//import java.io.PrintWriter;
|
||||
|
||||
|
@ -213,7 +213,7 @@ public class FreeTextSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.IOException;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -37,13 +39,11 @@ import org.apache.lucene.util.Accountables;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.OfflineSorter;
|
||||
import org.apache.lucene.util.OfflineSorter.SortInfo;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.NoOutputs;
|
||||
|
||||
|
@ -315,7 +315,7 @@ public class FSTCompletionLookup extends Lookup implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (normalCompletion != null) {
|
||||
resources.add(Accountables.namedAccountable("fst", normalCompletion.getFST()));
|
||||
|
@ -323,7 +323,7 @@ public class FSTCompletionLookup extends Lookup implements Accountable {
|
|||
if (higherWeightsCompletion != null && (normalCompletion == null || normalCompletion.getFST() != higherWeightsCompletion.getFST())) {
|
||||
resources.add(Accountables.namedAccountable("higher weights fst", higherWeightsCompletion.getFST()));
|
||||
}
|
||||
return resources;
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest.fst;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
@ -40,13 +41,13 @@ import org.apache.lucene.util.CharsRefBuilder;
|
|||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.FST.Arc;
|
||||
import org.apache.lucene.util.fst.FST.BytesReader;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.util.fst.Util.Result;
|
||||
import org.apache.lucene.util.fst.Util.TopResults;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
/**
|
||||
* Suggester based on a weighted FST: it first traverses the prefix,
|
||||
|
@ -301,7 +302,7 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -18,14 +18,15 @@ package org.apache.lucene.codecs.asserting;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.index.AssertingLeafReader;
|
||||
import org.apache.lucene.index.AssertingLeafReader.AssertingRandomAccessOrds;
|
||||
import org.apache.lucene.index.AssertingLeafReader.AssertingSortedSetDocValues;
|
||||
import org.apache.lucene.index.AssertingLeafReader;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -292,9 +293,9 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
public Collection<Accountable> getChildResources() {
|
||||
Collection<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkReadOnly(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.asserting;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
|
@ -114,9 +115,9 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
public Collection<Accountable> getChildResources() {
|
||||
Collection<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkReadOnly(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.asserting;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
|
@ -101,9 +102,9 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
public Collection<Accountable> getChildResources() {
|
||||
Collection<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkReadOnly(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.asserting;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
|
@ -86,9 +87,9 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
public Collection<Accountable> getChildResources() {
|
||||
Collection<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkReadOnly(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.asserting;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
|
@ -85,9 +86,9 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
public Collection<Accountable> getChildResources() {
|
||||
Collection<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkReadOnly(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.ramonly;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -98,7 +99,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fieldToTerms);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
|
@ -299,7 +300,7 @@ public abstract class BaseBitSetTestCase<T extends BitSet> extends LuceneTestCas
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
|
|
|
@ -30,6 +30,8 @@ import java.nio.file.FileSystem;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -198,7 +200,48 @@ public final class TestUtil {
|
|||
// ok
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Checks that the provided collection is read-only.
|
||||
* @see #checkIterator(Iterator)
|
||||
*/
|
||||
public static <T> void checkReadOnly(Collection<T> coll) {
|
||||
int size = 0;
|
||||
for (Iterator<?> it = coll.iterator(); it.hasNext(); ) {
|
||||
it.next();
|
||||
size += 1;
|
||||
}
|
||||
if (size != coll.size()) {
|
||||
throw new AssertionError("broken collection, reported size is "
|
||||
+ coll.size() + " but iterator has " + size + " elements: " + coll);
|
||||
}
|
||||
|
||||
if (coll.isEmpty() == false) {
|
||||
try {
|
||||
coll.remove(coll.iterator().next());
|
||||
throw new AssertionError("broken collection (supports remove): " + coll);
|
||||
} catch (UnsupportedOperationException e) {
|
||||
// ok
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
coll.add(null);
|
||||
throw new AssertionError("broken collection (supports add): " + coll);
|
||||
} catch (UnsupportedOperationException e) {
|
||||
// ok
|
||||
}
|
||||
|
||||
try {
|
||||
coll.addAll(Collections.singleton(null));
|
||||
throw new AssertionError("broken collection (supports addAll): " + coll);
|
||||
} catch (UnsupportedOperationException e) {
|
||||
// ok
|
||||
}
|
||||
|
||||
checkIterator(coll.iterator());
|
||||
}
|
||||
|
||||
public static void syncConcurrentMerges(IndexWriter writer) {
|
||||
syncConcurrentMerges(writer.getConfig().getMergeScheduler());
|
||||
}
|
||||
|
|
|
@ -342,7 +342,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", suggesters);
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.File;
|
|||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -213,7 +214,7 @@ public class SolrSuggester implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return lookup.getChildResources();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue