LUCENE-3923: fail the build on wrong svn:eol-style

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1377702 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-08-27 14:47:19 +00:00
parent 8e48cb3157
commit 458fcb4446
24 changed files with 1718 additions and 1573 deletions

View File

@ -74,6 +74,12 @@
</pathconvert>
<fail if="validate.patternsFound">The following files contain @author tags or nocommits:${line.separator}${validate.patternsFound}</fail>
</target>
<target name="check-svn-properties">
<subant target="-check-svn-properties" inheritall="false" failonerror="true">
<fileset dir="lucene" includes="build.xml" />
</subant>
</target>
<target name="rat-sources" description="Runs rat across all sources and tests">
<sequential><subant target="rat-sources" inheritall="false" failonerror="true">
@ -256,7 +262,7 @@
</target>
<!-- Jenkins tasks -->
<target name="jenkins-hourly" depends="clean,test,validate,-jenkins-javadocs-lint,-svn-status"/>
<target name="jenkins-hourly" depends="clean,test,validate,-jenkins-javadocs-lint,-svn-status,check-svn-properties"/>
<target name="jenkins-clover">
<antcall target="-jenkins-clover">

View File

@ -1,9 +1,9 @@
<component name="libraryTable">
<library name="HSQLDB">
<CLASSES>
<root url="jar://$PROJECT_DIR$/solr/example/example-DIH/solr/db/lib/hsqldb-1.8.0.10.jar!/" />
</CLASSES>
<JAVADOC />
<SOURCES />
</library>
<component name="libraryTable">
<library name="HSQLDB">
<CLASSES>
<root url="jar://$PROJECT_DIR$/solr/example/example-DIH/solr/db/lib/hsqldb-1.8.0.10.jar!/" />
</CLASSES>
<JAVADOC />
<SOURCES />
</library>
</component>

View File

@ -198,6 +198,16 @@
</forbidden-apis>
</target>
<!-- note: we don't include this in validate because we want to check from releases -->
<target name="-check-svn-properties" depends="compile-tools,resolve,load-custom-tasks">
<svn-eol-style svnExecutable="${svn.exe}">
<fileset dir="${basedir}/..">
<exclude name="**/build/**"/>
<exclude name="**/*.jar"/>
</fileset>
</svn-eol-style>
</target>
<target name="resolve">
<sequential>
<ant dir="test-framework" target="resolve" inheritall="false">

View File

@ -1,485 +1,485 @@
package org.apache.lucene.codecs.bloom;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FuzzySet;
import org.apache.lucene.util.FuzzySet.ContainsResult;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.hash.MurmurHash2;
/**
* <p>
* A {@link PostingsFormat} useful for low doc-frequency fields such as primary
* keys. Bloom filters are maintained in a ".blm" file which offers "fast-fail"
* for reads in segments known to have no record of the key. A choice of
* delegate PostingsFormat is used to record all other Postings data.
* </p>
* <p>
* A choice of {@link BloomFilterFactory} can be passed to tailor Bloom Filter
* settings on a per-field basis. The default configuration is
* {@link DefaultBloomFilterFactory} which allocates a ~8mb bitset and hashes
* values using {@link MurmurHash2}. This should be suitable for most purposes.
* </p>
* <p>
* The format of the blm file is as follows:
* </p>
* <ul>
* <li>BloomFilter (.blm) --&gt; Header, DelegatePostingsFormatName,
* NumFilteredFields, Filter<sup>NumFilteredFields</sup></li>
* <li>Filter --&gt; FieldNumber, FuzzySet</li>
* <li>FuzzySet --&gt;See {@link FuzzySet#serialize(DataOutput)}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>DelegatePostingsFormatName --&gt; {@link DataOutput#writeString(String)
* String} The name of a ServiceProvider registered {@link PostingsFormat}</li>
* <li>NumFilteredFields --&gt; {@link DataOutput#writeInt Uint32}</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt Uint32} The number of the
* field in this segment</li>
* </ul>
* @lucene.experimental
*/
public class BloomFilteringPostingsFormat extends PostingsFormat {
public static final String BLOOM_CODEC_NAME = "BloomFilter";
public static final int BLOOM_CODEC_VERSION = 1;
/** Extension of Bloom Filters file */
static final String BLOOM_EXTENSION = "blm";
BloomFilterFactory bloomFilterFactory = new DefaultBloomFilterFactory();
private PostingsFormat delegatePostingsFormat;
/**
* Creates Bloom filters for a selection of fields created in the index. This
* is recorded as a set of Bitsets held as a segment summary in an additional
* "blm" file. This PostingsFormat delegates to a choice of delegate
* PostingsFormat for encoding all other postings data.
*
* @param delegatePostingsFormat
* The PostingsFormat that records all the non-bloom filter data i.e.
* postings info.
* @param bloomFilterFactory
* The {@link BloomFilterFactory} responsible for sizing BloomFilters
* appropriately
*/
public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat,
BloomFilterFactory bloomFilterFactory) {
super(BLOOM_CODEC_NAME);
this.delegatePostingsFormat = delegatePostingsFormat;
this.bloomFilterFactory = bloomFilterFactory;
}
/**
* Creates Bloom filters for a selection of fields created in the index. This
* is recorded as a set of Bitsets held as a segment summary in an additional
* "blm" file. This PostingsFormat delegates to a choice of delegate
* PostingsFormat for encoding all other postings data. This choice of
* constructor defaults to the {@link DefaultBloomFilterFactory} for
* configuring per-field BloomFilters.
*
* @param delegatePostingsFormat
* The PostingsFormat that records all the non-bloom filter data i.e.
* postings info.
*/
public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat) {
this(delegatePostingsFormat, new DefaultBloomFilterFactory());
}
// Used only by core Lucene at read-time via Service Provider instantiation -
// do not use at Write-time in application code.
public BloomFilteringPostingsFormat() {
super(BLOOM_CODEC_NAME);
}
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
if (delegatePostingsFormat == null) {
throw new UnsupportedOperationException("Error - " + getClass().getName()
+ " has been constructed without a choice of PostingsFormat");
}
return new BloomFilteredFieldsConsumer(
delegatePostingsFormat.fieldsConsumer(state), state,
delegatePostingsFormat);
}
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new BloomFilteredFieldsProducer(state);
}
public class BloomFilteredFieldsProducer extends FieldsProducer {
private FieldsProducer delegateFieldsProducer;
HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<String,FuzzySet>();
public BloomFilteredFieldsProducer(SegmentReadState state)
throws IOException {
String bloomFileName = IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
IndexInput bloomIn = null;
try {
bloomIn = state.dir.openInput(bloomFileName, state.context);
CodecUtil.checkHeader(bloomIn, BLOOM_CODEC_NAME, BLOOM_CODEC_VERSION,
BLOOM_CODEC_VERSION);
// // Load the hash function used in the BloomFilter
// hashFunction = HashFunction.forName(bloomIn.readString());
// Load the delegate postings format
PostingsFormat delegatePostingsFormat = PostingsFormat.forName(bloomIn
.readString());
this.delegateFieldsProducer = delegatePostingsFormat
.fieldsProducer(state);
int numBlooms = bloomIn.readInt();
for (int i = 0; i < numBlooms; i++) {
int fieldNum = bloomIn.readInt();
FuzzySet bloom = FuzzySet.deserialize(bloomIn);
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNum);
bloomsByFieldName.put(fieldInfo.name, bloom);
}
} finally {
IOUtils.close(bloomIn);
}
}
public Iterator<String> iterator() {
return delegateFieldsProducer.iterator();
}
public void close() throws IOException {
delegateFieldsProducer.close();
}
public Terms terms(String field) throws IOException {
FuzzySet filter = bloomsByFieldName.get(field);
if (filter == null) {
return delegateFieldsProducer.terms(field);
} else {
Terms result = delegateFieldsProducer.terms(field);
if (result == null) {
return null;
}
return new BloomFilteredTerms(result, filter);
}
}
public int size() {
return delegateFieldsProducer.size();
}
class BloomFilteredTerms extends Terms {
private Terms delegateTerms;
private FuzzySet filter;
public BloomFilteredTerms(Terms terms, FuzzySet filter) {
this.delegateTerms = terms;
this.filter = filter;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled,
final BytesRef startTerm) throws IOException {
return delegateTerms.intersect(compiled, startTerm);
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
TermsEnum result;
if ((reuse != null) && (reuse instanceof BloomFilteredTermsEnum)) {
// recycle the existing BloomFilteredTermsEnum by asking the delegate
// to recycle its contained TermsEnum
BloomFilteredTermsEnum bfte = (BloomFilteredTermsEnum) reuse;
if (bfte.filter == filter) {
bfte.delegateTermsEnum = delegateTerms
.iterator(bfte.delegateTermsEnum);
return bfte;
}
}
// We have been handed something we cannot reuse (either null, wrong
// class or wrong filter) so allocate a new object
result = new BloomFilteredTermsEnum(delegateTerms.iterator(reuse),
filter);
return result;
}
@Override
public Comparator<BytesRef> getComparator() throws IOException {
return delegateTerms.getComparator();
}
@Override
public long size() throws IOException {
return delegateTerms.size();
}
@Override
public long getSumTotalTermFreq() throws IOException {
return delegateTerms.getSumTotalTermFreq();
}
@Override
public long getSumDocFreq() throws IOException {
return delegateTerms.getSumDocFreq();
}
@Override
public int getDocCount() throws IOException {
return delegateTerms.getDocCount();
}
@Override
public boolean hasOffsets() {
return delegateTerms.hasOffsets();
}
@Override
public boolean hasPositions() {
return delegateTerms.hasPositions();
}
@Override
public boolean hasPayloads() {
return delegateTerms.hasPayloads();
}
}
class BloomFilteredTermsEnum extends TermsEnum {
TermsEnum delegateTermsEnum;
private FuzzySet filter;
public BloomFilteredTermsEnum(TermsEnum iterator, FuzzySet filter) {
this.delegateTermsEnum = iterator;
this.filter = filter;
}
@Override
public final BytesRef next() throws IOException {
return delegateTermsEnum.next();
}
@Override
public final Comparator<BytesRef> getComparator() {
return delegateTermsEnum.getComparator();
}
@Override
public final boolean seekExact(BytesRef text, boolean useCache)
throws IOException {
// The magical fail-fast speed up that is the entire point of all of
// this code - save a disk seek if there is a match on an in-memory
// structure
// that may occasionally give a false positive but guaranteed no false
// negatives
if (filter.contains(text) == ContainsResult.NO) {
return false;
}
return delegateTermsEnum.seekExact(text, useCache);
}
@Override
public final SeekStatus seekCeil(BytesRef text, boolean useCache)
throws IOException {
return delegateTermsEnum.seekCeil(text, useCache);
}
@Override
public final void seekExact(long ord) throws IOException {
delegateTermsEnum.seekExact(ord);
}
@Override
public final BytesRef term() throws IOException {
return delegateTermsEnum.term();
}
@Override
public final long ord() throws IOException {
return delegateTermsEnum.ord();
}
@Override
public final int docFreq() throws IOException {
return delegateTermsEnum.docFreq();
}
@Override
public final long totalTermFreq() throws IOException {
return delegateTermsEnum.totalTermFreq();
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
DocsAndPositionsEnum reuse, int flags) throws IOException {
return delegateTermsEnum.docsAndPositions(liveDocs, reuse, flags);
}
@Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
throws IOException {
return delegateTermsEnum.docs(liveDocs, reuse, flags);
}
}
}
class BloomFilteredFieldsConsumer extends FieldsConsumer {
private FieldsConsumer delegateFieldsConsumer;
private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
private SegmentWriteState state;
// private PostingsFormat delegatePostingsFormat;
public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
this.delegateFieldsConsumer = fieldsConsumer;
// this.delegatePostingsFormat=delegatePostingsFormat;
this.state = state;
}
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
FuzzySet bloomFilter = bloomFilterFactory.getSetForField(state,field);
if (bloomFilter != null) {
assert bloomFilters.containsKey(field) == false;
bloomFilters.put(field, bloomFilter);
return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field),bloomFilter);
} else {
// No, use the unfiltered fieldsConsumer - we are not interested in
// recording any term Bitsets.
return delegateFieldsConsumer.addField(field);
}
}
@Override
public void close() throws IOException {
delegateFieldsConsumer.close();
// Now we are done accumulating values for these fields
List<Entry<FieldInfo,FuzzySet>> nonSaturatedBlooms = new ArrayList<Map.Entry<FieldInfo,FuzzySet>>();
for (Entry<FieldInfo,FuzzySet> entry : bloomFilters.entrySet()) {
FuzzySet bloomFilter = entry.getValue();
if(!bloomFilterFactory.isSaturated(bloomFilter,entry.getKey())){
nonSaturatedBlooms.add(entry);
}
}
String bloomFileName = IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
IndexOutput bloomOutput = null;
try {
bloomOutput = state.directory
.createOutput(bloomFileName, state.context);
CodecUtil.writeHeader(bloomOutput, BLOOM_CODEC_NAME,
BLOOM_CODEC_VERSION);
// remember the name of the postings format we will delegate to
bloomOutput.writeString(delegatePostingsFormat.getName());
// First field in the output file is the number of fields+blooms saved
bloomOutput.writeInt(nonSaturatedBlooms.size());
for (Entry<FieldInfo,FuzzySet> entry : nonSaturatedBlooms) {
FieldInfo fieldInfo = entry.getKey();
FuzzySet bloomFilter = entry.getValue();
bloomOutput.writeInt(fieldInfo.number);
saveAppropriatelySizedBloomFilter(bloomOutput, bloomFilter, fieldInfo);
}
} finally {
IOUtils.close(bloomOutput);
}
//We are done with large bitsets so no need to keep them hanging around
bloomFilters.clear();
}
private void saveAppropriatelySizedBloomFilter(IndexOutput bloomOutput,
FuzzySet bloomFilter, FieldInfo fieldInfo) throws IOException {
FuzzySet rightSizedSet = bloomFilterFactory.downsize(fieldInfo,
bloomFilter);
if (rightSizedSet == null) {
rightSizedSet = bloomFilter;
}
rightSizedSet.serialize(bloomOutput);
}
}
class WrappedTermsConsumer extends TermsConsumer {
private TermsConsumer delegateTermsConsumer;
private FuzzySet bloomFilter;
public WrappedTermsConsumer(TermsConsumer termsConsumer,FuzzySet bloomFilter) {
this.delegateTermsConsumer = termsConsumer;
this.bloomFilter = bloomFilter;
}
public PostingsConsumer startTerm(BytesRef text) throws IOException {
return delegateTermsConsumer.startTerm(text);
}
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
// Record this term in our BloomFilter
if (stats.docFreq > 0) {
bloomFilter.addValue(text);
}
delegateTermsConsumer.finishTerm(text, stats);
}
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
throws IOException {
delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
}
public Comparator<BytesRef> getComparator() throws IOException {
return delegateTermsConsumer.getComparator();
}
}
}
package org.apache.lucene.codecs.bloom;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FuzzySet;
import org.apache.lucene.util.FuzzySet.ContainsResult;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.hash.MurmurHash2;
/**
* <p>
* A {@link PostingsFormat} useful for low doc-frequency fields such as primary
* keys. Bloom filters are maintained in a ".blm" file which offers "fast-fail"
* for reads in segments known to have no record of the key. A choice of
* delegate PostingsFormat is used to record all other Postings data.
* </p>
* <p>
* A choice of {@link BloomFilterFactory} can be passed to tailor Bloom Filter
* settings on a per-field basis. The default configuration is
* {@link DefaultBloomFilterFactory} which allocates a ~8mb bitset and hashes
* values using {@link MurmurHash2}. This should be suitable for most purposes.
* </p>
* <p>
* The format of the blm file is as follows:
* </p>
* <ul>
* <li>BloomFilter (.blm) --&gt; Header, DelegatePostingsFormatName,
* NumFilteredFields, Filter<sup>NumFilteredFields</sup></li>
* <li>Filter --&gt; FieldNumber, FuzzySet</li>
* <li>FuzzySet --&gt;See {@link FuzzySet#serialize(DataOutput)}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>DelegatePostingsFormatName --&gt; {@link DataOutput#writeString(String)
* String} The name of a ServiceProvider registered {@link PostingsFormat}</li>
* <li>NumFilteredFields --&gt; {@link DataOutput#writeInt Uint32}</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt Uint32} The number of the
* field in this segment</li>
* </ul>
* @lucene.experimental
*/
public class BloomFilteringPostingsFormat extends PostingsFormat {
public static final String BLOOM_CODEC_NAME = "BloomFilter";
public static final int BLOOM_CODEC_VERSION = 1;
/** Extension of Bloom Filters file */
static final String BLOOM_EXTENSION = "blm";
BloomFilterFactory bloomFilterFactory = new DefaultBloomFilterFactory();
private PostingsFormat delegatePostingsFormat;
/**
* Creates Bloom filters for a selection of fields created in the index. This
* is recorded as a set of Bitsets held as a segment summary in an additional
* "blm" file. This PostingsFormat delegates to a choice of delegate
* PostingsFormat for encoding all other postings data.
*
* @param delegatePostingsFormat
* The PostingsFormat that records all the non-bloom filter data i.e.
* postings info.
* @param bloomFilterFactory
* The {@link BloomFilterFactory} responsible for sizing BloomFilters
* appropriately
*/
public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat,
BloomFilterFactory bloomFilterFactory) {
super(BLOOM_CODEC_NAME);
this.delegatePostingsFormat = delegatePostingsFormat;
this.bloomFilterFactory = bloomFilterFactory;
}
/**
* Creates Bloom filters for a selection of fields created in the index. This
* is recorded as a set of Bitsets held as a segment summary in an additional
* "blm" file. This PostingsFormat delegates to a choice of delegate
* PostingsFormat for encoding all other postings data. This choice of
* constructor defaults to the {@link DefaultBloomFilterFactory} for
* configuring per-field BloomFilters.
*
* @param delegatePostingsFormat
* The PostingsFormat that records all the non-bloom filter data i.e.
* postings info.
*/
public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat) {
this(delegatePostingsFormat, new DefaultBloomFilterFactory());
}
// Used only by core Lucene at read-time via Service Provider instantiation -
// do not use at Write-time in application code.
public BloomFilteringPostingsFormat() {
super(BLOOM_CODEC_NAME);
}
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
if (delegatePostingsFormat == null) {
throw new UnsupportedOperationException("Error - " + getClass().getName()
+ " has been constructed without a choice of PostingsFormat");
}
return new BloomFilteredFieldsConsumer(
delegatePostingsFormat.fieldsConsumer(state), state,
delegatePostingsFormat);
}
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new BloomFilteredFieldsProducer(state);
}
public class BloomFilteredFieldsProducer extends FieldsProducer {
private FieldsProducer delegateFieldsProducer;
HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<String,FuzzySet>();
public BloomFilteredFieldsProducer(SegmentReadState state)
throws IOException {
String bloomFileName = IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
IndexInput bloomIn = null;
try {
bloomIn = state.dir.openInput(bloomFileName, state.context);
CodecUtil.checkHeader(bloomIn, BLOOM_CODEC_NAME, BLOOM_CODEC_VERSION,
BLOOM_CODEC_VERSION);
// // Load the hash function used in the BloomFilter
// hashFunction = HashFunction.forName(bloomIn.readString());
// Load the delegate postings format
PostingsFormat delegatePostingsFormat = PostingsFormat.forName(bloomIn
.readString());
this.delegateFieldsProducer = delegatePostingsFormat
.fieldsProducer(state);
int numBlooms = bloomIn.readInt();
for (int i = 0; i < numBlooms; i++) {
int fieldNum = bloomIn.readInt();
FuzzySet bloom = FuzzySet.deserialize(bloomIn);
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNum);
bloomsByFieldName.put(fieldInfo.name, bloom);
}
} finally {
IOUtils.close(bloomIn);
}
}
public Iterator<String> iterator() {
return delegateFieldsProducer.iterator();
}
public void close() throws IOException {
delegateFieldsProducer.close();
}
public Terms terms(String field) throws IOException {
FuzzySet filter = bloomsByFieldName.get(field);
if (filter == null) {
return delegateFieldsProducer.terms(field);
} else {
Terms result = delegateFieldsProducer.terms(field);
if (result == null) {
return null;
}
return new BloomFilteredTerms(result, filter);
}
}
public int size() {
return delegateFieldsProducer.size();
}
class BloomFilteredTerms extends Terms {
private Terms delegateTerms;
private FuzzySet filter;
public BloomFilteredTerms(Terms terms, FuzzySet filter) {
this.delegateTerms = terms;
this.filter = filter;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled,
final BytesRef startTerm) throws IOException {
return delegateTerms.intersect(compiled, startTerm);
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
TermsEnum result;
if ((reuse != null) && (reuse instanceof BloomFilteredTermsEnum)) {
// recycle the existing BloomFilteredTermsEnum by asking the delegate
// to recycle its contained TermsEnum
BloomFilteredTermsEnum bfte = (BloomFilteredTermsEnum) reuse;
if (bfte.filter == filter) {
bfte.delegateTermsEnum = delegateTerms
.iterator(bfte.delegateTermsEnum);
return bfte;
}
}
// We have been handed something we cannot reuse (either null, wrong
// class or wrong filter) so allocate a new object
result = new BloomFilteredTermsEnum(delegateTerms.iterator(reuse),
filter);
return result;
}
@Override
public Comparator<BytesRef> getComparator() throws IOException {
return delegateTerms.getComparator();
}
@Override
public long size() throws IOException {
return delegateTerms.size();
}
@Override
public long getSumTotalTermFreq() throws IOException {
return delegateTerms.getSumTotalTermFreq();
}
@Override
public long getSumDocFreq() throws IOException {
return delegateTerms.getSumDocFreq();
}
@Override
public int getDocCount() throws IOException {
return delegateTerms.getDocCount();
}
@Override
public boolean hasOffsets() {
return delegateTerms.hasOffsets();
}
@Override
public boolean hasPositions() {
return delegateTerms.hasPositions();
}
@Override
public boolean hasPayloads() {
return delegateTerms.hasPayloads();
}
}
class BloomFilteredTermsEnum extends TermsEnum {
TermsEnum delegateTermsEnum;
private FuzzySet filter;
public BloomFilteredTermsEnum(TermsEnum iterator, FuzzySet filter) {
this.delegateTermsEnum = iterator;
this.filter = filter;
}
@Override
public final BytesRef next() throws IOException {
return delegateTermsEnum.next();
}
@Override
public final Comparator<BytesRef> getComparator() {
return delegateTermsEnum.getComparator();
}
@Override
public final boolean seekExact(BytesRef text, boolean useCache)
throws IOException {
// The magical fail-fast speed up that is the entire point of all of
// this code - save a disk seek if there is a match on an in-memory
// structure
// that may occasionally give a false positive but guaranteed no false
// negatives
if (filter.contains(text) == ContainsResult.NO) {
return false;
}
return delegateTermsEnum.seekExact(text, useCache);
}
@Override
public final SeekStatus seekCeil(BytesRef text, boolean useCache)
throws IOException {
return delegateTermsEnum.seekCeil(text, useCache);
}
@Override
public final void seekExact(long ord) throws IOException {
delegateTermsEnum.seekExact(ord);
}
@Override
public final BytesRef term() throws IOException {
return delegateTermsEnum.term();
}
@Override
public final long ord() throws IOException {
return delegateTermsEnum.ord();
}
@Override
public final int docFreq() throws IOException {
return delegateTermsEnum.docFreq();
}
@Override
public final long totalTermFreq() throws IOException {
return delegateTermsEnum.totalTermFreq();
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
DocsAndPositionsEnum reuse, int flags) throws IOException {
return delegateTermsEnum.docsAndPositions(liveDocs, reuse, flags);
}
@Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
throws IOException {
return delegateTermsEnum.docs(liveDocs, reuse, flags);
}
}
}
class BloomFilteredFieldsConsumer extends FieldsConsumer {
private FieldsConsumer delegateFieldsConsumer;
private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
private SegmentWriteState state;
// private PostingsFormat delegatePostingsFormat;
public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
this.delegateFieldsConsumer = fieldsConsumer;
// this.delegatePostingsFormat=delegatePostingsFormat;
this.state = state;
}
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
FuzzySet bloomFilter = bloomFilterFactory.getSetForField(state,field);
if (bloomFilter != null) {
assert bloomFilters.containsKey(field) == false;
bloomFilters.put(field, bloomFilter);
return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field),bloomFilter);
} else {
// No, use the unfiltered fieldsConsumer - we are not interested in
// recording any term Bitsets.
return delegateFieldsConsumer.addField(field);
}
}
@Override
public void close() throws IOException {
delegateFieldsConsumer.close();
// Now we are done accumulating values for these fields
List<Entry<FieldInfo,FuzzySet>> nonSaturatedBlooms = new ArrayList<Map.Entry<FieldInfo,FuzzySet>>();
for (Entry<FieldInfo,FuzzySet> entry : bloomFilters.entrySet()) {
FuzzySet bloomFilter = entry.getValue();
if(!bloomFilterFactory.isSaturated(bloomFilter,entry.getKey())){
nonSaturatedBlooms.add(entry);
}
}
String bloomFileName = IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
IndexOutput bloomOutput = null;
try {
bloomOutput = state.directory
.createOutput(bloomFileName, state.context);
CodecUtil.writeHeader(bloomOutput, BLOOM_CODEC_NAME,
BLOOM_CODEC_VERSION);
// remember the name of the postings format we will delegate to
bloomOutput.writeString(delegatePostingsFormat.getName());
// First field in the output file is the number of fields+blooms saved
bloomOutput.writeInt(nonSaturatedBlooms.size());
for (Entry<FieldInfo,FuzzySet> entry : nonSaturatedBlooms) {
FieldInfo fieldInfo = entry.getKey();
FuzzySet bloomFilter = entry.getValue();
bloomOutput.writeInt(fieldInfo.number);
saveAppropriatelySizedBloomFilter(bloomOutput, bloomFilter, fieldInfo);
}
} finally {
IOUtils.close(bloomOutput);
}
//We are done with large bitsets so no need to keep them hanging around
bloomFilters.clear();
}
private void saveAppropriatelySizedBloomFilter(IndexOutput bloomOutput,
FuzzySet bloomFilter, FieldInfo fieldInfo) throws IOException {
FuzzySet rightSizedSet = bloomFilterFactory.downsize(fieldInfo,
bloomFilter);
if (rightSizedSet == null) {
rightSizedSet = bloomFilter;
}
rightSizedSet.serialize(bloomOutput);
}
}
class WrappedTermsConsumer extends TermsConsumer {
private TermsConsumer delegateTermsConsumer;
private FuzzySet bloomFilter;
public WrappedTermsConsumer(TermsConsumer termsConsumer,FuzzySet bloomFilter) {
this.delegateTermsConsumer = termsConsumer;
this.bloomFilter = bloomFilter;
}
public PostingsConsumer startTerm(BytesRef text) throws IOException {
return delegateTermsConsumer.startTerm(text);
}
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
// Record this term in our BloomFilter
if (stats.docFreq > 0) {
bloomFilter.addValue(text);
}
delegateTermsConsumer.finishTerm(text, stats);
}
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
throws IOException {
delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
}
public Comparator<BytesRef> getComparator() throws IOException {
return delegateTermsConsumer.getComparator();
}
}
}

View File

@ -1,25 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codec PostingsFormat for fast access to low-frequency terms such as primary key fields.
</body>
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codec PostingsFormat for fast access to low-frequency terms such as primary key fields.
</body>
</html>

View File

@ -1,84 +1,84 @@
package org.apache.lucene.util.hash;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.util.Set;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NamedSPILoader;
/**
* Base class for hashing functions that can be referred to by name.
* Subclasses are expected to provide threadsafe implementations of the hash function
* on the range of bytes referenced in the provided {@link BytesRef}
* @lucene.experimental
*/
public abstract class HashFunction implements NamedSPILoader.NamedSPI {
/**
* Hashes the contents of the referenced bytes
* @param bytes the data to be hashed
* @return the hash of the bytes referenced by bytes.offset and length bytes.length
*/
public abstract int hash(BytesRef bytes);
private static final NamedSPILoader<HashFunction> loader =
new NamedSPILoader<HashFunction>(HashFunction.class);
private final String name;
public HashFunction(String name) {
NamedSPILoader.checkServiceName(name);
this.name = name;
}
/** Returns this codec's name */
@Override
public final String getName() {
return name;
}
/** looks up a hash function by name */
public static HashFunction forName(String name) {
return loader.lookup(name);
}
/** returns a list of all available hash function names */
public static Set<String> availableHashFunctionNames() {
return loader.availableServices();
}
/**
* Reloads the hash function list from the given {@link ClassLoader}.
* Changes to the function list are visible after the method ends, all
* iterators ({@link #availableHashFunctionNames()},...) stay consistent.
*
* <p><b>NOTE:</b> Only new functions are added, existing ones are
* never removed or replaced.
*
* <p><em>This method is expensive and should only be called for discovery
* of new functions on the given classpath/classloader!</em>
*/
public static void reloadHashFunctions(ClassLoader classloader) {
loader.reload(classloader);
}
@Override
public String toString() {
return name;
}
}
package org.apache.lucene.util.hash;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.util.Set;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NamedSPILoader;
/**
* Base class for hashing functions that can be referred to by name.
* Subclasses are expected to provide threadsafe implementations of the hash function
* on the range of bytes referenced in the provided {@link BytesRef}
* @lucene.experimental
*/
public abstract class HashFunction implements NamedSPILoader.NamedSPI {
/**
* Hashes the contents of the referenced bytes
* @param bytes the data to be hashed
* @return the hash of the bytes referenced by bytes.offset and length bytes.length
*/
public abstract int hash(BytesRef bytes);
private static final NamedSPILoader<HashFunction> loader =
new NamedSPILoader<HashFunction>(HashFunction.class);
private final String name;
public HashFunction(String name) {
NamedSPILoader.checkServiceName(name);
this.name = name;
}
/** Returns this codec's name */
@Override
public final String getName() {
return name;
}
/** looks up a hash function by name */
public static HashFunction forName(String name) {
return loader.lookup(name);
}
/** returns a list of all available hash function names */
public static Set<String> availableHashFunctionNames() {
return loader.availableServices();
}
/**
* Reloads the hash function list from the given {@link ClassLoader}.
* Changes to the function list are visible after the method ends, all
* iterators ({@link #availableHashFunctionNames()},...) stay consistent.
*
* <p><b>NOTE:</b> Only new functions are added, existing ones are
* never removed or replaced.
*
* <p><em>This method is expensive and should only be called for discovery
* of new functions on the given classpath/classloader!</em>
*/
public static void reloadHashFunctions(ClassLoader classloader) {
loader.reload(classloader);
}
@Override
public String toString() {
return name;
}
}

View File

@ -1,105 +1,105 @@
package org.apache.lucene.util.hash;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
/**
* This is a very fast, non-cryptographic hash suitable for general hash-based
* lookup. See http://murmurhash.googlepages.com/ for more details.
* <p>
* The C version of MurmurHash 2.0 found at that site was ported to Java by
* Andrzej Bialecki (ab at getopt org).
* </p>
* <p>
* The code from getopt.org was adapted by Mark Harwood in the form here as one of a pluggable choice of
* hashing functions as the core function had to be adapted to work with BytesRefs with offsets and lengths
* rather than raw byte arrays.
* </p>
* @lucene.experimental
*/
public class MurmurHash2 extends HashFunction{
public static final String HASH_NAME="MurmurHash2";
public MurmurHash2() {
super(HASH_NAME);
}
public static int hash(byte[] data, int seed, int offset, int len) {
int m = 0x5bd1e995;
int r = 24;
int h = seed ^ len;
int len_4 = len >> 2;
for (int i = 0; i < len_4; i++) {
int i_4 = offset + (i << 2);
int k = data[i_4 + 3];
k = k << 8;
k = k | (data[i_4 + 2] & 0xff);
k = k << 8;
k = k | (data[i_4 + 1] & 0xff);
k = k << 8;
k = k | (data[i_4 + 0] & 0xff);
k *= m;
k ^= k >>> r;
k *= m;
h *= m;
h ^= k;
}
int len_m = len_4 << 2;
int left = len - len_m;
if (left != 0) {
if (left >= 3) {
h ^= data[offset + len - 3] << 16;
}
if (left >= 2) {
h ^= data[offset + len - 2] << 8;
}
if (left >= 1) {
h ^= data[offset + len - 1];
}
h *= m;
}
h ^= h >>> 13;
h *= m;
h ^= h >>> 15;
return h;
}
/**
* Generates 32 bit hash from byte array with default seed value.
*
* @param data
* byte array to hash
* @param offset
* the start position in the array to hash
* @param len
* length of the array elements to hash
* @return 32 bit hash of the given array
*/
public static final int hash32(final byte[] data, int offset, int len) {
return MurmurHash2.hash(data, 0x9747b28c, offset, len);
}
@Override
public final int hash(BytesRef br) {
return hash32(br.bytes, br.offset, br.length);
}
}
package org.apache.lucene.util.hash;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
/**
* This is a very fast, non-cryptographic hash suitable for general hash-based
* lookup. See http://murmurhash.googlepages.com/ for more details.
* <p>
* The C version of MurmurHash 2.0 found at that site was ported to Java by
* Andrzej Bialecki (ab at getopt org).
* </p>
* <p>
* The code from getopt.org was adapted by Mark Harwood in the form here as one of a pluggable choice of
* hashing functions as the core function had to be adapted to work with BytesRefs with offsets and lengths
* rather than raw byte arrays.
* </p>
* @lucene.experimental
*/
public class MurmurHash2 extends HashFunction{
public static final String HASH_NAME="MurmurHash2";
public MurmurHash2() {
super(HASH_NAME);
}
public static int hash(byte[] data, int seed, int offset, int len) {
int m = 0x5bd1e995;
int r = 24;
int h = seed ^ len;
int len_4 = len >> 2;
for (int i = 0; i < len_4; i++) {
int i_4 = offset + (i << 2);
int k = data[i_4 + 3];
k = k << 8;
k = k | (data[i_4 + 2] & 0xff);
k = k << 8;
k = k | (data[i_4 + 1] & 0xff);
k = k << 8;
k = k | (data[i_4 + 0] & 0xff);
k *= m;
k ^= k >>> r;
k *= m;
h *= m;
h ^= k;
}
int len_m = len_4 << 2;
int left = len - len_m;
if (left != 0) {
if (left >= 3) {
h ^= data[offset + len - 3] << 16;
}
if (left >= 2) {
h ^= data[offset + len - 2] << 8;
}
if (left >= 1) {
h ^= data[offset + len - 1];
}
h *= m;
}
h ^= h >>> 13;
h *= m;
h ^= h >>> 15;
return h;
}
/**
* Generates 32 bit hash from byte array with default seed value.
*
* @param data
* byte array to hash
* @param offset
* the start position in the array to hash
* @param len
* length of the array elements to hash
* @return 32 bit hash of the given array
*/
public static final int hash32(final byte[] data, int offset, int len) {
return MurmurHash2.hash(data, 0x9747b28c, offset, len);
}
@Override
public final int hash(BytesRef br) {
return hash32(br.bytes, br.offset, br.length);
}
}

View File

@ -1,25 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Hashing functions load-able via SPI service
</body>
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Hashing functions load-able via SPI service
</body>
</html>

View File

@ -1,16 +1,16 @@
# 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.
org.apache.lucene.util.hash.MurmurHash2
# 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.
org.apache.lucene.util.hash.MurmurHash2

View File

@ -1,24 +1,24 @@
/*
* 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.
*/
/**
* Bounding Box Spatial Strategy
*
* Index a shape extent using 4 numeric fields and a flag to say if it crosses the dateline
*/
package org.apache.lucene.spatial.bbox;
/*
* 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.
*/
/**
* Bounding Box Spatial Strategy
*
* Index a shape extent using 4 numeric fields and a flag to say if it crosses the dateline
*/
package org.apache.lucene.spatial.bbox;

View File

@ -1,5 +1,5 @@
#id name shape
C5 CenterAt5 -5 -5 5 5
C10 CenterAt10 -10 -10 10 10
NW15 NorthWest 15 15 20 20
#id name shape
C5 CenterAt5 -5 -5 5 5
C10 CenterAt10 -10 -10 10 10
NW15 NorthWest 15 15 20 20

View File

@ -1,13 +1,13 @@
C5 @ IsWithin(-6 -6 6 6)
C5 @ BBoxWithin(-6 -6 6 6)
C10 @ Contains(-6 -6 6 6)
C10 @ IsEqualTo(-10 -10 10 10)
C5 C10 @ Intersects(-2 -2 2 2)
C5 C10 @ Overlaps(-2 -2 2 2)
C5 C10 @ BBoxIntersects(-2 -2 2 2)
NW15 @ IsDisjointTo(-10 -10 10 10)
C5 @ IsWithin(-6 -6 6 6)
C5 @ BBoxWithin(-6 -6 6 6)
C10 @ Contains(-6 -6 6 6)
C10 @ IsEqualTo(-10 -10 10 10)
C5 C10 @ Intersects(-2 -2 2 2)
C5 C10 @ Overlaps(-2 -2 2 2)
C5 C10 @ BBoxIntersects(-2 -2 2 2)
NW15 @ IsDisjointTo(-10 -10 10 10)

View File

@ -1,60 +1,60 @@
package org.apache.lucene.spatial.bbox;
/*
* 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 com.spatial4j.core.context.simple.SimpleSpatialContext;
import org.apache.lucene.spatial.SpatialMatchConcern;
import org.apache.lucene.spatial.StrategyTestCase;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
public class TestBBoxStrategy extends StrategyTestCase {
@Before
@Override
public void setUp() throws Exception {
super.setUp();
this.ctx = SimpleSpatialContext.GEO_KM;
this.strategy = new BBoxStrategy(ctx, "bbox");
}
@Test
public void testBasicOperaions() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_SIMPLE_BBOX);
executeQueries(SpatialMatchConcern.EXACT, QTEST_Simple_Queries_BBox);
}
@Test
public void testStatesBBox() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_STATES_BBOX);
executeQueries(SpatialMatchConcern.FILTER, QTEST_States_IsWithin_BBox);
executeQueries(SpatialMatchConcern.FILTER, QTEST_States_Intersects_BBox);
}
@Test
public void testCitiesWithinBBox() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_IsWithin_BBox);
}
}
package org.apache.lucene.spatial.bbox;
/*
* 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 com.spatial4j.core.context.simple.SimpleSpatialContext;
import org.apache.lucene.spatial.SpatialMatchConcern;
import org.apache.lucene.spatial.StrategyTestCase;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
public class TestBBoxStrategy extends StrategyTestCase {
@Before
@Override
public void setUp() throws Exception {
super.setUp();
this.ctx = SimpleSpatialContext.GEO_KM;
this.strategy = new BBoxStrategy(ctx, "bbox");
}
@Test
public void testBasicOperaions() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_SIMPLE_BBOX);
executeQueries(SpatialMatchConcern.EXACT, QTEST_Simple_Queries_BBox);
}
@Test
public void testStatesBBox() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_STATES_BBOX);
executeQueries(SpatialMatchConcern.FILTER, QTEST_States_IsWithin_BBox);
executeQueries(SpatialMatchConcern.FILTER, QTEST_States_Intersects_BBox);
}
@Test
public void testCitiesWithinBBox() throws IOException {
getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_IsWithin_BBox);
}
}

View File

@ -1,77 +1,77 @@
package org.apache.lucene.codecs.bloom;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.FuzzySet;
import org.apache.lucene.util.hash.MurmurHash2;
/**
* A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
* delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
* amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
* APPLICATION This is not a realistic application of Bloom Filters as they
* ordinarily are larger and operate on only primary key type fields.
*/
public class TestBloomFilteredLucene40Postings extends PostingsFormat {
private BloomFilteringPostingsFormat delegate;
// Special class used to avoid OOM exceptions where Junit tests create many
// fields.
static class LowMemoryBloomFactory extends BloomFilterFactory {
@Override
public FuzzySet getSetForField(SegmentWriteState state,FieldInfo info) {
return FuzzySet.createSetBasedOnMaxMemory(1024, new MurmurHash2());
}
@Override
public boolean isSaturated(FuzzySet bloomFilter, FieldInfo fieldInfo) {
// For test purposes always maintain the BloomFilter - even past the point
// of usefulness when all bits are set
return false;
}
}
public TestBloomFilteredLucene40Postings() {
super("TestBloomFilteredLucene40Postings");
delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
new LowMemoryBloomFactory());
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
return delegate.fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return delegate.fieldsProducer(state);
}
}
package org.apache.lucene.codecs.bloom;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.FuzzySet;
import org.apache.lucene.util.hash.MurmurHash2;
/**
* A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
* delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
* amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
* APPLICATION This is not a realistic application of Bloom Filters as they
* ordinarily are larger and operate on only primary key type fields.
*/
public class TestBloomFilteredLucene40Postings extends PostingsFormat {
private BloomFilteringPostingsFormat delegate;
// Special class used to avoid OOM exceptions where Junit tests create many
// fields.
static class LowMemoryBloomFactory extends BloomFilterFactory {
@Override
public FuzzySet getSetForField(SegmentWriteState state,FieldInfo info) {
return FuzzySet.createSetBasedOnMaxMemory(1024, new MurmurHash2());
}
@Override
public boolean isSaturated(FuzzySet bloomFilter, FieldInfo fieldInfo) {
// For test purposes always maintain the BloomFilter - even past the point
// of usefulness when all bits are set
return false;
}
}
public TestBloomFilteredLucene40Postings() {
super("TestBloomFilteredLucene40Postings");
delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
new LowMemoryBloomFactory());
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
return delegate.fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return delegate.fieldsProducer(state);
}
}

View File

@ -1,25 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for generating test indexes using the BloomFilteringPostingsFormat
</body>
</html>
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for generating test indexes using the BloomFilteringPostingsFormat
</body>
</html>

View File

@ -21,4 +21,7 @@
<taskdef
name="forbidden-apis"
classname="org.apache.lucene.validation.ForbiddenApisCheckTask" />
<taskdef
name="svn-eol-style"
classname="org.apache.lucene.validation.SVNEolCheckTask" />
</antlib>

View File

@ -0,0 +1,126 @@
package org.apache.lucene.validation;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.tools.ant.BuildException;
import org.apache.tools.ant.Task;
import org.apache.tools.ant.types.Resource;
import org.apache.tools.ant.types.ResourceCollection;
import org.apache.tools.ant.types.resources.FileResource;
import org.apache.tools.ant.types.resources.Resources;
/**
* Checks all files to ensure they have svn:eol-style, or
* have a binary svn:mime-type.
* <p>
* TODO: check that this value is actually correct, not just present.
* <p>
* WARNING: slow!
*/
public class SVNEolCheckTask extends Task {
private final Resources files = new Resources();
private String svnExecutable;
/** Set of files to check */
public void add(ResourceCollection rc) {
files.add(rc);
}
/** svn.exe executable */
public void setSvnExecutable(String svnExecutable) {
this.svnExecutable = svnExecutable;
}
@Override
public void execute() throws BuildException {
if (svnExecutable == null) {
throw new BuildException("svnExecutable parameter must be set!");
}
boolean success = true;
files.setProject(getProject());
Iterator<Resource> iter = (Iterator<Resource>) files.iterator();
while (iter.hasNext()) {
Resource r = iter.next();
if (!(r instanceof FileResource)) {
throw new BuildException("Only filesystem resource are supported: " + r.getName()
+ ", was: " + r.getClass().getName());
}
File f = ((FileResource) r).getFile();
List<String> cmd = new ArrayList<String>();
cmd.add(svnExecutable);
cmd.add("pget");
cmd.add("svn:eol-style");
cmd.add(f.getAbsolutePath());
String eolStyle = exec(cmd);
if (eolStyle.isEmpty()) {
cmd.clear();
cmd.add(svnExecutable);
cmd.add("pget");
cmd.add("svn:mime-type");
cmd.add(f.getAbsolutePath());
String binProp = exec(cmd);
if (!binProp.startsWith("application/") && !binProp.startsWith("image/")) {
success = false;
log(r.getName() + " missing svn:eol-style (or binary svn:mime-type).");
}
}
}
if (!success) {
throw new BuildException("Some svn properties are missing");
}
}
private String exec(List<String> cmd) throws BuildException {
ProcessBuilder pb = new ProcessBuilder(cmd);
pb.redirectErrorStream(true);
BufferedReader r = null;
StringBuilder sb = new StringBuilder();
try {
Process p = pb.start();
InputStream is = p.getInputStream();
r = new BufferedReader(new InputStreamReader(is, Charset.defaultCharset()));
int ch;
while ((ch = r.read()) > 0) {
sb.append((char)ch);
}
p.waitFor();
return sb.toString();
} catch (Exception e) {
throw new BuildException(e);
} finally {
if (r != null) {
try {
r.close();
} catch (IOException e) {}
}
}
}
}

View File

@ -1,154 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* FIFO Circular List.
*
* Once the size is reached, it will overwrite previous entries
*
*/
public class CircularList<T> implements Iterable<T>
{
private T[] data;
private int head=0;
private int tail=0;
private int size=0;
@SuppressWarnings("unchecked")
public CircularList(int size) {
data = (T[])new Object[size];
}
@SuppressWarnings("unchecked")
public synchronized void resize(int newsize) {
if(newsize==this.size) return;
T[] vals = (T[])new Object[newsize];
int i = 0;
if(newsize>size) {
for(i=0; i<size; i++) {
vals[i] = data[convert(i)];
}
}
else {
int off=size-newsize;
for(i=0; i<newsize; i++) {
vals[i] = data[convert(i+off)];
}
}
data = vals;
head = 0;
tail = i;
}
private int convert(int index) {
return (index + head) % data.length;
}
public boolean isEmpty() {
return head == tail; // or size == 0
}
public int size() {
return size;
}
public int getBufferSize() {
return data.length;
}
private void checkIndex(int index) {
if (index >= size || index < 0)
throw new IndexOutOfBoundsException("Index: "+index+", Size: "+size);
}
public T get(int index) {
checkIndex(index);
return data[convert(index)];
}
public synchronized void add(T o) {
data[tail] = o;
tail = (tail+1)%data.length;
if( size == data.length ) {
head = (head+1)%data.length;
}
size++;
if( size > data.length ) {
size = data.length;
}
}
public synchronized void clear() {
for( int i=0; i<data.length; i++ ) {
data[i] = null; // for GC
}
head = tail = size = 0;
}
public List<T> toList()
{
ArrayList<T> list = new ArrayList<T>( size );
for( int i=0; i<size; i++ ) {
list.add( data[convert(i)] );
}
return list;
}
@Override
public String toString()
{
StringBuilder str = new StringBuilder();
str.append( "[" );
for( int i=0; i<size; i++ ) {
if( i > 0 ) {
str.append( "," );
}
str.append( data[convert(i)] );
}
str.append( "]" );
return str.toString();
}
@Override
public Iterator<T> iterator() {
return new Iterator<T>() {
int idx = 0;
@Override
public boolean hasNext() {
return idx<size;
}
@Override
public T next() {
return get( idx++ );
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* FIFO Circular List.
*
* Once the size is reached, it will overwrite previous entries
*
*/
public class CircularList<T> implements Iterable<T>
{
private T[] data;
private int head=0;
private int tail=0;
private int size=0;
@SuppressWarnings("unchecked")
public CircularList(int size) {
data = (T[])new Object[size];
}
@SuppressWarnings("unchecked")
public synchronized void resize(int newsize) {
if(newsize==this.size) return;
T[] vals = (T[])new Object[newsize];
int i = 0;
if(newsize>size) {
for(i=0; i<size; i++) {
vals[i] = data[convert(i)];
}
}
else {
int off=size-newsize;
for(i=0; i<newsize; i++) {
vals[i] = data[convert(i+off)];
}
}
data = vals;
head = 0;
tail = i;
}
private int convert(int index) {
return (index + head) % data.length;
}
public boolean isEmpty() {
return head == tail; // or size == 0
}
public int size() {
return size;
}
public int getBufferSize() {
return data.length;
}
private void checkIndex(int index) {
if (index >= size || index < 0)
throw new IndexOutOfBoundsException("Index: "+index+", Size: "+size);
}
public T get(int index) {
checkIndex(index);
return data[convert(index)];
}
public synchronized void add(T o) {
data[tail] = o;
tail = (tail+1)%data.length;
if( size == data.length ) {
head = (head+1)%data.length;
}
size++;
if( size > data.length ) {
size = data.length;
}
}
public synchronized void clear() {
for( int i=0; i<data.length; i++ ) {
data[i] = null; // for GC
}
head = tail = size = 0;
}
public List<T> toList()
{
ArrayList<T> list = new ArrayList<T>( size );
for( int i=0; i<size; i++ ) {
list.add( data[convert(i)] );
}
return list;
}
@Override
public String toString()
{
StringBuilder str = new StringBuilder();
str.append( "[" );
for( int i=0; i<size; i++ ) {
if( i > 0 ) {
str.append( "," );
}
str.append( data[convert(i)] );
}
str.append( "]" );
return str.toString();
}
@Override
public Iterator<T> iterator() {
return new Iterator<T>() {
int idx = 0;
@Override
public boolean hasNext() {
return idx<size;
}
@Override
public T next() {
return get( idx++ );
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
}

View File

@ -1,107 +1,107 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.core.CoreContainer;
/**
* A Class to monitor Logging events and hold N events in memory
*
* This is abstract so we can support both JUL and Log4j (and other logging platforms)
*/
public abstract class LogWatcher<E> {
protected CircularList<E> history;
protected long last = -1;
/**
* @return The implementation name
*/
public abstract String getName();
/**
* @return The valid level names for this framework
*/
public abstract List<String> getAllLevels();
/**
* Sets the log level within this framework
*/
public abstract void setLogLevel(String category, String level);
/**
* @return all registered loggers
*/
public abstract Collection<LoggerInfo> getAllLoggers();
public abstract void setThreshold(String level);
public abstract String getThreshold();
public void add(E event, long timstamp) {
history.add(event);
last = timstamp;
}
public long getLastEvent() {
return last;
}
public int getHistorySize() {
return (history==null) ? -1 : history.getBufferSize();
}
public SolrDocumentList getHistory(long since, AtomicBoolean found) {
if(history==null) {
return null;
}
SolrDocumentList docs = new SolrDocumentList();
Iterator<E> iter = history.iterator();
while(iter.hasNext()) {
E e = iter.next();
long ts = getTimestamp(e);
if(ts == since) {
if(found!=null) {
found.set(true);
}
}
if(ts>since) {
docs.add(toSolrDocument(e));
}
}
docs.setNumFound(docs.size()); // make it not look too funny
return docs;
}
public abstract long getTimestamp(E event);
public abstract SolrDocument toSolrDocument(E event);
public abstract void registerListener(ListenerConfig cfg, CoreContainer container);
public void reset() {
history.clear();
last = -1;
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.core.CoreContainer;
/**
* A Class to monitor Logging events and hold N events in memory
*
* This is abstract so we can support both JUL and Log4j (and other logging platforms)
*/
public abstract class LogWatcher<E> {
protected CircularList<E> history;
protected long last = -1;
/**
* @return The implementation name
*/
public abstract String getName();
/**
* @return The valid level names for this framework
*/
public abstract List<String> getAllLevels();
/**
* Sets the log level within this framework
*/
public abstract void setLogLevel(String category, String level);
/**
* @return all registered loggers
*/
public abstract Collection<LoggerInfo> getAllLoggers();
public abstract void setThreshold(String level);
public abstract String getThreshold();
public void add(E event, long timstamp) {
history.add(event);
last = timstamp;
}
public long getLastEvent() {
return last;
}
public int getHistorySize() {
return (history==null) ? -1 : history.getBufferSize();
}
public SolrDocumentList getHistory(long since, AtomicBoolean found) {
if(history==null) {
return null;
}
SolrDocumentList docs = new SolrDocumentList();
Iterator<E> iter = history.iterator();
while(iter.hasNext()) {
E e = iter.next();
long ts = getTimestamp(e);
if(ts == since) {
if(found!=null) {
found.set(true);
}
}
if(ts>since) {
docs.add(toSolrDocument(e));
}
}
docs.setNumFound(docs.size()); // make it not look too funny
return docs;
}
public abstract long getTimestamp(E event);
public abstract SolrDocument toSolrDocument(E event);
public abstract void registerListener(ListenerConfig cfg, CoreContainer container);
public void reset() {
history.clear();
last = -1;
}
}

View File

@ -1,68 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import org.apache.solr.common.util.SimpleOrderedMap;
/**
* Wrapper class for Logger implementaions
*/
public abstract class LoggerInfo implements Comparable<LoggerInfo> {
public static final String ROOT_NAME = "root";
protected final String name;
protected String level;
public LoggerInfo(String name) {
this.name = name;
}
public String getLevel() {
return level;
}
public String getName() {
return name;
}
public abstract boolean isSet();
public SimpleOrderedMap<?> getInfo() {
SimpleOrderedMap<Object> info = new SimpleOrderedMap<Object>();
info.add("name", getName());
info.add("level", getLevel());
info.add("set", isSet());
return info;
}
@Override
public int compareTo(LoggerInfo other) {
if (this.equals(other))
return 0;
String tN = this.getName();
String oN = other.getName();
if(ROOT_NAME.equals(tN))
return -1;
if(ROOT_NAME.equals(oN))
return 1;
return tN.compareTo(oN);
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging;
import org.apache.solr.common.util.SimpleOrderedMap;
/**
* Wrapper class for Logger implementaions
*/
public abstract class LoggerInfo implements Comparable<LoggerInfo> {
public static final String ROOT_NAME = "root";
protected final String name;
protected String level;
public LoggerInfo(String name) {
this.name = name;
}
public String getLevel() {
return level;
}
public String getName() {
return name;
}
public abstract boolean isSet();
public SimpleOrderedMap<?> getInfo() {
SimpleOrderedMap<Object> info = new SimpleOrderedMap<Object>();
info.add("name", getName());
info.add("level", getLevel());
info.add("set", isSet());
return info;
}
@Override
public int compareTo(LoggerInfo other) {
if (this.equals(other))
return 0;
String tN = this.getName();
String oN = other.getName();
if(ROOT_NAME.equals(tN))
return -1;
if(ROOT_NAME.equals(oN))
return 1;
return tN.compareTo(oN);
}
}

View File

@ -1,70 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.solr.logging.LoggerInfo;
public class JulInfo extends LoggerInfo {
private static final Level[] LEVELS = {
null, // aka unset
Level.FINEST,
Level.FINE,
Level.CONFIG,
Level.INFO,
Level.WARNING,
Level.SEVERE,
Level.OFF
// Level.ALL -- ignore. It is useless.
};
final Logger logger;
public JulInfo(String name, Logger logger) {
super(name);
this.logger = logger;
}
@Override
public String getLevel() {
if(logger==null) {
return null;
}
Level level = logger.getLevel();
if (level != null) {
return level.getName();
}
for (Level l : LEVELS) {
if (l == null) {
// avoid NPE
continue;
}
if (logger.isLoggable(l)) {
// return first level loggable
return l.getName();
}
}
return Level.OFF.getName();
}
@Override
public boolean isSet() {
return (logger!=null && logger.getLevel()!=null);
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.solr.logging.LoggerInfo;
public class JulInfo extends LoggerInfo {
private static final Level[] LEVELS = {
null, // aka unset
Level.FINEST,
Level.FINE,
Level.CONFIG,
Level.INFO,
Level.WARNING,
Level.SEVERE,
Level.OFF
// Level.ALL -- ignore. It is useless.
};
final Logger logger;
public JulInfo(String name, Logger logger) {
super(name);
this.logger = logger;
}
@Override
public String getLevel() {
if(logger==null) {
return null;
}
Level level = logger.getLevel();
if (level != null) {
return level.getName();
}
for (Level l : LEVELS) {
if (l == null) {
// avoid NPE
continue;
}
if (logger.isLoggable(l)) {
// return first level loggable
return l.getName();
}
}
return Level.OFF.getName();
}
@Override
public boolean isSet() {
return (logger!=null && logger.getLevel()!=null);
}
}

View File

@ -1,169 +1,169 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.logging.Level;
import java.util.logging.LogManager;
import java.util.logging.LogRecord;
import java.util.logging.Logger;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.logging.CircularList;
import org.apache.solr.logging.ListenerConfig;
import org.apache.solr.logging.LoggerInfo;
import org.apache.solr.logging.LogWatcher;
import com.google.common.base.Throwables;
public class JulWatcher extends LogWatcher<LogRecord> {
final String name;
RecordHandler handler = null;
public JulWatcher(String name) {
this.name = name;
}
@Override
public String getName() {
return "JUL ("+name+")";
}
@Override
public List<String> getAllLevels() {
return Arrays.asList(
Level.FINEST.getName(),
Level.FINER.getName(),
Level.FINE.getName(),
Level.CONFIG.getName(),
Level.INFO.getName(),
Level.WARNING.getName(),
Level.SEVERE.getName(),
Level.OFF.getName() );
}
@Override
public void setLogLevel(String category, String level) {
if(LoggerInfo.ROOT_NAME.equals(category)) {
category = "";
}
Logger log = LogManager.getLogManager().getLogger(category);
if(level==null||"unset".equals(level)||"null".equals(level)) {
if(log!=null) {
log.setLevel(null);
}
}
else {
if(log==null) {
log = Logger.getLogger(category); // create it
}
log.setLevel(Level.parse(level));
}
}
@Override
public Collection<LoggerInfo> getAllLoggers() {
LogManager manager = LogManager.getLogManager();
Logger root = manager.getLogger("");
Map<String,LoggerInfo> map = new HashMap<String,LoggerInfo>();
Enumeration<String> names = manager.getLoggerNames();
while (names.hasMoreElements()) {
String name = names.nextElement();
Logger logger = Logger.getLogger(name);
if( logger == root) {
continue;
}
map.put(name, new JulInfo(name, logger));
while (true) {
int dot = name.lastIndexOf(".");
if (dot < 0)
break;
name = name.substring(0, dot);
if(!map.containsKey(name)) {
map.put(name, new JulInfo(name, null));
}
}
}
map.put(LoggerInfo.ROOT_NAME, new JulInfo(LoggerInfo.ROOT_NAME, root));
return map.values();
}
@Override
public void setThreshold(String level) {
if(handler==null) {
throw new IllegalStateException("Must have an handler");
}
handler.setLevel( Level.parse(level) );
}
@Override
public String getThreshold() {
if(handler==null) {
throw new IllegalStateException("Must have an handler");
}
return handler.getLevel().toString();
}
@Override
public void registerListener(ListenerConfig cfg, CoreContainer container) {
if(history!=null) {
throw new IllegalStateException("History already registered");
}
history = new CircularList<LogRecord>(cfg.size);
handler = new RecordHandler(this);
if(cfg.threshold != null) {
handler.setLevel(Level.parse(cfg.threshold));
}
else {
handler.setLevel(Level.WARNING);
}
Logger log = LogManager.getLogManager().getLogger("");
log.addHandler(handler);
}
@Override
public long getTimestamp(LogRecord event) {
return event.getMillis();
}
@Override
public SolrDocument toSolrDocument(LogRecord event) {
SolrDocument doc = new SolrDocument();
doc.setField("time", new Date(event.getMillis()));
doc.setField("level", event.getLevel().toString());
doc.setField("logger", event.getLoggerName());
doc.setField("message", event.getMessage().toString());
Throwable t = event.getThrown();
if(t!=null) {
doc.setField("trace", Throwables.getStackTraceAsString(t));
}
return doc;
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.logging.Level;
import java.util.logging.LogManager;
import java.util.logging.LogRecord;
import java.util.logging.Logger;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.logging.CircularList;
import org.apache.solr.logging.ListenerConfig;
import org.apache.solr.logging.LoggerInfo;
import org.apache.solr.logging.LogWatcher;
import com.google.common.base.Throwables;
public class JulWatcher extends LogWatcher<LogRecord> {
final String name;
RecordHandler handler = null;
public JulWatcher(String name) {
this.name = name;
}
@Override
public String getName() {
return "JUL ("+name+")";
}
@Override
public List<String> getAllLevels() {
return Arrays.asList(
Level.FINEST.getName(),
Level.FINER.getName(),
Level.FINE.getName(),
Level.CONFIG.getName(),
Level.INFO.getName(),
Level.WARNING.getName(),
Level.SEVERE.getName(),
Level.OFF.getName() );
}
@Override
public void setLogLevel(String category, String level) {
if(LoggerInfo.ROOT_NAME.equals(category)) {
category = "";
}
Logger log = LogManager.getLogManager().getLogger(category);
if(level==null||"unset".equals(level)||"null".equals(level)) {
if(log!=null) {
log.setLevel(null);
}
}
else {
if(log==null) {
log = Logger.getLogger(category); // create it
}
log.setLevel(Level.parse(level));
}
}
@Override
public Collection<LoggerInfo> getAllLoggers() {
LogManager manager = LogManager.getLogManager();
Logger root = manager.getLogger("");
Map<String,LoggerInfo> map = new HashMap<String,LoggerInfo>();
Enumeration<String> names = manager.getLoggerNames();
while (names.hasMoreElements()) {
String name = names.nextElement();
Logger logger = Logger.getLogger(name);
if( logger == root) {
continue;
}
map.put(name, new JulInfo(name, logger));
while (true) {
int dot = name.lastIndexOf(".");
if (dot < 0)
break;
name = name.substring(0, dot);
if(!map.containsKey(name)) {
map.put(name, new JulInfo(name, null));
}
}
}
map.put(LoggerInfo.ROOT_NAME, new JulInfo(LoggerInfo.ROOT_NAME, root));
return map.values();
}
@Override
public void setThreshold(String level) {
if(handler==null) {
throw new IllegalStateException("Must have an handler");
}
handler.setLevel( Level.parse(level) );
}
@Override
public String getThreshold() {
if(handler==null) {
throw new IllegalStateException("Must have an handler");
}
return handler.getLevel().toString();
}
@Override
public void registerListener(ListenerConfig cfg, CoreContainer container) {
if(history!=null) {
throw new IllegalStateException("History already registered");
}
history = new CircularList<LogRecord>(cfg.size);
handler = new RecordHandler(this);
if(cfg.threshold != null) {
handler.setLevel(Level.parse(cfg.threshold));
}
else {
handler.setLevel(Level.WARNING);
}
Logger log = LogManager.getLogManager().getLogger("");
log.addHandler(handler);
}
@Override
public long getTimestamp(LogRecord event) {
return event.getMillis();
}
@Override
public SolrDocument toSolrDocument(LogRecord event) {
SolrDocument doc = new SolrDocument();
doc.setField("time", new Date(event.getMillis()));
doc.setField("level", event.getLevel().toString());
doc.setField("logger", event.getLoggerName());
doc.setField("message", event.getMessage().toString());
Throwable t = event.getThrown();
if(t!=null) {
doc.setField("trace", Throwables.getStackTraceAsString(t));
}
return doc;
}
}

View File

@ -1,47 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.logging.LogRecord;
import org.apache.solr.logging.LogWatcher;
public final class RecordHandler extends java.util.logging.Handler {
final LogWatcher<LogRecord> framework;
public RecordHandler(LogWatcher<LogRecord> framework) {
this.framework = framework;
}
@Override
public void close() throws SecurityException {
//history.reset();
}
@Override
public void flush() {
// nothing
}
@Override
public void publish(LogRecord r) {
if(isLoggable(r)) {
framework.add(r, r.getMillis());
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.logging.jul;
import java.util.logging.LogRecord;
import org.apache.solr.logging.LogWatcher;
public final class RecordHandler extends java.util.logging.Handler {
final LogWatcher<LogRecord> framework;
public RecordHandler(LogWatcher<LogRecord> framework) {
this.framework = framework;
}
@Override
public void close() throws SecurityException {
//history.reset();
}
@Override
public void flush() {
// nothing
}
@Override
public void publish(LogRecord r) {
if(isLoggable(r)) {
framework.add(r, r.getMillis());
}
}
}

View File

@ -1,12 +1,12 @@
REM You can override pass the following parameters to this script:
REM
set JVM=java
REM Find location of this script
set SDIR=%~dp0
if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
"%JVM%" -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*" org.apache.solr.cloud.ZkCLI %*
REM You can override pass the following parameters to this script:
REM
set JVM=java
REM Find location of this script
set SDIR=%~dp0
if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
"%JVM%" -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*" org.apache.solr.cloud.ZkCLI %*