This commit is contained in:
Karl Wright 2019-02-25 03:34:05 -05:00
commit c599ad9802
40 changed files with 759 additions and 230 deletions

View File

@ -19,18 +19,19 @@ package org.apache.lucene.util.bkd;
import org.apache.lucene.util.BytesRef;
/**
* Represent a dimensional point value written in the BKD tree.
*/
/** Represents a dimensional point value written in the BKD tree.
*
* @lucene.internal
**/
public interface PointValue {
/** Return the packed values for the dimensions */
/** Returns the packed values for the dimensions */
BytesRef packedValue();
/** The document id */
/** Returns the document id */
int docID();
/** The byte representation of the document id */
/** Returns the byte representation of the document id */
BytesRef docIDBytes();
}

View File

@ -33,7 +33,7 @@ com.fasterxml.jackson.core.version = 2.9.6
/com.healthmarketscience.jackcess/jackcess-encrypt = 2.1.4
/com.ibm.icu/icu4j = 62.1
/com.jayway.jsonpath/json-path = 2.4.0
/com.lmax/disruptor = 3.4.0
/com.lmax/disruptor = 3.4.2
/com.pff/java-libpst = 0.8.1
com.rometools.version = 1.5.1
@ -164,11 +164,12 @@ org.apache.kerby.version = 1.0.1
/org.apache.kerby/kerby-pkix = ${org.apache.kerby.version}
/org.apache.kerby/kerby-util = ${org.apache.kerby.version}
org.apache.logging.log4j.version = 2.11.0
org.apache.logging.log4j.version = 2.11.2
/org.apache.logging.log4j/log4j-1.2-api = ${org.apache.logging.log4j.version}
/org.apache.logging.log4j/log4j-api = ${org.apache.logging.log4j.version}
/org.apache.logging.log4j/log4j-core = ${org.apache.logging.log4j.version}
/org.apache.logging.log4j/log4j-slf4j-impl = ${org.apache.logging.log4j.version}
/org.apache.logging.log4j/log4j-web = ${org.apache.logging.log4j.version}
/org.apache.opennlp/opennlp-tools = 1.9.1

View File

@ -31,6 +31,10 @@ Jetty 9.4.14.v20181114
Upgrade Notes
----------------------
* SOLR-12055 introduces async logging by default. There's a small window where log messages may be lost
in the event of some hard crash. Switch back to synchronous logging if this is unacceptable, see
see commeints in the log4j2 configuration files (log4j2.xml by default).
New Features
----------------------
@ -58,6 +62,8 @@ New Features
* SOLR-13171 : A true streaming parser for javabin payload/stream without creating any objects (noble)
* SOLR-13261: Make SortableTextField work with export/streaming
Bug Fixes
----------------------
@ -70,6 +76,9 @@ Bug Fixes
* SOLR-12708: Async collection actions should not hide internal failures (Mano Kovacs, Varun Thacker, Tomás Fernández Löbbe)
* SOLR-11883: 500 code on functional query syntax errors and parameter dereferencing errors
(Munendra S N via Mikhail Khludnev)
Improvements
----------------------
* SOLR-12999: Index replication could delete segments before downloading segments from master if there is not enough
@ -84,6 +93,16 @@ Other Changes
* SOLR-13222: Improve logging in StreamingSolrClients (Peter Cseh via Kevin Risden)
* SOLR-12055: Enable async logging by default. This change improves throughput for logging. This opens
up a small window where log messages could possibly be lost. If this is unacceptable, switching back to synchronous
logging can be done by changing the log4j2.xml file, no internal Solr code changed to make async logging the default.
(Erick Erickson)
* SOLR-12753: Async logging ring buffer and OOM error. When very long messages are written (1M messages or so),
it can produce an OOM error. Log messages are truncated at 10K via configuration in the log4j2.xml files.
* SOLR-9763: Remove the workaround implemented for HADOOP-12767 (Kevin Risden)
================== 8.0.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
@ -719,6 +738,7 @@ Upgrade Notes
an object style. Although the object style makes more sense, this was an overlooked back-compat break; sorry.
New Features
----------------------

View File

@ -58,6 +58,7 @@ import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.IntValueFieldType;
import org.apache.solr.schema.LongValueFieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.SortableTextField;
import org.apache.solr.schema.StrField;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SortSpec;
@ -69,7 +70,21 @@ import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import static org.apache.solr.common.util.Utils.makeMap;
/**
* Prepares and writes the documents requested by /export requests
*
* {@link ExportWriter} gathers and sorts the documents for a core using "stream sorting".
* <p>
* Stream sorting works by repeatedly processing and modifying a bitmap of matching documents. Each pass over the
* bitmap identifies the smallest {@link #DOCUMENT_BATCH_SIZE} docs that haven't been sent yet and stores them in a
* Priority Queue. They are then exported (written across the wire) and marked as sent (unset in the bitmap).
* This process repeats until all matching documents have been sent.
* <p>
* This streaming approach is light on memory (only {@link #DOCUMENT_BATCH_SIZE} documents are ever stored in memory at
* once), and it allows {@link ExportWriter} to scale well with regard to numDocs.
*/
public class ExportWriter implements SolrCore.RawWriter, Closeable {
private static final int DOCUMENT_BATCH_SIZE = 30000;
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private OutputStreamWriter respWriter;
final SolrQueryRequest req;
@ -211,72 +226,77 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
protected void identifyLowestSortingUnexportedDocs(List<LeafReaderContext> leaves, SortDoc sortDoc, SortQueue queue) throws IOException {
queue.reset();
SortDoc top = queue.top();
for (int i = 0; i < leaves.size(); i++) {
sortDoc.setNextReader(leaves.get(i));
DocIdSetIterator it = new BitSetIterator(sets[i], 0); // cost is not useful here
int docId;
while ((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
sortDoc.setValues(docId);
if (top.lessThan(sortDoc)) {
top.setValues(sortDoc);
top = queue.updateTop();
}
}
}
}
protected int transferBatchToArrayForOutput(SortQueue queue, SortDoc[] destinationArr) {
int outDocsIndex = -1;
for (int i = 0; i < queue.maxSize; i++) {
SortDoc s = queue.pop();
if (s.docId > -1) {
destinationArr[++outDocsIndex] = s;
}
}
return outDocsIndex;
}
protected void addDocsToItemWriter(List<LeafReaderContext> leaves, IteratorWriter.ItemWriter writer, SortDoc[] docsToExport, int outDocsIndex) throws IOException {
try {
for (int i = outDocsIndex; i >= 0; --i) {
SortDoc s = docsToExport[i];
writer.add((MapWriter) ew -> {
writeDoc(s, leaves, ew);
s.reset();
});
}
} catch (Throwable e) {
Throwable ex = e;
while (ex != null) {
String m = ex.getMessage();
if (m != null && m.contains("Broken pipe")) {
throw new IgnoreException();
}
ex = ex.getCause();
}
if (e instanceof IOException) {
throw ((IOException) e);
} else {
throw new IOException(e);
}
}
}
protected void writeDocs(SolrQueryRequest req, IteratorWriter.ItemWriter writer, Sort sort) throws IOException {
//Write the data.
List<LeafReaderContext> leaves = req.getSearcher().getTopReaderContext().leaves();
SortDoc sortDoc = getSortDoc(req.getSearcher(), sort.getSort());
int count = 0;
int queueSize = 30000;
if (totalHits < 30000) {
queueSize = totalHits;
}
final int queueSize = Math.min(DOCUMENT_BATCH_SIZE, totalHits);
SortQueue queue = new SortQueue(queueSize, sortDoc);
SortDoc[] outDocs = new SortDoc[queueSize];
while (count < totalHits) {
//long begin = System.nanoTime();
queue.reset();
SortDoc top = queue.top();
for (int i = 0; i < leaves.size(); i++) {
sortDoc.setNextReader(leaves.get(i));
DocIdSetIterator it = new BitSetIterator(sets[i], 0); // cost is not useful here
int docId;
while ((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
sortDoc.setValues(docId);
if (top.lessThan(sortDoc)) {
top.setValues(sortDoc);
top = queue.updateTop();
}
}
}
int outDocsIndex = -1;
for (int i = 0; i < queueSize; i++) {
SortDoc s = queue.pop();
if (s.docId > -1) {
outDocs[++outDocsIndex] = s;
}
}
//long end = System.nanoTime();
identifyLowestSortingUnexportedDocs(leaves, sortDoc, queue);
int outDocsIndex = transferBatchToArrayForOutput(queue, outDocs);
count += (outDocsIndex + 1);
try {
for (int i = outDocsIndex; i >= 0; --i) {
SortDoc s = outDocs[i];
writer.add((MapWriter) ew -> {
writeDoc(s, leaves, ew);
s.reset();
});
}
} catch (Throwable e) {
Throwable ex = e;
while (ex != null) {
String m = ex.getMessage();
if (m != null && m.contains("Broken pipe")) {
throw new IgnoreException();
}
ex = ex.getCause();
}
if (e instanceof IOException) {
throw ((IOException) e);
} else {
throw new IOException(e);
}
}
addDocsToItemWriter(leaves, writer, outDocs, outDocsIndex);
}
}
@ -339,7 +359,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
} else {
writers[i] = new DoubleFieldWriter(field);
}
} else if (fieldType instanceof StrField) {
} else if (fieldType instanceof StrField || fieldType instanceof SortableTextField) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
} else {
@ -358,7 +378,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
writers[i] = new BoolFieldWriter(field, fieldType);
}
} else {
throw new IOException("Export fields must either be one of the following types: int,float,long,double,string,date,boolean");
throw new IOException("Export fields must be one of the following types: int,float,long,double,string,date,boolean,SortableText");
}
}
return writers;
@ -402,7 +422,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
} else {
sortValues[i] = new LongValue(field, new LongAsc());
}
} else if (ft instanceof StrField) {
} else if (ft instanceof StrField || ft instanceof SortableTextField) {
LeafReader reader = searcher.getSlowAtomicReader();
SortedDocValues vals = reader.getSortedDocValues(field);
if (reverse) {
@ -428,7 +448,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
sortValues[i] = new StringValue(vals, field, new IntAsc());
}
} else {
throw new IOException("Sort fields must be one of the following types: int,float,long,double,string,date,boolean");
throw new IOException("Sort fields must be one of the following types: int,float,long,double,string,date,boolean,SortableText");
}
}
//SingleValueSortDoc etc are specialized classes which don't have array lookups. On benchmarking large datasets

View File

@ -16,9 +16,16 @@
*/
package org.apache.solr.search;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.queries.function.FunctionQuery;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.*;
import org.apache.lucene.queries.function.valuesource.ConstValueSource;
import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
import org.apache.lucene.queries.function.valuesource.LiteralValueSource;
import org.apache.lucene.queries.function.valuesource.QueryValueSource;
import org.apache.lucene.queries.function.valuesource.VectorValueSource;
import org.apache.lucene.search.Query;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
@ -27,9 +34,6 @@ import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.facet.AggValueSource;
import org.apache.solr.search.function.FieldNameValueSource;
import java.util.ArrayList;
import java.util.List;
public class FunctionQParser extends QParser {
public static final int FLAG_CONSUME_DELIMITER = 0x01; // consume delimiter after parsing arg
@ -130,7 +134,11 @@ public class FunctionQParser extends QParser {
*/
public String parseId() throws SyntaxError {
String value = parseArg();
if (argWasQuoted) throw new SyntaxError("Expected identifier instead of quoted string:" + value);
if (argWasQuoted()) {
throw new SyntaxError("Expected identifier instead of quoted string:" + value);
} else if (value == null) {
throw new SyntaxError("Expected identifier instead of 'null' for function " + sp);
}
return value;
}
@ -142,8 +150,11 @@ public class FunctionQParser extends QParser {
public Float parseFloat() throws SyntaxError {
String str = parseArg();
if (argWasQuoted()) throw new SyntaxError("Expected float instead of quoted string:" + str);
float value = Float.parseFloat(str);
return value;
try {
return Float.parseFloat(str);
} catch (NumberFormatException | NullPointerException e) {
throw new SyntaxError("Expected float instead of '" + str + "' for function " + sp);
}
}
/**
@ -153,8 +164,11 @@ public class FunctionQParser extends QParser {
public double parseDouble() throws SyntaxError {
String str = parseArg();
if (argWasQuoted()) throw new SyntaxError("Expected double instead of quoted string:" + str);
double value = Double.parseDouble(str);
return value;
try {
return Double.parseDouble(str);
} catch (NumberFormatException | NullPointerException e) {
throw new SyntaxError("Expected double instead of '" + str + "' for function " + sp);
}
}
/**
@ -163,9 +177,12 @@ public class FunctionQParser extends QParser {
*/
public int parseInt() throws SyntaxError {
String str = parseArg();
if (argWasQuoted()) throw new SyntaxError("Expected double instead of quoted string:" + str);
int value = Integer.parseInt(str);
return value;
if (argWasQuoted()) throw new SyntaxError("Expected integer instead of quoted string:" + str);
try {
return Integer.parseInt(str);
} catch (NumberFormatException | NullPointerException e) {
throw new SyntaxError("Expected integer instead of '" + str + "' for function " + sp);
}
}
@ -248,6 +265,12 @@ public class FunctionQParser extends QParser {
String qstr = getParam(param);
qstr = qstr==null ? "" : qstr;
nestedQuery = subQuery(qstr, null).getQuery();
// nestedQuery would be null when de-referenced query value is not specified
// Ex: query($qq) in request with no qq param specified
if (nestedQuery == null) {
throw new SyntaxError("Missing param " + param + " while parsing function '" + sp.val + "'");
}
}
else {
int start = sp.pos;
@ -277,9 +300,14 @@ public class FunctionQParser extends QParser {
sp.pos += end-start; // advance past nested query
nestedQuery = sub.getQuery();
// handling null check on nestedQuery separately, so that proper error can be returned
// one case this would be possible when v is specified but v's value is empty or has only spaces
if (nestedQuery == null) {
throw new SyntaxError("Nested function query returned null for '" + sp.val + "'");
}
}
consumeArgumentDelimiter();
return nestedQuery;
}
@ -369,8 +397,7 @@ public class FunctionQParser extends QParser {
}
valueSource = argParser.parse(this);
sp.expect(")");
}
else {
} else {
if ("true".equals(id)) {
valueSource = new BoolConstValueSource(true);
} else if ("false".equals(id)) {
@ -413,14 +440,13 @@ public class FunctionQParser extends QParser {
hasParen = sp.opt("(");
ValueSourceParser argParser = req.getCore().getValueSourceParser(id);
argParser = req.getCore().getValueSourceParser(id);
if (argParser == null) {
throw new SyntaxError("Unknown aggregation " + id + " in (" + sp + ")");
}
ValueSource vv = argParser.parse(this);
if (!(vv instanceof AggValueSource)) {
if (argParser == null) {
if (argParser == null) { // why this??
throw new SyntaxError("Expected aggregation from " + id + " but got (" + vv + ") in (" + sp + ")");
}
}

View File

@ -29,7 +29,6 @@ import javax.servlet.ServletException;
import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletRequestWrapper;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.AuthInfo;
@ -103,18 +102,6 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
@Override
public void doFilter(ServletRequest request, ServletResponse response,
FilterChain filterChain) throws IOException, ServletException {
// HttpClient 4.4.x throws NPE if query string is null and parsed through URLEncodedUtils.
// See HTTPCLIENT-1746 and HADOOP-12767
HttpServletRequest httpRequest = (HttpServletRequest)request;
String queryString = httpRequest.getQueryString();
final String nonNullQueryString = queryString == null ? "" : queryString;
HttpServletRequest requestNonNullQueryString = new HttpServletRequestWrapper(httpRequest){
@Override
public String getQueryString() {
return nonNullQueryString;
}
};
// include Impersonator User Name in case someone (e.g. logger) wants it
FilterChain filterChainWrapper = new FilterChain() {
@Override
@ -136,7 +123,7 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
// A hack until HADOOP-15681 get committed
Locale.setDefault(Locale.US);
super.doFilter(requestNonNullQueryString, response, filterChainWrapper);
super.doFilter(request, response, filterChainWrapper);
}
@Override

View File

@ -27,7 +27,6 @@ import javax.servlet.ServletException;
import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletRequestWrapper;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.AuthInfo;
@ -79,18 +78,6 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
@Override
public void doFilter(ServletRequest request, ServletResponse response,
FilterChain filterChain) throws IOException, ServletException {
// HttpClient 4.4.x throws NPE if query string is null and parsed through URLEncodedUtils.
// See HTTPCLIENT-1746 and HADOOP-12767
HttpServletRequest httpRequest = (HttpServletRequest)request;
String queryString = httpRequest.getQueryString();
final String nonNullQueryString = queryString == null ? "" : queryString;
HttpServletRequest requestNonNullQueryString = new HttpServletRequestWrapper(httpRequest){
@Override
public String getQueryString() {
return nonNullQueryString;
}
};
// include Impersonator User Name in case someone (e.g. logger) wants it
FilterChain filterChainWrapper = new FilterChain() {
@Override
@ -112,7 +99,7 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
// A hack until HADOOP-15681 get committed
Locale.setDefault(Locale.US);
super.doFilter(requestNonNullQueryString, response, filterChainWrapper);
super.doFilter(request, response, filterChainWrapper);
}
@Override
@ -232,4 +219,4 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
return ret;
}
}
}
}

View File

@ -126,6 +126,24 @@ public final class StartupLoggingUtils {
+ "Please consult your logging framework of choice on how to configure the appropriate logging.", msg);
}
/**
* Perhaps odd to put in startup utils, but this is where the logging-init code is so it seems logical to put the
* shutdown here too.
*
* Tests are particularly sensitive to this call or the object release tracker will report "lmax.disruptor" not
* terminating when asynch logging (new default as of 8.1) is enabled.
*
*
*/
@SuppressForbidden(reason = "Legitimate log4j2 access")
public static void shutdown() {
// if (!isLog4jActive()) {
// logNotSupported("Not running log4j2, could not call shutdown for async logging.");
// return;
// }
// LogManager.shutdown(true);
}
/**
* Return a string representing the current static ROOT logging level
* @return a string TRACE, DEBUG, WARN, ERROR or INFO representing current log level. Default is INFO

View File

@ -15,26 +15,55 @@
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- Configuration for asynchronous logging -->
<Configuration>
<Appenders>
<Console name="STDERR" target="SYSTEM_ERR">
<PatternLayout>
<Pattern>
%-4r %-5p (%t) [%X{node_name} %X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
%maxLen{%-4r %-5p (%t) [%X{node_name} %X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%notEmpty{
=>%ex{short}}}{10240}%n
</Pattern>
</PatternLayout>
</Console>
</Appenders>
<Loggers>
<Logger name="org.apache.zookeeper" level="WARN"/>
<Logger name="org.apache.hadoop" level="WARN"/>
<Logger name="org.apache.directory" level="WARN"/>
<Logger name="org.apache.solr.hadoop" level="INFO"/>
<AsyncLogger name="org.apache.zookeeper" level="WARN"/>
<AsyncLogger name="org.apache.hadoop" level="WARN"/>
<AsyncLogger name="org.apache.directory" level="WARN"/>
<AsyncLogger name="org.apache.solr.hadoop" level="INFO"/>
<Logger name="org.eclipse.jetty" level="INFO"/>
<Root level="INFO">
<AsyncRoot level="INFO">
<AppenderRef ref="STDERR"/>
</Root>
</AsyncRoot>
</Loggers>
</Configuration>
<!-- Configuration for synchronous logging
there _may_ be a very small window where log messages will not be flushed
to the log file on abnormal shutdown. If even this risk is unacceptable, use
the configuration below
-->
<!--Configuration>
<Appenders>
<Console name="STDERR" target="SYSTEM_ERR">
<PatternLayout>
<Pattern>
%-4r %-5p (%t) [%X{node_name} %X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n
</Pattern>
</PatternLayout>
</Console>
</Appenders>
<Loggers>
<Logger name="org.apache.zookeeper" level="WARN"/>
<Logger name="org.apache.hadoop" level="WARN"/>
<Logger name="org.apache.directory" level="WARN"/>
<Logger name="org.apache.solr.hadoop" level="INFO"/>
<Logger name="org.eclipse.jetty" level="INFO"/>
<Root level="INFO">
<AppenderRef ref="STDERR"/>
</Root>
</Loggers>
</Configuration-->

View File

@ -26,7 +26,7 @@
seconds part (.999) is optional.
-->
<fieldType name="date" class="${solr.tests.DateFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
<!-- Point Fields -->
<fieldType name="pint" class="solr.IntPointField" docValues="true"/>
<fieldType name="plong" class="solr.LongPointField" docValues="true"/>
@ -40,6 +40,27 @@
<fieldType name="uuid" class="solr.UUIDField"/>
<!-- SortableTextField generaly functions exactly like TextField,
except that it supports, and by default uses, docValues for sorting (or faceting)
on the first 1024 characters of the original field values (which is configurable).
This makes it a bit more useful then TextField in many situations, but the trade-off
is that it takes up more space on disk; which is why it's not used in place of TextField
for every fieldType in this _default schema.
-->
<fieldType name="text_gen_sort" class="solr.SortableTextField" positionIncrementGap="100" multiValued="true">
<analyzer type="index">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
</fieldType>
<field name="id" type="string" required="true" indexed="true" docValues="true"/>
<field name="floatdv_m" type="float" indexed="false" stored="false" docValues="true" multiValued="true"/>
<field name="intdv_m" type="int" indexed="false" stored="false" docValues="true" multiValued="true"/>
@ -47,6 +68,7 @@
<field name="longdv_m" type="long" indexed="false" stored="false" docValues="true" multiValued="true"/>
<field name="datedv_m" type="date" indexed="false" stored="false" docValues="true" multiValued="true"/>
<field name="stringdv_m" type="string" indexed="false" stored="false" docValues="true" multiValued="true"/>
<field name="sortabledv_m" type="text_gen_sort" indexed="true" stored="true" multiValued="true" />
<field name="floatdv" type="float" indexed="false" stored="false" docValues="true"/>
<field name="intdv" type="int" indexed="false" stored="false" docValues="true"/>
@ -55,6 +77,7 @@
<field name="datedv" type="date" indexed="false" stored="false" docValues="true"/>
<field name="stringdv" type="string" indexed="false" stored="false" docValues="true"/>
<field name="booleandv" type="boolean" indexed="false" stored="false" docValues="true" />
<field name="sortabledv" type="text_gen_sort" indexed="true" stored="true" multiValued="false" />
<dynamicField name="*_s_dv" type="string" indexed="true" stored="true" docValues="true" multiValued="false"/>

View File

@ -23,5 +23,17 @@
<field name="_version_" type="long" indexed="true" stored="true"/>
<field name="_root_" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
<field name="id" type="string" indexed="true" stored="true"/>
<fieldType name="sortabletext" class="solr.SortableTextField" positionIncrementGap="100" multiValued="true">
<analyzer type="index">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
<analyzer type="query">
<tokenizer class="solr.StandardTokenizerFactory"/>
<filter class="solr.LowerCaseFilterFactory"/>
</analyzer>
</fieldType>
<uniqueKey>id</uniqueKey>
</schema>

View File

@ -98,7 +98,8 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
new FieldProps("floatField", "float", 1),
new FieldProps("dateField", "date", 1),
new FieldProps("stringField", "string", 1),
new FieldProps("boolField", "boolean", 1)
new FieldProps("boolField", "boolean", 1),
new FieldProps("sortableText", "sortabletext", 1)
));
fieldsToTestMulti =
@ -109,10 +110,11 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
new FieldProps("floatFieldMulti", "float", 5),
new FieldProps("dateFieldMulti", "date", 5),
new FieldProps("stringFieldMulti", "string", 5),
new FieldProps("boolFieldMulti", "boolean", 2)
new FieldProps("boolFieldMulti", "boolean", 2),
new FieldProps("sortableFieldMulti", "sortabletext", 5)
));
// Fields to test for grouping and sorting with sortMinssingFirst/Last.
// Fields to test for grouping and sorting with sortMissingFirst/Last.
fieldsToTestGroupSortFirst =
Collections.unmodifiableList(Arrays.asList(
new FieldProps("intGSF", "int"),
@ -121,7 +123,8 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
new FieldProps("floatGSF", "float"),
new FieldProps("dateGSF", "date"),
new FieldProps("stringGSF", "string"),
new FieldProps("boolGSF", "boolean")
new FieldProps("boolGSF", "boolean"),
new FieldProps("sortableGSF", "sortabletext")
));
fieldsToTestGroupSortLast =
@ -132,7 +135,8 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
new FieldProps("floatGSL", "float"),
new FieldProps("dateGSL", "date"),
new FieldProps("stringGSL", "string"),
new FieldProps("boolGSL", "boolean")
new FieldProps("boolGSL", "boolean"),
new FieldProps("sortableGSL", "sortabletext")
));
List<Update> updateList = new ArrayList<>(fieldsToTestSingle.size() +
@ -210,18 +214,18 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
final QueryResponse rsp = client.query(COLLECTION, solrQuery);
for (FieldProps props : fieldsToTestSingle) {
testFacet(props, rsp);
doTestFacet(props, rsp);
}
for (FieldProps props : fieldsToTestMulti) {
testFacet(props, rsp);
doTestFacet(props, rsp);
}
}
// We should be able to sort thing with missing first/last and that are _NOT_ present at all on one server.
@Test
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-12028")
//@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-12028")
public void testGroupingSorting() throws IOException, SolrServerException {
CloudSolrClient client = cluster.getSolrClient();
@ -314,12 +318,17 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
// 12-Jun-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 04-May-2018
// commented 15-Sep-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2-Aug-2018
public void testGroupingDVOnly() throws IOException, SolrServerException {
doGroupingDvOnly(fieldsToTestGroupSortFirst, "boolGSF");
doGroupingDvOnly(fieldsToTestGroupSortLast, "boolGSL");
}
private void doGroupingDvOnly(List<FieldProps> fieldProps, String boolName) throws IOException, SolrServerException {
List<SolrInputDocument> docs = new ArrayList<>(50);
for (int idx = 0; idx < 49; ++idx) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", idx);
boolean doInc = ((idx % 7) == 0);
for (FieldProps prop : fieldsToTestGroupSortFirst) {
for (FieldProps prop : fieldProps) {
doc.addField(prop.getName(), prop.getValue(doInc));
}
docs.add(doc);
@ -337,13 +346,10 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
.commit(client, COLLECTION);
// OK, we should have one group with 10 entries for null, a group with 1 entry and 7 groups with 7
for (FieldProps prop : fieldsToTestGroupSortFirst) {
// Special handling until SOLR-9802 is fixed
for (FieldProps prop : fieldProps) {
// Solr 9802
if (prop.getName().startsWith("date")) continue;
// SOLR-9802 to here
// TODO: gsf fails this
if (prop.getName().endsWith("GSF") ) continue;
final SolrQuery solrQuery = new SolrQuery(
"q", "*:*",
@ -376,7 +382,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
case 25:
case 24:
++boolCount;
assertEquals("We should have more counts for boolean fields!", "boolGSF", prop.getName());
assertEquals("We should have more counts for boolean fields!", boolName, prop.getName());
break;
default:
@ -442,7 +448,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
}
private void testFacet(FieldProps props, QueryResponse rsp) {
private void doTestFacet(FieldProps props, QueryResponse rsp) {
String name = props.getName();
final List<FacetField.Count> counts = rsp.getFacetField(name).getValues();
long expectedCount = props.getExpectedCount();
@ -483,7 +489,7 @@ class FieldProps {
base = Math.abs(random().nextLong());
} else if (name.startsWith("bool")) {
base = true; // Must start with a known value since bools only have a two values....
} else if (name.startsWith("string")) {
} else if (name.startsWith("string") || name.startsWith("sortable")) {
base = "base_string_" + random().nextInt(1_000_000) + "_";
} else {
throw new RuntimeException("Should have found a prefix for the field before now!");
@ -531,7 +537,7 @@ class FieldProps {
base = !((boolean) base);
return ret;
}
if (name.startsWith("string")) {
if (name.startsWith("string") || name.startsWith("sortable")) {
return String.format(Locale.ROOT, "%s_%08d", (String) base, counter);
}
throw new RuntimeException("Should have found a prefix for the field before now!");

View File

@ -218,7 +218,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
CollectionAdminResponse response = CollectionAdminRequest.reloadCollection(collectionName)
.process(cluster.getSolrClient());
assertEquals(0, response.getStatus());
waitForState("failed waiting for active colletion", collectionName, clusterShape(2, 4));
waitForState("failed waiting for active colletion", collectionName, clusterShape(2, 8));
reloaded = true;
}
}
@ -431,7 +431,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
if (removeReplica) {
CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
} else {
leaderJetty.stop();
leaderJetty.start();
}
waitForState("Expected collection to be 1x2", collectionName, clusterShape(1, 2));
// added replica should replicate from the leader
@ -456,7 +456,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
cluster.getSolrClient().commit(collectionName);
waitForNumDocsInAllActiveReplicas(2);
pullReplicaJetty.stop();
pullReplicaJetty.start();
waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
waitForNumDocsInAllActiveReplicas(2);
}
@ -531,7 +531,6 @@ public class TestTlogReplica extends SolrCloudTestCase {
@SuppressWarnings("unchecked")
public void testRecovery() throws Exception {
boolean useKill = random().nextBoolean();
createAndWaitForCollection(1, 0, 2, 0);
CloudSolrClient cloudClient = cluster.getSolrClient();
@ -543,16 +542,12 @@ public class TestTlogReplica extends SolrCloudTestCase {
.add(sdoc("id", "5"))
.process(cloudClient, collectionName);
JettySolrRunner solrRunner = getSolrRunner(false).get(0);
if (useKill) {
solrRunner.stop();
} else {
solrRunner.stop();
}
solrRunner.stop();
waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
new UpdateRequest()
.add(sdoc("id", "6"))
.process(cloudClient, collectionName);
solrRunner.stop();
solrRunner.start();
waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
// We skip peerSync, so replica will always trigger commit on leader
// We query only the non-leader replicas, since we haven't opened a new searcher on the leader yet
@ -575,7 +570,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
solrRunner.stop();
waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
DirectUpdateHandler2.commitOnClose = true;
solrRunner.stop();
solrRunner.start();
waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
waitForNumDocsInAllReplicas(5, getNonLeaderReplias(collectionName), 10); //timeout for stale collection state
checkRTG(3,7, cluster.getJettySolrRunners());
@ -593,12 +588,8 @@ public class TestTlogReplica extends SolrCloudTestCase {
fail("Test interrupted: " + e.getMessage());
}
};
if (useKill) {
solrRunner.stop();
} else {
solrRunner.stop();
}
solrRunner.stop();
solrRunner.start();
waitingForReplay.acquire();
// If I add the doc immediately, the leader fails to communicate with the follower with broken pipe.
// Options are, wait or retry...
@ -672,7 +663,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
.add(sdoc("id", "3"))
.add(sdoc("id", "4"))
.process(cloudClient, collectionName);
oldLeaderJetty.stop();
oldLeaderJetty.start();
waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
checkRTG(1,4, cluster.getJettySolrRunners());
new UpdateRequest()
@ -707,7 +698,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
return !leader.getNodeName().equals(oldLeaderNodeName);
}
);
oldLeaderJetty.stop();
oldLeaderJetty.start();
waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
checkRTG(1,1, cluster.getJettySolrRunners());
SolrDocument doc = cluster.getSolrClient().getById(collectionName,"1");

View File

@ -23,7 +23,6 @@ import java.util.HashMap;
import java.util.Map;
import com.google.common.collect.Sets;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestRuleLimitSysouts;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
@ -41,7 +40,7 @@ import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.when;
@TestRuleLimitSysouts.Limit(bytes=32000)
public class ImplicitSnitchTest extends LuceneTestCase {
public class ImplicitSnitchTest extends SolrTestCaseJ4 {
private ImplicitSnitch snitch;
private SnitchContext context;

View File

@ -17,6 +17,7 @@
package org.apache.solr.handler;
import java.io.StringWriter;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -30,7 +31,9 @@ import org.apache.logging.log4j.core.config.LoggerConfig;
import org.apache.logging.log4j.core.layout.PatternLayout;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.TimeOut;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -48,7 +51,6 @@ public class RequestLoggingTest extends SolrTestCaseJ4 {
@Before
public void setupAppender() {
LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
LoggerConfig config = ctx.getConfiguration().getLoggerConfig("RequestLoggingTest");
writer = new StringWriter();
appender = WriterAppender.createAppender(
@ -62,18 +64,18 @@ public class RequestLoggingTest extends SolrTestCaseJ4 {
}
@Test
public void testLogBeforeExecuteWithCoreLogger() {
public void testLogBeforeExecuteWithCoreLogger() throws InterruptedException {
Logger logger = LogManager.getLogger(SolrCore.class);
testLogBeforeExecute(logger);
}
@Test
public void testLogBeforeExecuteWithRequestLogger() {
public void testLogBeforeExecuteWithRequestLogger() throws InterruptedException {
Logger logger = LogManager.getLogger("org.apache.solr.core.SolrCore.Request");
testLogBeforeExecute(logger);
}
public void testLogBeforeExecute(Logger logger) {
public void testLogBeforeExecute(Logger logger) throws InterruptedException {
Level level = logger.getLevel();
LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
@ -85,9 +87,22 @@ public class RequestLoggingTest extends SolrTestCaseJ4 {
try {
assertQ(req("q", "*:*"));
String output = writer.toString();
Matcher matcher = Pattern.compile("DEBUG.*q=\\*:\\*.*").matcher(output);
assertTrue(matcher.find());
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
boolean found = false;
Matcher matcher;
String pat = "DEBUG.*q=\\*:\\*.*";
String output = "";
Pattern pattern = Pattern.compile(pat);
do {
output = writer.toString();
matcher = pattern.matcher(output);
found = matcher.find();
if (found) {
break;
}
timeOut.sleep(10);
} while (timeOut.hasTimedOut() == false);
assertTrue("Did not find expected pattern: '" + pat + "' in output: '" + output + "'", found);
final String group = matcher.group();
final String msg = "Should not have post query information";
assertFalse(msg, group.contains("hits"));

View File

@ -129,7 +129,10 @@ public class TestExportWriter extends SolrTestCaseJ4 {
"datedv_m", "2017-06-16T01:00:00Z",
"datedv_m", "2017-06-16T02:00:00Z",
"datedv_m", "2017-06-16T03:00:00Z",
"datedv_m", "2017-06-16T04:00:00Z"));
"datedv_m", "2017-06-16T04:00:00Z",
"sortabledv_m", "this is some text one_1",
"sortabledv_m", "this is some text two_1",
"sortabledv_m", "this is some text three_1"));
assertU(adoc("id","7",
"floatdv","2.1",
@ -166,7 +169,8 @@ public class TestExportWriter extends SolrTestCaseJ4 {
"int_is_t", "1",
"int_is_t", "1",
"int_is_t", "1",
"int_is_t", "1"));
"int_is_t", "1",
"sortabledv", "this is some text_1"));
assertU(commit());
assertU(adoc("id","8",
"floatdv","2.1",
@ -191,7 +195,11 @@ public class TestExportWriter extends SolrTestCaseJ4 {
"int_is_p", "1",
"int_is_p", "1",
"int_is_p", "1",
"int_is_p", "1"));
"int_is_p", "1",
"sortabledv", "this is some text_2",
"sortabledv_m", "this is some text one_2",
"sortabledv_m", "this is some text two_2",
"sortabledv_m", "this is some text three_2"));
assertU(commit());
@ -491,6 +499,24 @@ public class TestExportWriter extends SolrTestCaseJ4 {
s = h.query(req("q", "id:8", "qt", "/export", "fl", "stringdv", "sort", "intdv asc"));
assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"stringdv\":\"chello \\\"world\\\"\"}]}}");
// Test sortable text fields:
s = h.query(req("q", "id:(1 OR 3 OR 8)", "qt", "/export", "fl", "sortabledv_m,sortabledv", "sort", "sortabledv asc"));
assertJsonEquals(s, "{\n" +
" \"responseHeader\":{\"status\":0},\n" +
" \"response\":{\n" +
" \"numFound\":3,\n" +
" \"docs\":[{\n" +
" \"sortabledv_m\":[\"this is some text one_1\"\n" +
" ,\"this is some text three_1\"\n" +
" ,\"this is some text two_1\"]}\n" +
" ,{\n" +
" \"sortabledv\":\"this is some text_1\"}\n" +
" ,{\n" +
" \"sortabledv_m\":[\"this is some text one_2\"\n" +
" ,\"this is some text three_2\"\n" +
" ,\"this is some text two_2\"],\n" +
" \"sortabledv\":\"this is some text_2\"}]}}");
}
private void assertJsonEquals(String actual, String expected) {

View File

@ -16,65 +16,61 @@
*/
package org.apache.solr.logging;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestLogWatcher {
public class TestLogWatcher extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private LogWatcherConfig config;
@Before
public void setUp() {
config = new LogWatcherConfig(true, null, null, 50);
public void before() {
config = new LogWatcherConfig(true, null, "INFO", 1);
}
// Create several log watchers and ensure that new messages go to the new watcher.
@Test
public void testLog4jWatcher() {
LogWatcher watcher = LogWatcher.newRegisteredLogWatcher(config, null);
public void testLog4jWatcher() throws InterruptedException {
LogWatcher watcher = null;
int lim = random().nextInt(3) + 2;
for (int idx = 0; idx < lim; ++idx) {
String msg = "This is a test message: " + idx;
watcher = LogWatcher.newRegisteredLogWatcher(config, null);
assertEquals(watcher.getLastEvent(), -1);
// First ensure there's nothing in the new watcher.
assertEquals(-1, watcher.getLastEvent());
log.warn("This is a test message");
// Now log a message and ensure that the new watcher sees it.
log.warn(msg);
assertTrue(watcher.getLastEvent() > -1);
SolrDocumentList events = watcher.getHistory(-1, new AtomicBoolean());
assertEquals(events.size(), 1);
SolrDocument event = events.get(0);
assertEquals(event.get("logger"), "org.apache.solr.logging.TestLogWatcher");
assertEquals(event.get("message"), "This is a test message");
// Loop to give the logger time to process the async message and notify the new watcher.
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
boolean foundMsg = false;
// In local testing this loop usually succeeds 1-2 tries.
do {
// Returns an empty (but non-null) list even if there are no messages yet.
SolrDocumentList events = watcher.getHistory(-1, new AtomicBoolean());
for (SolrDocument doc : events) {
if (doc.get("message").equals(msg)) {
foundMsg = true;
break;
}
}
Thread.sleep(10);
} while (timeOut.hasTimedOut() == false);
assertTrue("Should have found message " + msg + " in loop: " + idx, foundMsg);
}
}
// This seems weird to do the same thing twice, this is valid. We need to test whether listeners are replaced....
@Test
public void testLog4jWatcherRepeat() {
LogWatcher watcher = LogWatcher.newRegisteredLogWatcher(config, null);
assertEquals(watcher.getLastEvent(), -1);
log.warn("This is a test message");
assertTrue(watcher.getLastEvent() > -1);
SolrDocumentList events = watcher.getHistory(-1, new AtomicBoolean());
assertEquals(events.size(), 1);
SolrDocument event = events.get(0);
assertEquals(event.get("logger"), "org.apache.solr.logging.TestLogWatcher");
assertEquals(event.get("message"), "This is a test message");
}
}

View File

@ -16,10 +16,10 @@
*/
package org.apache.solr.schema;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
public class UUIDFieldTest extends LuceneTestCase {
public class UUIDFieldTest extends SolrTestCaseJ4 {
public void testToInternal() {
boolean ok = false;
UUIDField uuidfield = new UUIDField();

View File

@ -17,10 +17,10 @@
package org.apache.solr.search;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.SolrTestCaseJ4;
import org.junit.Test;
public class SignificantTermsQParserPluginTest extends LuceneTestCase {
public class SignificantTermsQParserPluginTest extends SolrTestCaseJ4 {
/**
* Test the backwards compatibility for a typo in the SignificantTermsQParserPlugin. It will fail if the backwards

View File

@ -423,6 +423,53 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
,"*//doc[1]/str[.='120']"
,"*//doc[2]/str[.='121']"
);
// test a query that doesn't specify nested query val
assertQEx("Should fail because of missing qq",
"Missing param qq while parsing function 'query($qq)'",
req("q", "*:*", "fq","id:120 OR id:121", "defType","edismax", "boost","query($qq)"),
SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail because of missing sortfunc in sort",
"Can't determine a Sort Order (asc or desc) in sort spec '{!func v=$sortfunc} desc'",
req("q", "*:*", "fq","id:120 OR id:121", "sort","{!func v=$sortfunc} desc", "sortfunc","query($qq)"),
SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail because of missing qq in boost",
"Nested local params must have value in v parameter. got 'query({!dismax v=$qq})",
req("q", "*:*", "fq","id:120 OR id:121", "defType","edismax", "boost","query({!dismax v=$qq})"),
SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail as empty value is specified for v",
"Nested function query returned null for 'query({!v=})'",
req("q", "*:*", "defType","edismax", "boost","query({!v=})"), SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail as v's value contains only spaces",
"Nested function query returned null for 'query({!v= })'",
req("q", "*:*", "defType","edismax", "boost","query({!v= })"), SolrException.ErrorCode.BAD_REQUEST
);
// no field specified in ord()
assertQEx("Should fail as no field is specified in ord func",
"Expected identifier instead of 'null' for function 'ord()'",
req("q", "*:*", "defType","edismax","boost","ord()"), SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail as no field is specified in rord func",
"Expected identifier instead of 'null' for function 'rord()'",
req("q", "*:*", "defType","edismax","boost","rord()"), SolrException.ErrorCode.BAD_REQUEST
);
// test parseFloat
assertQEx("Should fail as less args are specified for recip func",
"Expected float instead of 'null' for function 'recip(1,2)'",
req("q", "*:*","defType","edismax", "boost","recip(1,2)"), SolrException.ErrorCode.BAD_REQUEST
);
assertQEx("Should fail as invalid value is specified for recip func",
"Expected float instead of 'f' for function 'recip(1,2,3,f)'",
req("q", "*:*","defType","edismax", "boost","recip(1,2,3,f)"), SolrException.ErrorCode.BAD_REQUEST
);
// this should pass
assertQ(req("q", "*:*","defType","edismax", "boost","recip(1, 2, 3, 4)"));
}
@Test

View File

@ -16,7 +16,6 @@
*/
package org.apache.solr.util;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestRuleLimitSysouts.Limit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.util.ObjectReleaseTracker;
@ -24,7 +23,7 @@ import org.junit.Test;
@Limit(bytes=150000) // raise limit as this writes to sys err
public class TestObjectReleaseTracker extends LuceneTestCase {
public class TestObjectReleaseTracker extends SolrTestCaseJ4 {
@Test
public void testObjectReleaseTracker() {

View File

@ -23,12 +23,11 @@ import java.util.Arrays;
import org.apache.commons.io.IOUtils;
import org.apache.lucene.analysis.util.ResourceLoader;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.core.SolrResourceLoader;
import org.xml.sax.InputSource;
public class TestSystemIdResolver extends LuceneTestCase {
public class TestSystemIdResolver extends SolrTestCaseJ4 {
public void tearDown() throws Exception {
System.clearProperty("solr.allow.unsafe.resourceloading");

View File

@ -1 +0,0 @@
507e22993e3a78a3ba453e76a9756a294752648d

View File

@ -0,0 +1 @@
e2543a63086b4189fbe418d05d56633bc1a815f7

View File

@ -1 +0,0 @@
5e488f2c717964162016cd5138e2b1ea4dcc379f

View File

@ -0,0 +1 @@
afb9ef0baba766725c3733e6a2626877dba72715

View File

@ -1 +0,0 @@
bede79a3f150711634a3047985517431bf6499f2

View File

@ -0,0 +1 @@
f5e9a2ffca496057d6891a3de65128efc636e26e

View File

@ -1 +0,0 @@
e6b751e02120c08702d98750f6a80bc25343b7f5

View File

@ -0,0 +1 @@
6c2fb3f5b7cd27504726aef1b674b542a0c9cf53

View File

@ -1 +0,0 @@
9ba207b78e470fe7765ebee14f1f0336c9cbcc18

View File

@ -0,0 +1 @@
4d44e4edc4a7fb39f09b95b09f560a15976fa1ba

View File

@ -0,0 +1 @@
d11ebc03fdf773d32143e0f7ea0fc131c21311e7

View File

@ -0,0 +1,202 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright 1999-2005 The Apache Software Foundation
Licensed 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.

View File

@ -0,0 +1,17 @@
Apache Log4j
Copyright 1999-2017 Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
ResolverUtil.java
Copyright 2005-2006 Tim Fennell
Dumbster SMTP test server
Copyright 2004 Jason Paul Kitchen
TypeUtil.java
Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
picocli (http://picocli.info)
Copyright 2017 Remko Popma

View File

@ -29,6 +29,7 @@
<dependencies>
<dependency org="org.apache.logging.log4j" name="log4j-api" rev="${/org.apache.logging.log4j/log4j-api}" conf="logging"/>
<dependency org="org.apache.logging.log4j" name="log4j-core" rev="${/org.apache.logging.log4j/log4j-core}" conf="logging"/>
<dependency org="org.apache.logging.log4j" name="log4j-web" rev="${/org.apache.logging.log4j/log4j-web}" conf="logging"/>
<dependency org="org.apache.logging.log4j" name="log4j-slf4j-impl" rev="${/org.apache.logging.log4j/log4j-slf4j-impl}" conf="logging"/>
<dependency org="org.apache.logging.log4j" name="log4j-1.2-api" rev="${/org.apache.logging.log4j/log4j-1.2-api}" conf="logging"/>
<dependency org="com.lmax" name="disruptor" rev="${/com.lmax/disruptor}" conf="logging"/>

View File

@ -18,22 +18,50 @@
<!-- Use this file for logging exlusively to the console, useful for
some development tasks. Should not be used for production -->
<!-- Configuration for asynchronous logging -->
<Configuration>
<Appenders>
<Console name="STDERR" target="SYSTEM_ERR">
<PatternLayout>
<Pattern>
%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %c; %m%n
%maxLen{%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %c; %m%notEmpty{ =>%ex{short}}}{10240}%n
</Pattern>
</PatternLayout>
</Console>
</Appenders>
<Loggers>
<Logger name="org.apache.zookeeper" level="WARN"/>
<Logger name="org.apache.hadoop" level="WARN"/>
<AsyncLogger name="org.apache.zookeeper" level="WARN"/>
<AsyncLogger name="org.apache.hadoop" level="WARN"/>
<Root level="INFO">
<AsyncRoot level="INFO">
<AppenderRef ref="STDERR"/>
</Root>
</AsyncRoot>
</Loggers>
</Configuration>
<!-- Configuration for synchronous logging
there _may_ be a very small window where log messages will not be flushed
to the log file on abnormal shutdown. If even this risk is unacceptable, use
the configuration below
-->
<!--Configuration>
<Appenders>
<Console name="STDERR" target="SYSTEM_ERR">
<PatternLayout>
<Pattern>
%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %c; %m%n
</Pattern>
</PatternLayout>
</Console>
</Appenders>
<Loggers>
<Logger name="org.apache.zookeeper" level="WARN"/>
<Logger name="org.apache.hadoop" level="WARN"/>
<Root level="INFO">
<AppenderRef ref="STDERR"/>
</Root>
</Loggers>
</Configuration-->

View File

@ -16,7 +16,72 @@
limitations under the License.
-->
<!-- Configuration for asynchronous logging -->
<Configuration>
<Appenders>
<Console name="STDOUT" target="SYSTEM_OUT">
<PatternLayout>
<Pattern>
%maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
</Pattern>
</PatternLayout>
</Console>
<RollingRandomAccessFile
name="MainLogFile"
fileName="${sys:solr.log.dir}/solr.log"
filePattern="${sys:solr.log.dir}/solr.log.%i" >
<PatternLayout>
<Pattern>
%maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
</Pattern>
</PatternLayout>
<Policies>
<OnStartupTriggeringPolicy />
<SizeBasedTriggeringPolicy size="32 MB"/>
</Policies>
<DefaultRolloverStrategy max="10"/>
</RollingRandomAccessFile>
<RollingRandomAccessFile
name="SlowLogFile"
fileName="${sys:solr.log.dir}/solr_slow_requests.log"
filePattern="${sys:solr.log.dir}/solr_slow_requests.log.%i" >
<PatternLayout>
<Pattern>
%maxLen{%d{yyyy-MM-dd HH:mm:ss.SSS} %-5p (%t) [%X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%notEmpty{ =>%ex{short}}}{10240}%n
</Pattern>
</PatternLayout>
<Policies>
<OnStartupTriggeringPolicy />
<SizeBasedTriggeringPolicy size="32 MB"/>
</Policies>
<DefaultRolloverStrategy max="10"/>
</RollingRandomAccessFile>
</Appenders>
<Loggers>
<AsyncLogger name="org.apache.hadoop" level="warn"/>
<AsyncLogger name="org.apache.solr.update.LoggingInfoStream" level="off"/>
<AsyncLogger name="org.apache.zookeeper" level="warn"/>
<AsyncLogger name="org.apache.solr.core.SolrCore.SlowRequest" level="info" additivity="false">
<AppenderRef ref="SlowLogFile"/>
</AsyncLogger>
<AsyncRoot level="info">
<AppenderRef ref="MainLogFile"/>
<AppenderRef ref="STDOUT"/>
</AsyncRoot>
</Loggers>
</Configuration>
<!-- Configuration for synchronous logging
there _may_ be a very small window where log messages will not be flushed
to the log file on abnormal shutdown. If even this risk is unacceptable, use
the configuration below
-->
<!--Configuration>
<Appenders>
<Console name="STDOUT" target="SYSTEM_OUT">
@ -73,4 +138,5 @@
<AppenderRef ref="STDOUT"/>
</Root>
</Loggers>
</Configuration>
</Configuration-->

View File

@ -26,6 +26,16 @@ When planning your Solr upgrade, consider the customizations you have made to yo
Detailed steps for upgrading a Solr cluster are in the section <<upgrading-a-solr-cluster.adoc#upgrading-a-solr-cluster,Upgrading a Solr Cluster>>.
== Upgrading to 8.x Releases
=== Solr 8.1
*Logging*
Solr 8.1 changed the default log4j2 logging mode from synchronous to asynchronous. This will improve logging throughput and reduce system contention at the cost of a _slight_ chance that some logging messages may be missed in the event of abnormal Solr termination.
If even this slight risk is unacceptable, the log4j configuration file ../server/resources/log4j2.xml has the synchronous logging configuration in a commented section and can be edited to re-enable synchronous logging.
== Upgrading to 7.x Releases
=== Solr 7.6

View File

@ -348,6 +348,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
sslConfig = null;
testSolrHome = null;
StartupLoggingUtils.shutdown();
IpTables.unblockAllPorts();
LogLevel.Configurer.restoreLogLevels(savedClassLogLevels);