Merge remote-tracking branch 'origin/master'

This commit is contained in:
Noble Paul 2016-09-21 11:30:14 +05:30
commit 3acfa08aea
13 changed files with 157 additions and 219 deletions

View File

@ -38,11 +38,31 @@ Other
API Changes
New Features
Bug Fixes
Improvements
* LUCENE-7439: FuzzyQuery now matches all terms within the specified
edit distance, even if they are short terms (Mike McCandless)
Optimizations
Other
Build
* LUCENE-7292: Fix build to use "--release 8" instead of "-release 8" on
Java 9 (this changed with recent EA build b135). (Uwe Schindler)
======================= Lucene 6.2.1 =======================
API Changes
* LUCENE-7436: MinHashFilter's constructor, and some of its default
settings, should be public. (Doug Turnbull via Mike McCandless)
New Features
Bug Fixes
* LUCENE-7417: The standard Highlighter could throw an IllegalArgumentException when
@ -61,18 +81,6 @@ Bug Fixes
were restored in their original packages: LowercaseFilter and StopFilter,
as well as several utility classes. (Uwe Schindler, Mike McCandless)
Improvements
* LUCENE-7439: FuzzyQuery now matches all terms within the specified
edit distance, even if they are short terms (Mike McCandless)
Optimizations
Other
======================= Lucene 6.2.1 =======================
(No Changes)
======================= Lucene 6.2.0 =======================
API Changes

View File

@ -228,7 +228,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
"6.1.0-cfs",
"6.1.0-nocfs",
"6.2.0-cfs",
"6.2.0-nocfs"
"6.2.0-nocfs",
"6.2.1-cfs",
"6.2.1-nocfs"
};
final String[] unsupportedNames = {

View File

@ -1942,7 +1942,7 @@ ${ant.project.name}.test.dependencies=${test.classpath.list}
<sequential>
<local name="javac.release.args"/>
<condition property="javac.release.args" value="-source @{javac.release} -target @{javac.release}" else="-release @{javac.release}">
<condition property="javac.release.args" value="-source @{javac.release} -target @{javac.release}" else="--release @{javac.release}">
<equals arg1="${build.java.runtime}" arg2="1.8"/>
</condition>
<mkdir dir="@{destdir}"/>
@ -2072,7 +2072,7 @@ ${ant.project.name}.test.dependencies=${test.classpath.list}
<attribute name="linksource" default="no"/>
<sequential>
<local name="javadoc.release.args"/>
<condition property="javadoc.release.args" value="-source ${javac.release}" else="-release ${javac.release}">
<condition property="javadoc.release.args" value="-source ${javac.release}" else="--release ${javac.release}">
<equals arg1="${build.java.runtime}" arg2="1.8"/>
</condition>
<antcall target="download-java8-javadoc-packagelist"/>

View File

@ -92,40 +92,6 @@ Bug Fixes
* SOLR-9310: PeerSync fails on a node restart due to IndexFingerPrint mismatch (Pushkar Raste, noble)
* SOLR-9389: HDFS Transaction logs stay open for writes which leaks Xceivers. (Tim Owen via Mark Miller)
* SOLR-6744: fl renaming / alias of uniqueKey field generates null pointer exception in SolrCloud configuration
(Mike Drob via Tomás Fernández Löbbe)
* SOLR-9445: Admin requests are retried by CloudSolrClient and LBHttpSolrClient on failure. (shalin)
* SOLR-9439: Shard split clean up logic for older failed splits is faulty. The delete shard API
has also been made more resilient against failures resulting from non-existent cores. (shalin)
* SOLR-9430: Fix locale lookup in DIH <propertyWriter/> to use BCP47 language tags
to be consistent with other places in Solr. Language names still work for backwards
compatibility. (Uwe Schindler, Boris Steiner)
* SOLR-9188: blockUnknown property makes inter-node communication impossible (noble)
* SOLR-9455: Deleting a sub-shard in recovery state can mark parent shard as inactive. (shalin)
* SOLR-9461: DELETENODE, REPLACENODE should pass down the 'async' param to subcommands (shalin, noble)
* SOLR-9444: Fix path usage for cloud backup/restore. (Hrishikesh Gadre, Uwe Schindler, Varun Thacker)
* SOLR-9381: Snitch for freedisk uses '/' instead of 'coreRootDirectory' (Tim Owen, noble)
* SOLR-9488: Shard split can fail to write commit data on shutdown/restart causing replicas to recover
without replicating the index. This can cause data loss. (shalin)
* SOLR-9490: Fixed bugs in BoolField that caused it to erroneously return "false" for all docs depending
on usage (Colvin Cowie, Dan Fox, hossman)
* SOLR-9438: Shard split can be marked successful and sub-shard states switched to 'active' even when
one or more sub-shards replicas do not recover due to the leader crashing or restarting between the time
the replicas are created and before they can recover. This can cause data loss. (shalin)
* SOLR-9484: The modify collection API should wait for the modified properties to show up in the
cluster state. (Cao Manh Dat, shalin)
@ -134,8 +100,10 @@ Bug Fixes
* SOLR-9522: Improve error handling in ZKPropertiesWriter (Varun Thacker)
* SOLR-9512: CloudSolrClient will try and keep up with leader changes if its
state cache points to a down server (Alan Woodward, noble)
* SOLR-8080: bin/solr start script now exits with informative message if using wrong Java version (janhoy)
* SOLR-9475: bin/install_solr_service.sh script got improved detection of Linux distro, especially within
virtualized/Docker environment through parsing of /etc/*-release files. Now also supports CentOS. (janhoy)
* SOLR-9524: SolrIndexSearcher.getIndexFingerprint uses dubious synchronization (Mike Drob, noble)
@ -157,8 +125,6 @@ Optimizations
Other Changes
----------------------
* SOLR-7362: Fix TestReqParamsAPI test failures (noble, Varun Thacker)
* SOLR-9412: Add failOnMissingParams option to MacroExpander, add TestMacroExpander class.
(Jon Dorando, Christine Poerschke)
@ -184,9 +150,47 @@ Bug Fixes
* SOLR-9494: Use of {!collapse} sometimes doesn't correctly return true for Collector.needsScores(), especially when the
query was cached. This can cause an exception when 'q' is a SpanQuery or potentially others. (David Smiley)
* SOLR-9408: Fix TreeMergeOutputFormat to add timestamp metadata to a commit. SolrCloud replication relies on this.
(Jessica Cheng Mallet via Varun Thacker)
* SOLR-6744: fl renaming / alias of uniqueKey field generates null pointer exception in SolrCloud configuration
(Mike Drob via Tomás Fernández Löbbe)
* SOLR-9445: Admin requests are retried by CloudSolrClient and LBHttpSolrClient on failure. (shalin)
* SOLR-9439: Shard split clean up logic for older failed splits is faulty. The delete shard API
has also been made more resilient against failures resulting from non-existent cores. (shalin)
* SOLR-9430: Fix locale lookup in DIH <propertyWriter/> to use BCP47 language tags
to be consistent with other places in Solr. Language names still work for backwards
compatibility. (Uwe Schindler, Boris Steiner)
* SOLR-9389: HDFS Transaction logs stay open for writes which leaks Xceivers. (Tim Owen via Mark Miller)
* SOLR-9188: blockUnknown property makes inter-node communication impossible (noble)
* SOLR-9455: Deleting a sub-shard in recovery state can mark parent shard as inactive. (shalin)
* SOLR-9461: DELETENODE, REPLACENODE should pass down the 'async' param to subcommands (shalin, noble)
* SOLR-9444: Fix path usage for cloud backup/restore. (Hrishikesh Gadre, Uwe Schindler, Varun Thacker)
* SOLR-9381: Snitch for freedisk uses '/' instead of 'coreRootDirectory' (Tim Owen, noble)
* SOLR-9488: Shard split can fail to write commit data on shutdown/restart causing replicas to recover
without replicating the index. This can cause data loss. (shalin)
* SOLR-9490: Fixed bugs in BoolField that caused it to erroneously return "false" for all docs depending
on usage (Colvin Cowie, Dan Fox, hossman)
* SOLR-9438: Shard split can be marked successful and sub-shard states switched to 'active' even when
one or more sub-shards replicas do not recover due to the leader crashing or restarting between the time
the replicas are created and before they can recover. This can cause data loss. (shalin)
* SOLR-9408: Fix TreeMergeOutputFormat to add timestamp metadata to a commit. SolrCloud replication relies on this.
(Jessica Cheng Mallet via Varun Thacker)
Other Changes
----------------------
* SOLR-7362: Fix TestReqParamsAPI test failures (noble, Varun Thacker)
================== 6.2.0 ==================

View File

@ -52,20 +52,29 @@ print_usage() {
echo ""
} # end print_usage
if [ -f "/proc/version" ]; then
proc_version=`cat /proc/version`
else
proc_version=`uname -a`
proc_version=`cat /etc/*-release 2>/dev/null`
if [[ $? -gt 0 ]]; then
if [ -f "/proc/version" ]; then
proc_version=`cat /proc/version`
else
proc_version=`uname -a`
fi
fi
if [[ $proc_version == *"Debian"* ]]; then
distro=Debian
elif [[ $proc_version == *"Red Hat"* ]]; then
distro=RedHat
elif [[ $proc_version == *"CentOS"* ]]; then
distro=CentOS
elif [[ $proc_version == *"Ubuntu"* ]]; then
distro=Ubuntu
elif [[ $proc_version == *"SUSE"* ]]; then
distro=SUSE
elif [[ $proc_version == *"Darwin"* ]]; then
echo "Sorry, this script does not support macOS. You'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide."
echo "You could also try installing via Homebrew (http://brew.sh/), e.g. brew install solr"
exit 1
else
echo -e "\nERROR: Your Linux distribution ($proc_version) not supported by this script!\nYou'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide.\n" 1>&2
exit 1
@ -214,7 +223,7 @@ fi
solr_uid="`id -u "$SOLR_USER"`"
if [ $? -ne 0 ]; then
echo "Creating new user: $SOLR_USER"
if [ "$distro" == "RedHat" ]; then
if [ "$distro" == "RedHat" ] || [ "$distro" == "CentOS" ] ; then
adduser "$SOLR_USER"
elif [ "$distro" == "SUSE" ]; then
useradd -m "$SOLR_USER"
@ -316,15 +325,15 @@ find "$SOLR_VAR_DIR" -type d -print0 | xargs -0 chmod 0750
find "$SOLR_VAR_DIR" -type f -print0 | xargs -0 chmod 0640
# configure autostart of service
if [[ "$distro" == "RedHat" || "$distro" == "SUSE" ]]; then
if [[ "$distro" == "RedHat" || "$distro" == "CentOS" || "$distro" == "SUSE" ]]; then
chkconfig "$SOLR_SERVICE" on
else
update-rc.d "$SOLR_SERVICE" defaults
fi
echo "Service $SOLR_SERVICE installed."
echo "Customize Solr startup configuration in /etc/default/$SOLR_SERVICE.in.sh"
# start service
service "$SOLR_SERVICE" start
sleep 5
service "$SOLR_SERVICE" status
echo "Service $SOLR_SERVICE installed."

View File

@ -49,6 +49,9 @@ SOLR_SCRIPT="$0"
verbose=false
THIS_OS=`uname -s`
# What version of Java is required to run this version of Solr.
JAVA_VER_REQ="8" # For printing in echo
stop_all=false
# for now, we don't support running this script from cygwin due to problems
@ -116,16 +119,34 @@ else
JAVA=java
fi
# test that Java exists and is executable on this server
"$JAVA" -version >/dev/null 2>&1 || {
# test that Java exists, is executable and correct version
JAVA_VER=$("$JAVA" -version 2>&1)
if [[ $? -ne 0 ]] ; then
echo >&2 "Java not found, or an error was encountered when running java."
echo >&2 "A working Java 8 is required to run Solr!"
echo >&2 "Please install Java 8 or fix JAVA_HOME before running this script."
echo >&2 "Command that we tried: '${JAVA} -version'"
echo >&2 "A working Java $JAVA_VER_REQ JRE is required to run Solr!"
echo >&2 "Please install latest version of Java $JAVA_VER_REQ or set JAVA_HOME properly."
echo >&2 "Command that we tried: '${JAVA} -version', with response:"
echo >&2 "${JAVA_VER}"
echo
echo >&2 "Debug information:"
echo >&2 "JAVA_HOME: ${JAVA_HOME:-N/A}"
echo >&2 "Active Path:"
echo >&2 "${PATH}"
exit 1
}
else
JAVA_VER=$(echo $JAVA_VER | awk -F '"' '/version/ {print $2}')
if [[ "$JAVA_VER" < "1.$JAVA_VER_REQ" ]] ; then
echo >&2 "Your current version of Java is too old to run this version of Solr"
echo >&2 "We found version $JAVA_VER, using command '${JAVA}'"
echo >&2 "Please install latest version of Java $JAVA_VER_REQ or set JAVA_HOME properly."
echo
echo >&2 "Debug information:"
echo >&2 "JAVA_HOME: ${JAVA_HOME:-N/A}"
echo >&2 "Active Path:"
echo >&2 "${PATH}"
exit 1
fi
fi
# Select HTTP OR HTTPS related configurations
SOLR_URL_SCHEME=http

View File

@ -191,10 +191,6 @@ public class CloudSolrClient extends SolrClient {
}
}
private void invalidateCollectionState(String collection) {
collectionStateCache.remove(collection);
}
/**
* Create a new client object that connects to Zookeeper and is always aware
* of the SolrCloud state. If there is a fully redundant Zookeeper quorum and
@ -725,29 +721,23 @@ public class CloudSolrClient extends SolrClient {
long start = System.nanoTime();
if (parallelUpdates) {
final Map<String, Future<LBHttpSolrClient.Rsp>> responseFutures = new HashMap<>(routes.size());
final Map<String, Future<NamedList<?>>> responseFutures = new HashMap<>(routes.size());
for (final Map.Entry<String, LBHttpSolrClient.Req> entry : routes.entrySet()) {
final String url = entry.getKey();
final LBHttpSolrClient.Req lbRequest = entry.getValue();
try {
MDC.put("CloudSolrClient.url", url);
responseFutures.put(url, threadPool.submit(() -> lbClient.request(lbRequest)));
responseFutures.put(url, threadPool.submit(() -> lbClient.request(lbRequest).getResponse()));
} finally {
MDC.remove("CloudSolrClient.url");
}
}
for (final Map.Entry<String, Future<LBHttpSolrClient.Rsp>> entry: responseFutures.entrySet()) {
for (final Map.Entry<String, Future<NamedList<?>>> entry: responseFutures.entrySet()) {
final String url = entry.getKey();
final Future<LBHttpSolrClient.Rsp> responseFuture = entry.getValue();
final Future<NamedList<?>> responseFuture = entry.getValue();
try {
LBHttpSolrClient.Rsp response = responseFuture.get();
shardResponses.add(url, response.getResponse());
if (url.startsWith(response.getServer())) { // startsWith to deal with stray trailing slashes
// we didn't hit our first-preference server, which means that our cached
// collection state is no longer valid
invalidateCollectionState(collection);
}
shardResponses.add(url, responseFuture.get());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
@ -770,14 +760,8 @@ public class CloudSolrClient extends SolrClient {
String url = entry.getKey();
LBHttpSolrClient.Req lbRequest = entry.getValue();
try {
LBHttpSolrClient.Rsp response = lbClient.request(lbRequest);
NamedList<Object> rsp = response.getResponse();
NamedList<Object> rsp = lbClient.request(lbRequest).getResponse();
shardResponses.add(url, rsp);
if (response.getServer().equals(url) == false) {
// we didn't hit our first-preference server, which means that our cached
// collection state is no longer valid
invalidateCollectionState(collection);
}
} catch (Exception e) {
if(e instanceof SolrException) {
throw (SolrException) e;
@ -828,7 +812,10 @@ public class CloudSolrClient extends SolrClient {
private Map<String,List<String>> buildUrlMap(DocCollection col) {
Map<String, List<String>> urlMap = new HashMap<>();
for (Slice slice : col) {
Collection<Slice> slices = col.getActiveSlices();
Iterator<Slice> sliceIterator = slices.iterator();
while (sliceIterator.hasNext()) {
Slice slice = sliceIterator.next();
String name = slice.getName();
List<String> urls = new ArrayList<>();
Replica leader = slice.getLeader();
@ -839,15 +826,19 @@ public class CloudSolrClient extends SolrClient {
// take unoptimized general path - we cannot find a leader yet
return null;
}
urls.add(leader.getCoreUrl());
for (Replica replica : slice.getReplicas()) {
if (!replica.getNodeName().equals(leader.getNodeName()) &&
!replica.getName().equals(leader.getName())) {
urls.add(replica.getCoreUrl());
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(leader);
String url = zkProps.getCoreUrl();
urls.add(url);
if (!directUpdatesToLeadersOnly) {
for (Replica replica : slice.getReplicas()) {
if (!replica.getNodeName().equals(leader.getNodeName()) &&
!replica.getName().equals(leader.getName())) {
ZkCoreNodeProps zkProps1 = new ZkCoreNodeProps(replica);
String url1 = zkProps1.getCoreUrl();
urls.add(url1);
}
}
}
urlMap.put(name, urls);
}
return urlMap;

View File

@ -247,19 +247,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
return activeShards == expectedShards;
}
/**
* Check that all shards in a collection have a leader
*/
public static boolean isUpdateable(Set<String> liveNodes, DocCollection collectionState, int expectedShards) {
int updateableShards = 0;
for (Slice slice : collectionState) {
Replica leader = slice.getLeader();
if (leader != null && leader.isActive(liveNodes))
updateableShards++;
}
return updateableShards == expectedShards;
}
@Override
public Iterator<Slice> iterator() {
return slices.values().iterator();

View File

@ -156,6 +156,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
// Test single threaded routed updates for UpdateRequest
NamedList<Object> response = cluster.getSolrClient().request(request, COLLECTION);
if (cluster.getSolrClient().isDirectUpdatesToLeadersOnly()) {
checkSingleServer(response);
}
CloudSolrClient.RouteResponse rr = (CloudSolrClient.RouteResponse) response;
Map<String,LBHttpSolrClient.Req> routes = rr.getRoutes();
Iterator<Map.Entry<String,LBHttpSolrClient.Req>> it = routes.entrySet()
@ -184,6 +187,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
.deleteById("0")
.deleteById("2")
.commit(cluster.getSolrClient(), COLLECTION);
if (cluster.getSolrClient().isDirectUpdatesToLeadersOnly()) {
checkSingleServer(uResponse.getResponse());
}
QueryResponse qResponse = cluster.getSolrClient().query(COLLECTION, new SolrQuery("*:*"));
SolrDocumentList docs = qResponse.getResults();
@ -194,6 +200,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
threadedClient.setParallelUpdates(true);
threadedClient.setDefaultCollection(COLLECTION);
response = threadedClient.request(request);
if (threadedClient.isDirectUpdatesToLeadersOnly()) {
checkSingleServer(response);
}
rr = (CloudSolrClient.RouteResponse) response;
routes = rr.getRoutes();
it = routes.entrySet()
@ -599,4 +608,16 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
}
}
private static void checkSingleServer(NamedList<Object> response) {
final CloudSolrClient.RouteResponse rr = (CloudSolrClient.RouteResponse) response;
final Map<String,LBHttpSolrClient.Req> routes = rr.getRoutes();
final Iterator<Map.Entry<String,LBHttpSolrClient.Req>> it =
routes.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<String,LBHttpSolrClient.Req> entry = it.next();
assertEquals("wrong number of servers: "+entry.getValue().getServers(),
1, entry.getValue().getServers().size());
}
}
}

View File

@ -1,81 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.impl;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.DocCollection;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestCloudSolrClientStateCacheing extends SolrCloudTestCase {
private final String id = "id";
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(4)
.addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
.configure();
}
@Test
public void testCacheInvalidationOnLeaderChange() throws Exception {
final String collectionName = "cacheInvalidation";
try (CloudSolrClient solrClient = new CloudSolrClient.Builder()
.withZkHost(cluster.getZkServer().getZkAddress())
.sendDirectUpdatesToShardLeadersOnly()
.build()) {
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
.process(solrClient);
// send one update that will populate the client's cluster state cache
new UpdateRequest()
.add(id, "0", "a_t", "hello1")
.add(id, "2", "a_t", "hello2")
.add(id, "3", "a_t", "hello2")
.commit(solrClient, collectionName);
// take down a leader node
JettySolrRunner leaderJetty = cluster.getLeaderJetty(collectionName, "shard1");
leaderJetty.stop();
// wait for a new leader to be elected
solrClient.waitForState(collectionName, DEFAULT_TIMEOUT, TimeUnit.SECONDS,
(n, c) -> DocCollection.isUpdateable(n, c, 2));
// send another update - this should still succeed, even though the client's
// cached leader will be incorrect
new UpdateRequest()
.add(id, "4", "a_t", "hello1")
.add(id, "5", "a_t", "hello2")
.add(id, "6", "a_t", "hello2")
.commit(solrClient, collectionName);
}
}
}

View File

@ -46,8 +46,6 @@ import org.apache.solr.client.solrj.embedded.SSLConfig;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkStateReader;
@ -503,26 +501,4 @@ public class MiniSolrCloudCluster {
}
return ok ? null : parsed;
}
/**
* Get the Jetty that a particular Replica is located on
*/
public JettySolrRunner getReplicaJetty(Replica replica) {
for (JettySolrRunner jetty : jettys) {
if (replica.getCoreUrl().startsWith(jetty.getBaseUrl().toString()))
return jetty;
}
throw new IllegalStateException("No jetty found for replica with core url " + replica.getCoreUrl());
}
/**
* Get the Jetty that the leader of a particular collection shard is located on
*/
public JettySolrRunner getLeaderJetty(String collectionName, String shard) {
DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
Replica leader = collectionState.getLeader(shard);
if (leader == null)
throw new IllegalStateException("No leader for shard " + shard);
return getReplicaJetty(leader);
}
}