mirror of https://github.com/apache/lucene.git
SOLR-12182: Don't persist base_url in ZK as the scheme is variable, compute from node_name instead (#2010)
This commit is contained in:
parent
cb5ba42bd7
commit
a0492840ee
|
@ -156,6 +156,9 @@ Bug Fixes
|
||||||
* SOLR-14546: Fix for a relatively hard to hit issue in OverseerTaskProcessor that could lead to out of order execution
|
* SOLR-14546: Fix for a relatively hard to hit issue in OverseerTaskProcessor that could lead to out of order execution
|
||||||
of Collection API tasks competing for a lock (Ilan Ginzburg).
|
of Collection API tasks competing for a lock (Ilan Ginzburg).
|
||||||
|
|
||||||
|
* SOLR-12182: Don't persist base_url in ZK as the URL scheme is variable, compute from node_name instead when reading
|
||||||
|
state back from ZK. (Timothy Potter)
|
||||||
|
|
||||||
================== 8.8.0 ==================
|
================== 8.8.0 ==================
|
||||||
|
|
||||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.SolrZkClient;
|
import org.apache.solr.common.cloud.SolrZkClient;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.core.CoreContainer;
|
import org.apache.solr.core.CoreContainer;
|
||||||
import org.apache.solr.core.CoreDescriptor;
|
import org.apache.solr.core.CoreDescriptor;
|
||||||
import org.apache.solr.core.SolrResourceLoader;
|
import org.apache.solr.core.SolrResourceLoader;
|
||||||
|
@ -77,7 +76,7 @@ public class CloudUtil {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
|
|
||||||
String cnn = replica.getName();
|
String cnn = replica.getName();
|
||||||
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
|
String baseUrl = replica.getBaseUrl();
|
||||||
log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
|
log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
|
||||||
|
|
||||||
if (thisCnn != null && thisCnn.equals(cnn)
|
if (thisCnn != null && thisCnn.equals(cnn)
|
||||||
|
|
|
@ -344,16 +344,11 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
||||||
// though
|
// though
|
||||||
try {
|
try {
|
||||||
CloudDescriptor cloudDesc = this.coreDescriptor.getCloudDescriptor();
|
CloudDescriptor cloudDesc = this.coreDescriptor.getCloudDescriptor();
|
||||||
ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(
|
ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(cloudDesc.getCollectionName(), cloudDesc.getShardId());
|
||||||
cloudDesc.getCollectionName(), cloudDesc.getShardId());
|
final String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderprops);
|
||||||
final String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
|
final String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
||||||
final String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
|
|
||||||
|
|
||||||
String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName);
|
boolean isLeader = ourUrl.equals(leaderUrl); // TODO: We can probably delete most of this code if we say this
|
||||||
|
|
||||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
|
||||||
|
|
||||||
boolean isLeader = leaderUrl.equals(ourUrl); // TODO: We can probably delete most of this code if we say this
|
|
||||||
// strategy can only be used for pull replicas
|
// strategy can only be used for pull replicas
|
||||||
if (isLeader && !cloudDesc.isLeader()) {
|
if (isLeader && !cloudDesc.isLeader()) {
|
||||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Cloud state still says we are leader.");
|
throw new SolrException(ErrorCode.SERVER_ERROR, "Cloud state still says we are leader.");
|
||||||
|
|
|
@ -165,7 +165,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
||||||
zkStateReader.getClusterState().getCollection(collection).getSlice(shardId).getReplicas().size() < 2) {
|
zkStateReader.getClusterState().getCollection(collection).getSlice(shardId).getReplicas().size() < 2) {
|
||||||
Replica leader = zkStateReader.getLeader(collection, shardId);
|
Replica leader = zkStateReader.getLeader(collection, shardId);
|
||||||
if (leader != null
|
if (leader != null
|
||||||
&& leader.getBaseUrl().equals(leaderProps.get(ZkStateReader.BASE_URL_PROP))
|
&& leader.getNodeName().equals(leaderProps.get(ZkStateReader.NODE_NAME_PROP))
|
||||||
&& leader.getCoreName().equals(leaderProps.get(ZkStateReader.CORE_NAME_PROP))) {
|
&& leader.getCoreName().equals(leaderProps.get(ZkStateReader.CORE_NAME_PROP))) {
|
||||||
isAlreadyLeader = true;
|
isAlreadyLeader = true;
|
||||||
}
|
}
|
||||||
|
@ -174,7 +174,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
||||||
ZkNodeProps m = ZkNodeProps.fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
|
ZkNodeProps m = ZkNodeProps.fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
|
||||||
ZkStateReader.SHARD_ID_PROP, shardId,
|
ZkStateReader.SHARD_ID_PROP, shardId,
|
||||||
ZkStateReader.COLLECTION_PROP, collection,
|
ZkStateReader.COLLECTION_PROP, collection,
|
||||||
ZkStateReader.BASE_URL_PROP, leaderProps.get(ZkStateReader.BASE_URL_PROP),
|
ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP),
|
||||||
ZkStateReader.CORE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NAME_PROP),
|
ZkStateReader.CORE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NAME_PROP),
|
||||||
ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
|
ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
|
||||||
assert zkController != null;
|
assert zkController != null;
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.solr.cloud;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UnsupportedEncodingException;
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.net.InetAddress;
|
import java.net.InetAddress;
|
||||||
import java.net.NetworkInterface;
|
import java.net.NetworkInterface;
|
||||||
|
@ -103,7 +102,6 @@ import org.apache.zookeeper.data.Stat;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
|
||||||
|
@ -924,6 +922,8 @@ public class ZkController implements Closeable {
|
||||||
try {
|
try {
|
||||||
createClusterZkNodes(zkClient);
|
createClusterZkNodes(zkClient);
|
||||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
|
// this must happen after zkStateReader has initialized the cluster props
|
||||||
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
|
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
|
||||||
|
|
||||||
checkForExistingEphemeralNode();
|
checkForExistingEphemeralNode();
|
||||||
|
@ -1103,6 +1103,7 @@ public class ZkController implements Closeable {
|
||||||
if (zkRunOnly) {
|
if (zkRunOnly) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
String nodeName = getNodeName();
|
String nodeName = getNodeName();
|
||||||
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
|
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
|
||||||
log.info("Register node as live in ZooKeeper:{}", nodePath);
|
log.info("Register node as live in ZooKeeper:{}", nodePath);
|
||||||
|
@ -1401,8 +1402,7 @@ public class ZkController implements Closeable {
|
||||||
byte[] data = zkClient.getData(
|
byte[] data = zkClient.getData(
|
||||||
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
|
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
|
||||||
true);
|
true);
|
||||||
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(
|
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
|
||||||
ZkNodeProps.load(data));
|
|
||||||
return leaderProps;
|
return leaderProps;
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
throw e;
|
throw e;
|
||||||
|
@ -1442,7 +1442,6 @@ public class ZkController implements Closeable {
|
||||||
|
|
||||||
Map<String, Object> props = new HashMap<>();
|
Map<String, Object> props = new HashMap<>();
|
||||||
// we only put a subset of props into the leader node
|
// we only put a subset of props into the leader node
|
||||||
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
|
|
||||||
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
||||||
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
||||||
props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||||
|
@ -1541,7 +1540,6 @@ public class ZkController implements Closeable {
|
||||||
Map<String,Object> props = new HashMap<>();
|
Map<String,Object> props = new HashMap<>();
|
||||||
props.put(Overseer.QUEUE_OPERATION, "state");
|
props.put(Overseer.QUEUE_OPERATION, "state");
|
||||||
props.put(ZkStateReader.STATE_PROP, state.toString());
|
props.put(ZkStateReader.STATE_PROP, state.toString());
|
||||||
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
|
|
||||||
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
||||||
props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
|
props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
|
||||||
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
||||||
|
@ -1646,7 +1644,6 @@ public class ZkController implements Closeable {
|
||||||
OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
|
OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
|
||||||
ZkStateReader.NODE_NAME_PROP, getNodeName(),
|
ZkStateReader.NODE_NAME_PROP, getNodeName(),
|
||||||
ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
|
ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
|
||||||
ZkStateReader.BASE_URL_PROP, getBaseUrl(),
|
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||||
overseerJobQueue.offer(Utils.toJSON(m));
|
overseerJobQueue.offer(Utils.toJSON(m));
|
||||||
}
|
}
|
||||||
|
@ -2131,12 +2128,7 @@ public class ZkController implements Closeable {
|
||||||
static String generateNodeName(final String hostName,
|
static String generateNodeName(final String hostName,
|
||||||
final String hostPort,
|
final String hostPort,
|
||||||
final String hostContext) {
|
final String hostContext) {
|
||||||
try {
|
return hostName + ':' + hostPort + '_' + URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), StandardCharsets.UTF_8);
|
||||||
return hostName + ':' + hostPort + '_' +
|
|
||||||
URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), "UTF-8");
|
|
||||||
} catch (UnsupportedEncodingException e) {
|
|
||||||
throw new Error("JVM Does not seem to support UTF-8", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2197,7 +2189,6 @@ public class ZkController implements Closeable {
|
||||||
String coreNodeName = params.get(CORE_NODE_NAME_PROP);
|
String coreNodeName = params.get(CORE_NODE_NAME_PROP);
|
||||||
String coreName = params.get(CORE_NAME_PROP);
|
String coreName = params.get(CORE_NAME_PROP);
|
||||||
String electionNode = params.get(ELECTION_NODE_PROP);
|
String electionNode = params.get(ELECTION_NODE_PROP);
|
||||||
String baseUrl = params.get(BASE_URL_PROP);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
MDCLoggingContext.setCoreDescriptor(cc, cc.getCoreDescriptor(coreName));
|
MDCLoggingContext.setCoreDescriptor(cc, cc.getCoreDescriptor(coreName));
|
||||||
|
@ -2209,7 +2200,8 @@ public class ZkController implements Closeable {
|
||||||
ElectionContext prevContext = electionContexts.get(contextKey);
|
ElectionContext prevContext = electionContexts.get(contextKey);
|
||||||
if (prevContext != null) prevContext.cancelElection();
|
if (prevContext != null) prevContext.cancelElection();
|
||||||
|
|
||||||
ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
|
String ourUrl = ZkCoreNodeProps.getCoreUrl(UrlScheme.INSTANCE.getBaseUrlForNodeName(getNodeName()), coreName);
|
||||||
|
ZkNodeProps zkProps = new ZkNodeProps(CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
|
||||||
|
|
||||||
LeaderElector elect = ((ShardLeaderElectionContextBase) prevContext).getLeaderElector();
|
LeaderElector elect = ((ShardLeaderElectionContextBase) prevContext).getLeaderElector();
|
||||||
ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
|
ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
|
||||||
|
@ -2225,7 +2217,6 @@ public class ZkController implements Closeable {
|
||||||
Replica.Type replicaType = core.getCoreDescriptor().getCloudDescriptor().getReplicaType();
|
Replica.Type replicaType = core.getCoreDescriptor().getCloudDescriptor().getReplicaType();
|
||||||
if (replicaType == Type.TLOG) {
|
if (replicaType == Type.TLOG) {
|
||||||
String leaderUrl = getLeader(core.getCoreDescriptor().getCloudDescriptor(), cloudConfig.getLeaderVoteWait());
|
String leaderUrl = getLeader(core.getCoreDescriptor().getCloudDescriptor(), cloudConfig.getLeaderVoteWait());
|
||||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
|
||||||
if (!leaderUrl.equals(ourUrl)) {
|
if (!leaderUrl.equals(ourUrl)) {
|
||||||
// restart the replication thread to ensure the replication is running in each new replica
|
// restart the replication thread to ensure the replication is running in each new replica
|
||||||
// especially if previous role is "leader" (i.e., no replication thread)
|
// especially if previous role is "leader" (i.e., no replication thread)
|
||||||
|
|
|
@ -210,7 +210,6 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
|
||||||
ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
|
ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
|
||||||
ZkStateReader.CORE_NAME_PROP, createReplica.coreName,
|
ZkStateReader.CORE_NAME_PROP, createReplica.coreName,
|
||||||
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
||||||
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(createReplica.node),
|
|
||||||
ZkStateReader.NODE_NAME_PROP, createReplica.node,
|
ZkStateReader.NODE_NAME_PROP, createReplica.node,
|
||||||
ZkStateReader.REPLICA_TYPE, createReplica.replicaType.name());
|
ZkStateReader.REPLICA_TYPE, createReplica.replicaType.name());
|
||||||
if (createReplica.coreNodeName != null) {
|
if (createReplica.coreNodeName != null) {
|
||||||
|
|
|
@ -208,7 +208,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
|
||||||
ZkStateReader.SHARD_ID_PROP, replicaPosition.shard,
|
ZkStateReader.SHARD_ID_PROP, replicaPosition.shard,
|
||||||
ZkStateReader.CORE_NAME_PROP, coreName,
|
ZkStateReader.CORE_NAME_PROP, coreName,
|
||||||
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
||||||
ZkStateReader.BASE_URL_PROP, baseUrl,
|
|
||||||
ZkStateReader.NODE_NAME_PROP, nodeName,
|
ZkStateReader.NODE_NAME_PROP, nodeName,
|
||||||
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
|
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
|
||||||
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
|
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
|
||||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.solr.common.cloud.DocRouter;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.SolrZkClient;
|
import org.apache.solr.common.cloud.SolrZkClient;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkConfigManager;
|
import org.apache.solr.common.cloud.ZkConfigManager;
|
||||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
|
@ -91,11 +92,11 @@ import org.apache.zookeeper.KeeperException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
|
||||||
|
@ -304,7 +305,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
|
||||||
private void processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
|
private void processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, CORE_NAME_PROP, ELECTION_NODE_PROP,
|
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, CORE_NAME_PROP, ELECTION_NODE_PROP,
|
||||||
CORE_NODE_NAME_PROP, BASE_URL_PROP, REJOIN_AT_HEAD_PROP);
|
CORE_NODE_NAME_PROP, NODE_NAME_PROP, REJOIN_AT_HEAD_PROP);
|
||||||
|
|
||||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||||
params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
|
params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
|
||||||
|
@ -314,9 +315,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
|
||||||
params.set(CORE_NAME_PROP, message.getStr(CORE_NAME_PROP));
|
params.set(CORE_NAME_PROP, message.getStr(CORE_NAME_PROP));
|
||||||
params.set(CORE_NODE_NAME_PROP, message.getStr(CORE_NODE_NAME_PROP));
|
params.set(CORE_NODE_NAME_PROP, message.getStr(CORE_NODE_NAME_PROP));
|
||||||
params.set(ELECTION_NODE_PROP, message.getStr(ELECTION_NODE_PROP));
|
params.set(ELECTION_NODE_PROP, message.getStr(ELECTION_NODE_PROP));
|
||||||
params.set(BASE_URL_PROP, message.getStr(BASE_URL_PROP));
|
params.set(NODE_NAME_PROP, message.getStr(NODE_NAME_PROP));
|
||||||
|
|
||||||
String baseUrl = message.getStr(BASE_URL_PROP);
|
String baseUrl = UrlScheme.INSTANCE.getBaseUrlForNodeName(message.getStr(NODE_NAME_PROP));
|
||||||
ShardRequest sreq = new ShardRequest();
|
ShardRequest sreq = new ShardRequest();
|
||||||
sreq.nodeName = message.getStr(ZkStateReader.CORE_NAME_PROP);
|
sreq.nodeName = message.getStr(ZkStateReader.CORE_NAME_PROP);
|
||||||
// yes, they must use same admin handler path everywhere...
|
// yes, they must use same admin handler path everywhere...
|
||||||
|
@ -426,10 +427,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
|
||||||
ZkNodeProps m = new ZkNodeProps(
|
ZkNodeProps m = new ZkNodeProps(
|
||||||
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
|
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
|
||||||
ZkStateReader.CORE_NAME_PROP, core,
|
ZkStateReader.CORE_NAME_PROP, core,
|
||||||
ZkStateReader.NODE_NAME_PROP, replica.getStr(ZkStateReader.NODE_NAME_PROP),
|
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
|
||||||
ZkStateReader.COLLECTION_PROP, collectionName,
|
ZkStateReader.COLLECTION_PROP, collectionName,
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, replicaName,
|
ZkStateReader.CORE_NODE_NAME_PROP, replicaName);
|
||||||
ZkStateReader.BASE_URL_PROP, replica.getStr(ZkStateReader.BASE_URL_PROP));
|
|
||||||
overseer.offerStateUpdate(Utils.toJSON(m));
|
overseer.offerStateUpdate(Utils.toJSON(m));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -457,7 +457,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
|
||||||
ZkStateReader.CORE_NAME_PROP, solrCoreName,
|
ZkStateReader.CORE_NAME_PROP, solrCoreName,
|
||||||
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
|
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
|
||||||
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
|
||||||
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
|
|
||||||
ZkStateReader.NODE_NAME_PROP, subShardNodeName,
|
ZkStateReader.NODE_NAME_PROP, subShardNodeName,
|
||||||
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
|
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
|
||||||
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
|
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
|
||||||
|
|
|
@ -77,7 +77,6 @@ public class SliceMutator {
|
||||||
Replica replica = new Replica(coreNodeName,
|
Replica replica = new Replica(coreNodeName,
|
||||||
makeMap(
|
makeMap(
|
||||||
ZkStateReader.CORE_NAME_PROP, message.getStr(ZkStateReader.CORE_NAME_PROP),
|
ZkStateReader.CORE_NAME_PROP, message.getStr(ZkStateReader.CORE_NAME_PROP),
|
||||||
ZkStateReader.BASE_URL_PROP, message.getStr(ZkStateReader.BASE_URL_PROP),
|
|
||||||
ZkStateReader.STATE_PROP, message.getStr(ZkStateReader.STATE_PROP),
|
ZkStateReader.STATE_PROP, message.getStr(ZkStateReader.STATE_PROP),
|
||||||
ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
|
ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
|
||||||
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice);
|
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice);
|
||||||
|
@ -87,7 +86,6 @@ public class SliceMutator {
|
||||||
public ZkWriteCommand removeReplica(ClusterState clusterState, ZkNodeProps message) {
|
public ZkWriteCommand removeReplica(ClusterState clusterState, ZkNodeProps message) {
|
||||||
final String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
|
final String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
|
||||||
final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
|
final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
|
||||||
final String baseUrl = message.getStr(ZkStateReader.BASE_URL_PROP);
|
|
||||||
if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
|
if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
|
||||||
|
|
||||||
DocCollection coll = clusterState.getCollectionOrNull(collection);
|
DocCollection coll = clusterState.getCollectionOrNull(collection);
|
||||||
|
@ -97,10 +95,10 @@ public class SliceMutator {
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<String, Slice> newSlices = new LinkedHashMap<>();
|
Map<String, Slice> newSlices = new LinkedHashMap<>();
|
||||||
|
final String nodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
Replica replica = slice.getReplica(cnn);
|
Replica replica = slice.getReplica(cnn);
|
||||||
if (replica != null && (baseUrl == null || baseUrl.equals(replica.getBaseUrl()))) {
|
if (replica != null && (nodeName == null || nodeName.equals(replica.getNodeName()))) {
|
||||||
Map<String, Replica> newReplicas = slice.getReplicasCopy();
|
Map<String, Replica> newReplicas = slice.getReplicasCopy();
|
||||||
newReplicas.remove(cnn);
|
newReplicas.remove(cnn);
|
||||||
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(),collection);
|
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(),collection);
|
||||||
|
@ -112,15 +110,7 @@ public class SliceMutator {
|
||||||
}
|
}
|
||||||
|
|
||||||
public ZkWriteCommand setShardLeader(ClusterState clusterState, ZkNodeProps message) {
|
public ZkWriteCommand setShardLeader(ClusterState clusterState, ZkNodeProps message) {
|
||||||
StringBuilder sb = new StringBuilder();
|
String leaderUrl = ZkCoreNodeProps.getCoreUrl(message);
|
||||||
String baseUrl = message.getStr(ZkStateReader.BASE_URL_PROP);
|
|
||||||
String coreName = message.getStr(ZkStateReader.CORE_NAME_PROP);
|
|
||||||
sb.append(baseUrl);
|
|
||||||
if (baseUrl != null && !baseUrl.endsWith("/")) sb.append("/");
|
|
||||||
sb.append(coreName == null ? "" : coreName);
|
|
||||||
if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
|
|
||||||
String leaderUrl = sb.length() > 0 ? sb.toString() : null;
|
|
||||||
|
|
||||||
String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
|
String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
|
||||||
String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
|
String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
|
||||||
DocCollection coll = clusterState.getCollectionOrNull(collectionName);
|
DocCollection coll = clusterState.getCollectionOrNull(collectionName);
|
||||||
|
@ -137,7 +127,7 @@ public class SliceMutator {
|
||||||
final Map<String, Replica> newReplicas = new LinkedHashMap<>();
|
final Map<String, Replica> newReplicas = new LinkedHashMap<>();
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
// TODO: this should only be calculated once and cached somewhere?
|
// TODO: this should only be calculated once and cached somewhere?
|
||||||
String coreURL = ZkCoreNodeProps.getCoreUrl(replica.getStr(ZkStateReader.BASE_URL_PROP), replica.getStr(ZkStateReader.CORE_NAME_PROP));
|
String coreURL = ZkCoreNodeProps.getCoreUrl(replica.getBaseUrl(), replica.getStr(ZkStateReader.CORE_NAME_PROP));
|
||||||
|
|
||||||
if (replica == oldLeader && !coreURL.equals(leaderUrl)) {
|
if (replica == oldLeader && !coreURL.equals(leaderUrl)) {
|
||||||
replica = new ReplicaMutator(cloudManager).unsetLeader(replica);
|
replica = new ReplicaMutator(cloudManager).unsetLeader(replica);
|
||||||
|
|
|
@ -54,7 +54,6 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
|
import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
|
||||||
import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
|
import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
|
* The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
|
||||||
|
@ -205,7 +204,7 @@ public class BlobRepository {
|
||||||
*/
|
*/
|
||||||
ByteBuffer fetchBlob(String key) {
|
ByteBuffer fetchBlob(String key) {
|
||||||
Replica replica = getSystemCollReplica();
|
Replica replica = getSystemCollReplica();
|
||||||
String url = replica.getStr(BASE_URL_PROP) + "/" + CollectionAdminParams.SYSTEM_COLL + "/blob/" + key + "?wt=filestream";
|
String url = replica.getBaseUrl() + "/" + CollectionAdminParams.SYSTEM_COLL + "/blob/" + key + "?wt=filestream";
|
||||||
return fetchFromUrl(key, url);
|
return fetchFromUrl(key, url);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -36,16 +36,22 @@ import org.apache.solr.common.AlreadyClosedException;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.cloud.ClusterProperties;
|
import org.apache.solr.common.cloud.ClusterProperties;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkConfigManager;
|
import org.apache.solr.common.cloud.ZkConfigManager;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.common.cloud.ZooKeeperException;
|
import org.apache.solr.common.cloud.ZooKeeperException;
|
||||||
import org.apache.solr.common.util.ExecutorUtil;
|
import org.apache.solr.common.util.ExecutorUtil;
|
||||||
import org.apache.solr.logging.MDCLoggingContext;
|
|
||||||
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
||||||
|
import org.apache.solr.logging.MDCLoggingContext;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTP;
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTPS;
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTPS_PORT_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.URL_SCHEME;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used by {@link CoreContainer} to hold ZooKeeper / SolrCloud info, especially {@link ZkController}.
|
* Used by {@link CoreContainer} to hold ZooKeeper / SolrCloud info, especially {@link ZkController}.
|
||||||
* Mainly it does some ZK initialization, and ensures a loading core registers in ZK.
|
* Mainly it does some ZK initialization, and ensures a loading core registers in ZK.
|
||||||
|
@ -131,10 +137,14 @@ public class ZkContainer {
|
||||||
ZkController zkController = new ZkController(cc, zookeeperHost, zkClientConnectTimeout, config, descriptorsSupplier);
|
ZkController zkController = new ZkController(cc, zookeeperHost, zkClientConnectTimeout, config, descriptorsSupplier);
|
||||||
|
|
||||||
if (zkRun != null) {
|
if (zkRun != null) {
|
||||||
if (StringUtils.isNotEmpty(System.getProperty("solr.jetty.https.port"))) {
|
if (StringUtils.isNotEmpty(System.getProperty(HTTPS_PORT_PROP))) {
|
||||||
// Embedded ZK and probably running with SSL
|
// Embedded ZK and probably running with SSL
|
||||||
new ClusterProperties(zkController.getZkClient()).setClusterProperty(ZkStateReader.URL_SCHEME, "https");
|
new ClusterProperties(zkController.getZkClient()).setClusterProperty(ZkStateReader.URL_SCHEME, HTTPS);
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(HTTPS);
|
||||||
|
} else {
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(System.getProperty(URL_SCHEME, HTTP));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (zkServer.getServers().size() > 1 && confDir == null && boostrapConf == false) {
|
if (zkServer.getServers().size() > 1 && confDir == null && boostrapConf == false) {
|
||||||
// we are part of an ensemble and we are not uploading the config - pause to give the config time
|
// we are part of an ensemble and we are not uploading the config - pause to give the config time
|
||||||
// to get up
|
// to get up
|
||||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.DocRouter;
|
import org.apache.solr.common.cloud.DocRouter;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.common.params.ShardParams;
|
import org.apache.solr.common.params.ShardParams;
|
||||||
|
@ -196,7 +197,7 @@ public class ClusterStatus {
|
||||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
|
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
|
||||||
}
|
}
|
||||||
if (requestedShards == null || requestedShards.isEmpty()) {
|
if (requestedShards == null || requestedShards.isEmpty()) {
|
||||||
return collection;
|
return postProcessCollectionJSON(collection);
|
||||||
} else {
|
} else {
|
||||||
Map<String, Object> shards = (Map<String, Object>) collection.get("shards");
|
Map<String, Object> shards = (Map<String, Object>) collection.get("shards");
|
||||||
Map<String, Object> selected = new HashMap<>();
|
Map<String, Object> selected = new HashMap<>();
|
||||||
|
@ -207,12 +208,10 @@ public class ClusterStatus {
|
||||||
selected.put(selectedShard, shards.get(selectedShard));
|
selected.put(selectedShard, shards.get(selectedShard));
|
||||||
collection.put("shards", selected);
|
collection.put("shards", selected);
|
||||||
}
|
}
|
||||||
return collection;
|
return postProcessCollectionJSON(collection);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Walks the tree of collection status to verify that any replicas not reporting a "down" status is
|
* Walks the tree of collection status to verify that any replicas not reporting a "down" status is
|
||||||
* on a live node, if any replicas reporting their status as "active" but the node is not live is
|
* on a live node, if any replicas reporting their status as "active" but the node is not live is
|
||||||
|
@ -243,4 +242,23 @@ public class ClusterStatus {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public static Map<String,Object> postProcessCollectionJSON(Map<String, Object> collection) {
|
||||||
|
final Map<String, Map<String,Object>> shards = collection != null
|
||||||
|
? (Map<String, Map<String,Object>>)collection.getOrDefault("shards", Collections.emptyMap())
|
||||||
|
: Collections.emptyMap();
|
||||||
|
shards.values().forEach(s -> {
|
||||||
|
final Map<String, Map<String,Object>> replicas =
|
||||||
|
(Map<String, Map<String,Object>>)s.getOrDefault("replicas", Collections.emptyMap());
|
||||||
|
replicas.values().forEach(r -> {
|
||||||
|
String nodeName = (String)r.get(ZkStateReader.NODE_NAME_PROP);
|
||||||
|
if (nodeName != null) {
|
||||||
|
// UI needs the base_url set
|
||||||
|
r.put(ZkStateReader.BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(nodeName));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
});
|
||||||
|
return collection;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -109,6 +109,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static java.util.stream.Collectors.toMap;
|
import static java.util.stream.Collectors.toMap;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.URL_SCHEME;
|
||||||
import static org.apache.solr.common.params.CommonParams.ID;
|
import static org.apache.solr.common.params.CommonParams.ID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -190,7 +191,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
|
||||||
.getOrDefault("history", Collections.emptyMap());
|
.getOrDefault("history", Collections.emptyMap());
|
||||||
args.putAll(props);
|
args.putAll(props);
|
||||||
|
|
||||||
overseerUrlScheme = cloudManager.getClusterStateProvider().getClusterProperty("urlScheme", "http");
|
overseerUrlScheme = cloudManager.getClusterStateProvider().getClusterProperty(URL_SCHEME, "http");
|
||||||
} else {
|
} else {
|
||||||
overseerUrlScheme = "http";
|
overseerUrlScheme = "http";
|
||||||
}
|
}
|
||||||
|
|
|
@ -409,7 +409,7 @@ class RebalanceLeaders {
|
||||||
propMap.put(QUEUE_OPERATION, REBALANCELEADERS.toLower());
|
propMap.put(QUEUE_OPERATION, REBALANCELEADERS.toLower());
|
||||||
propMap.put(CORE_NAME_PROP, core);
|
propMap.put(CORE_NAME_PROP, core);
|
||||||
propMap.put(CORE_NODE_NAME_PROP, replica.getName());
|
propMap.put(CORE_NODE_NAME_PROP, replica.getName());
|
||||||
propMap.put(ZkStateReader.BASE_URL_PROP, replica.getProperties().get(ZkStateReader.BASE_URL_PROP));
|
propMap.put(ZkStateReader.NODE_NAME_PROP, replica.getNodeName());
|
||||||
propMap.put(REJOIN_AT_HEAD_PROP, Boolean.toString(rejoinAtHead)); // Get ourselves to be first in line.
|
propMap.put(REJOIN_AT_HEAD_PROP, Boolean.toString(rejoinAtHead)); // Get ourselves to be first in line.
|
||||||
propMap.put(ELECTION_NODE_PROP, electionNode);
|
propMap.put(ELECTION_NODE_PROP, electionNode);
|
||||||
String asyncId = REBALANCELEADERS.toLower() + "_" + core + "_" + Math.abs(System.nanoTime());
|
String asyncId = REBALANCELEADERS.toLower() + "_" + core + "_" + Math.abs(System.nanoTime());
|
||||||
|
|
|
@ -59,7 +59,6 @@ class RequestSyncShardOp implements CoreAdminHandler.CoreAdminOp {
|
||||||
syncStrategy = new SyncStrategy(core.getCoreContainer());
|
syncStrategy = new SyncStrategy(core.getCoreContainer());
|
||||||
|
|
||||||
Map<String, Object> props = new HashMap<>();
|
Map<String, Object> props = new HashMap<>();
|
||||||
props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
|
|
||||||
props.put(ZkStateReader.CORE_NAME_PROP, cname);
|
props.put(ZkStateReader.CORE_NAME_PROP, cname);
|
||||||
props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
|
props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
|
||||||
|
|
||||||
|
|
|
@ -537,16 +537,18 @@ public final class ZookeeperInfoHandler extends RequestHandlerBase {
|
||||||
|
|
||||||
if (childDataStr != null) {
|
if (childDataStr != null) {
|
||||||
Map<String, Object> extColl = (Map<String, Object>) Utils.fromJSONString(childDataStr);
|
Map<String, Object> extColl = (Map<String, Object>) Utils.fromJSONString(childDataStr);
|
||||||
Object collectionState = extColl.get(collection);
|
// add the base_url to replica props
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
Map<String, Object> collectionState = (Map<String, Object>)extColl.get(collection);
|
||||||
|
|
||||||
if (applyStatusFilter) {
|
if (applyStatusFilter) {
|
||||||
// verify this collection matches the filtered state
|
// verify this collection matches the filtered state
|
||||||
if (page.matchesStatusFilter((Map<String, Object>) collectionState, liveNodes)) {
|
if (page.matchesStatusFilter(collectionState, liveNodes)) {
|
||||||
matchesStatusFilter.add(collection);
|
matchesStatusFilter.add(collection);
|
||||||
collectionStates.put(collection, collectionState);
|
collectionStates.put(collection, ClusterStatus.postProcessCollectionJSON(collectionState));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
collectionStates.put(collection, collectionState);
|
collectionStates.put(collection, ClusterStatus.postProcessCollectionJSON(collectionState));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -116,7 +116,6 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.slf4j.MarkerFactory;
|
import org.slf4j.MarkerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
|
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
|
||||||
|
@ -1071,13 +1070,13 @@ public class HttpSolrCall {
|
||||||
// if it's by core name, make sure they match
|
// if it's by core name, make sure they match
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (replica.getStr(BASE_URL_PROP).equals(cores.getZkController().getBaseUrl())) {
|
if (replica.getBaseUrl().equals(cores.getZkController().getBaseUrl())) {
|
||||||
// don't count a local core
|
// don't count a local core
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (origCorename != null) {
|
if (origCorename != null) {
|
||||||
coreUrl = replica.getStr(BASE_URL_PROP) + "/" + origCorename;
|
coreUrl = replica.getBaseUrl() + "/" + origCorename;
|
||||||
} else {
|
} else {
|
||||||
coreUrl = replica.getCoreUrl();
|
coreUrl = replica.getCoreUrl();
|
||||||
if (coreUrl.endsWith("/")) {
|
if (coreUrl.endsWith("/")) {
|
||||||
|
|
|
@ -104,6 +104,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.cloud.ClusterState;
|
import org.apache.solr.common.cloud.ClusterState;
|
||||||
import org.apache.solr.common.cloud.DocCollection;
|
import org.apache.solr.common.cloud.DocCollection;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.SolrZkClient;
|
import org.apache.solr.common.cloud.SolrZkClient;
|
||||||
|
@ -2717,6 +2718,7 @@ public class SolrCLI implements CLIO {
|
||||||
|
|
||||||
protected void runImpl(CommandLine cli) throws Exception {
|
protected void runImpl(CommandLine cli) throws Exception {
|
||||||
this.urlScheme = cli.getOptionValue("urlScheme", "http");
|
this.urlScheme = cli.getOptionValue("urlScheme", "http");
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(this.urlScheme);
|
||||||
|
|
||||||
serverDir = new File(cli.getOptionValue("serverDir"));
|
serverDir = new File(cli.getOptionValue("serverDir"));
|
||||||
if (!serverDir.isDirectory())
|
if (!serverDir.isDirectory())
|
||||||
|
|
|
@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
|
||||||
*/
|
*/
|
||||||
@Slow
|
@Slow
|
||||||
@SolrTestCaseJ4.SuppressPointFields(bugUrl="Test explicitly compares Trie to Points, randomization defeats the point")
|
@SolrTestCaseJ4.SuppressPointFields(bugUrl="Test explicitly compares Trie to Points, randomization defeats the point")
|
||||||
|
@SolrTestCaseJ4.SuppressSSL
|
||||||
public class TestRandomDVFaceting extends SolrTestCaseJ4 {
|
public class TestRandomDVFaceting extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.junit.Test;
|
||||||
/**
|
/**
|
||||||
* Tests for {@link ClusterStateMockUtil}
|
* Tests for {@link ClusterStateMockUtil}
|
||||||
*/
|
*/
|
||||||
|
@SolrTestCaseJ4.SuppressSSL // tests expect http scheme
|
||||||
public class ClusterStateMockUtilTest extends SolrTestCaseJ4 {
|
public class ClusterStateMockUtilTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -241,8 +241,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
|
||||||
ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
|
ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
|
||||||
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
|
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
|
||||||
ZkStateReader.COLLECTION_PROP, collectionName,
|
ZkStateReader.COLLECTION_PROP, collectionName,
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, replica.getName(),
|
ZkStateReader.CORE_NODE_NAME_PROP, replica.getName());
|
||||||
ZkStateReader.BASE_URL_PROP, replica.getBaseUrl());
|
|
||||||
|
|
||||||
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
|
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
|
||||||
|
|
||||||
|
@ -307,8 +306,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
|
||||||
ZkStateReader.CORE_NAME_PROP, replica1.getCoreName(),
|
ZkStateReader.CORE_NAME_PROP, replica1.getCoreName(),
|
||||||
ZkStateReader.NODE_NAME_PROP, replica1.getNodeName(),
|
ZkStateReader.NODE_NAME_PROP, replica1.getNodeName(),
|
||||||
ZkStateReader.COLLECTION_PROP, collectionName,
|
ZkStateReader.COLLECTION_PROP, collectionName,
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, replica1.getName(),
|
ZkStateReader.CORE_NODE_NAME_PROP, replica1.getName());
|
||||||
ZkStateReader.BASE_URL_PROP, replica1.getBaseUrl());
|
|
||||||
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
|
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
|
||||||
|
|
||||||
boolean replicaDeleted = false;
|
boolean replicaDeleted = false;
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||||
import org.apache.solr.SolrTestCaseJ4;
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
import org.apache.solr.common.cloud.OnReconnect;
|
import org.apache.solr.common.cloud.OnReconnect;
|
||||||
import org.apache.solr.common.cloud.SolrZkClient;
|
import org.apache.solr.common.cloud.SolrZkClient;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
|
@ -205,21 +206,22 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
||||||
public void testBasic() throws Exception {
|
public void testBasic() throws Exception {
|
||||||
LeaderElector elector = new LeaderElector(zkClient);
|
LeaderElector elector = new LeaderElector(zkClient);
|
||||||
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
||||||
"http://127.0.0.1/solr/", ZkStateReader.CORE_NAME_PROP, "");
|
UrlScheme.INSTANCE.applyUrlScheme("http://127.0.0.1/solr/"), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
ZkController zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
ZkController zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
||||||
ElectionContext context = new ShardLeaderElectionContextBase(elector,
|
ElectionContext context = new ShardLeaderElectionContextBase(elector,
|
||||||
"shard2", "collection1", "dummynode1", props, zkController);
|
"shard2", "collection1", "dummynode1", props, zkController);
|
||||||
elector.setup(context);
|
elector.setup(context);
|
||||||
elector.joinElection(context, false);
|
elector.joinElection(context, false);
|
||||||
assertEquals("http://127.0.0.1/solr/",
|
assertEquals(UrlScheme.INSTANCE.getUrlScheme() + "://127.0.0.1/solr/",
|
||||||
getLeaderUrl("collection1", "shard2"));
|
getLeaderUrl("collection1", "shard2"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCancelElection() throws Exception {
|
public void testCancelElection() throws Exception {
|
||||||
|
UrlScheme u = UrlScheme.INSTANCE;
|
||||||
LeaderElector first = new LeaderElector(zkClient);
|
LeaderElector first = new LeaderElector(zkClient);
|
||||||
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
||||||
"http://127.0.0.1/solr/", ZkStateReader.CORE_NAME_PROP, "1");
|
u.applyUrlScheme("http://127.0.0.1/solr/"), ZkStateReader.CORE_NAME_PROP, "1");
|
||||||
ZkController zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
ZkController zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
||||||
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
|
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
|
||||||
"slice1", "collection2", "dummynode1", props, zkController);
|
"slice1", "collection2", "dummynode1", props, zkController);
|
||||||
|
@ -227,21 +229,25 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
||||||
first.joinElection(firstContext, false);
|
first.joinElection(firstContext, false);
|
||||||
|
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
assertEquals("original leader was not registered", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
|
|
||||||
|
String url1 = u.applyUrlScheme("http://127.0.0.1/solr/1/");
|
||||||
|
String url2 = u.applyUrlScheme("http://127.0.0.1/solr/2/");
|
||||||
|
|
||||||
|
assertEquals("original leader was not registered", url1, getLeaderUrl("collection2", "slice1"));
|
||||||
|
|
||||||
LeaderElector second = new LeaderElector(zkClient);
|
LeaderElector second = new LeaderElector(zkClient);
|
||||||
props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
||||||
"http://127.0.0.1/solr/", ZkStateReader.CORE_NAME_PROP, "2");
|
u.applyUrlScheme("http://127.0.0.1/solr/"), ZkStateReader.CORE_NAME_PROP, "2");
|
||||||
zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
zkController = MockSolrSource.makeSimpleMock(null, null, zkClient);
|
||||||
ElectionContext context = new ShardLeaderElectionContextBase(second,
|
ElectionContext context = new ShardLeaderElectionContextBase(second,
|
||||||
"slice1", "collection2", "dummynode2", props, zkController);
|
"slice1", "collection2", "dummynode2", props, zkController);
|
||||||
second.setup(context);
|
second.setup(context);
|
||||||
second.joinElection(context, false);
|
second.joinElection(context, false);
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
assertEquals("original leader should have stayed leader", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
|
assertEquals("original leader should have stayed leader", url1, getLeaderUrl("collection2", "slice1"));
|
||||||
firstContext.cancelElection();
|
firstContext.cancelElection();
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
assertEquals("new leader was not registered", "http://127.0.0.1/solr/2/", getLeaderUrl("collection2", "slice1"));
|
assertEquals("new leader was not registered", url2, getLeaderUrl("collection2", "slice1"));
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getLeaderUrl(final String collection, final String slice)
|
private String getLeaderUrl(final String collection, final String slice)
|
||||||
|
@ -384,7 +390,12 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
||||||
for (int i = 1; i <= numShards; i ++) {
|
for (int i = 1; i <= numShards; i ++) {
|
||||||
// if this test fails, getLeaderUrl will more likely throw an exception and fail the test,
|
// if this test fails, getLeaderUrl will more likely throw an exception and fail the test,
|
||||||
// but add an assertEquals as well for good measure
|
// but add an assertEquals as well for good measure
|
||||||
assertEquals("2/", getLeaderUrl("collection1", "parshard" + i));
|
String leaderUrl = getLeaderUrl("collection1", "parshard" + i);
|
||||||
|
int at = leaderUrl.indexOf("://");
|
||||||
|
if (at != -1) {
|
||||||
|
leaderUrl = leaderUrl.substring(at + 3);
|
||||||
|
}
|
||||||
|
assertEquals("2/", leaderUrl);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// cleanup any threads still running
|
// cleanup any threads still running
|
||||||
|
@ -414,6 +425,11 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
private int getLeaderThread() throws KeeperException, InterruptedException {
|
private int getLeaderThread() throws KeeperException, InterruptedException {
|
||||||
String leaderUrl = getLeaderUrl("collection1", "shard1");
|
String leaderUrl = getLeaderUrl("collection1", "shard1");
|
||||||
|
// strip off the scheme
|
||||||
|
final int at = leaderUrl.indexOf("://");
|
||||||
|
if (at != -1) {
|
||||||
|
leaderUrl = leaderUrl.substring(at + 3);
|
||||||
|
}
|
||||||
return Integer.parseInt(leaderUrl.replaceAll("/", ""));
|
return Integer.parseInt(leaderUrl.replaceAll("/", ""));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.solr.cloud;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
import org.apache.solr.SolrTestCaseJ4Test;
|
import org.apache.solr.SolrTestCaseJ4Test;
|
||||||
import org.apache.solr.cloud.overseer.NodeMutator;
|
import org.apache.solr.cloud.overseer.NodeMutator;
|
||||||
import org.apache.solr.cloud.overseer.ZkWriteCommand;
|
import org.apache.solr.cloud.overseer.ZkWriteCommand;
|
||||||
|
@ -28,6 +29,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@SolrTestCaseJ4.SuppressSSL // tests compare for http:
|
||||||
public class NodeMutatorTest extends SolrTestCaseJ4Test {
|
public class NodeMutatorTest extends SolrTestCaseJ4Test {
|
||||||
|
|
||||||
private static final String NODE3 = "baseUrl3_";
|
private static final String NODE3 = "baseUrl3_";
|
||||||
|
|
|
@ -102,6 +102,7 @@ import com.codahale.metrics.Snapshot;
|
||||||
import com.codahale.metrics.Timer;
|
import com.codahale.metrics.Timer;
|
||||||
|
|
||||||
@Slow
|
@Slow
|
||||||
|
@SolrTestCaseJ4.SuppressSSL
|
||||||
public class OverseerTest extends SolrTestCaseJ4 {
|
public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
@ -213,8 +214,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName,
|
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName,
|
||||||
ZkStateReader.COLLECTION_PROP, collection,
|
ZkStateReader.COLLECTION_PROP, collection,
|
||||||
ZkStateReader.SHARD_ID_PROP, shard,
|
ZkStateReader.SHARD_ID_PROP, shard,
|
||||||
ZkStateReader.NUM_SHARDS_PROP, Integer.toString(numShards),
|
ZkStateReader.NUM_SHARDS_PROP, Integer.toString(numShards));
|
||||||
ZkStateReader.BASE_URL_PROP, "http://" + nodeName + "/solr/");
|
|
||||||
ZkDistributedQueue q = overseer.getStateUpdateQueue();
|
ZkDistributedQueue q = overseer.getStateUpdateQueue();
|
||||||
q.offer(Utils.toJSON(m));
|
q.offer(Utils.toJSON(m));
|
||||||
}
|
}
|
||||||
|
@ -233,15 +233,14 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
zkClient.makePath("/collections/" + collection + "/leader_elect/"
|
zkClient.makePath("/collections/" + collection + "/leader_elect/"
|
||||||
+ shardId + "/election", true);
|
+ shardId + "/election", true);
|
||||||
} catch (NodeExistsException nee) {}
|
} catch (NodeExistsException nee) {}
|
||||||
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
|
ZkNodeProps props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, nodeName,
|
||||||
"http://" + nodeName + "/solr/", ZkStateReader.NODE_NAME_PROP,
|
ZkStateReader.CORE_NAME_PROP, coreName,
|
||||||
nodeName, ZkStateReader.CORE_NAME_PROP, coreName,
|
|
||||||
ZkStateReader.SHARD_ID_PROP, shardId,
|
ZkStateReader.SHARD_ID_PROP, shardId,
|
||||||
ZkStateReader.COLLECTION_PROP, collection,
|
ZkStateReader.COLLECTION_PROP, collection,
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||||
LeaderElector elector = new LeaderElector(zkClient);
|
LeaderElector elector = new LeaderElector(zkClient);
|
||||||
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
|
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
|
||||||
elector, shardId, collection, nodeName + "_" + coreName, props,
|
elector, shardId, collection, nodeName + coreName, props,
|
||||||
MockSolrSource.makeSimpleMock(overseer, zkStateReader, null));
|
MockSolrSource.makeSimpleMock(overseer, zkStateReader, null));
|
||||||
elector.setup(ctx);
|
elector.setup(ctx);
|
||||||
electionContext.put(coreName, ctx);
|
electionContext.put(coreName, ctx);
|
||||||
|
@ -405,7 +404,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
final int numShards = 6; // this is not the number of shards in the collection
|
final int numShards = 6; // this is not the number of shards in the collection
|
||||||
|
|
||||||
|
@ -449,7 +448,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
final int numShards = 3;
|
final int numShards = 3;
|
||||||
mockController.createCollection(COLLECTION, 3);
|
mockController.createCollection(COLLECTION, 3);
|
||||||
|
@ -516,7 +515,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
try (ZkController zkController = createMockZkController(server.getZkAddress(), zkClient, reader)) {
|
try (ZkController zkController = createMockZkController(server.getZkAddress(), zkClient, reader)) {
|
||||||
|
|
||||||
|
@ -527,7 +526,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.DOWNNODE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.DOWNNODE.toLower(),
|
||||||
ZkStateReader.NODE_NAME_PROP, "127.0.0.1");
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr");
|
||||||
List<ZkWriteCommand> commands = new NodeMutator().downNode(reader.getClusterState(), m);
|
List<ZkWriteCommand> commands = new NodeMutator().downNode(reader.getClusterState(), m);
|
||||||
|
|
||||||
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
|
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
|
||||||
|
@ -591,8 +590,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
createCollection(COLLECTION, 1);
|
createCollection(COLLECTION, 1);
|
||||||
|
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
ZkStateReader.NODE_NAME_PROP, "node1_",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
|
@ -607,8 +605,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
//publish node state (active)
|
//publish node state (active)
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
ZkStateReader.NODE_NAME_PROP, "node1_",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
|
@ -664,7 +661,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
reader = new ZkStateReader(zkClient);
|
reader = new ZkStateReader(zkClient);
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
overseerClient = electNewOverseer(server.getZkAddress());
|
overseerClient = electNewOverseer(server.getZkAddress());
|
||||||
|
|
||||||
|
@ -726,7 +723,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
reader = new ZkStateReader(zkClient);
|
reader = new ZkStateReader(zkClient);
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
LeaderElector overseerElector = new LeaderElector(zkClient);
|
LeaderElector overseerElector = new LeaderElector(zkClient);
|
||||||
if (overseers.size() > 0) {
|
if (overseers.size() > 0) {
|
||||||
|
@ -905,7 +902,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
for (int i = 0; i < atLeast(4); i++) {
|
for (int i = 0; i < atLeast(4); i++) {
|
||||||
killCounter.incrementAndGet(); // for each round allow 1 kill
|
killCounter.incrementAndGet(); // for each round allow 1 kill
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "node1_", overseers);
|
||||||
|
|
||||||
TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
||||||
while (!timeout.hasTimedOut()) {
|
while (!timeout.hasTimedOut()) {
|
||||||
|
@ -953,7 +950,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
mockController2 = new MockZKController(server.getZkAddress(), "node2", overseers);
|
mockController2 = new MockZKController(server.getZkAddress(), "node2_", overseers);
|
||||||
|
|
||||||
timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
||||||
while (!timeout.hasTimedOut()) {
|
while (!timeout.hasTimedOut()) {
|
||||||
|
@ -1037,7 +1034,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
reader = new ZkStateReader(zkClient);
|
reader = new ZkStateReader(zkClient);
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
overseerClient = electNewOverseer(server.getZkAddress());
|
overseerClient = electNewOverseer(server.getZkAddress());
|
||||||
|
|
||||||
|
@ -1053,7 +1050,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
mockController.close();
|
mockController.close();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
mockController.publishState(COLLECTION, "core1", "core_node1","shard1", Replica.State.RECOVERING, 1, true, overseers.get(0));
|
mockController.publishState(COLLECTION, "core1", "core_node1","shard1", Replica.State.RECOVERING, 1, true, overseers.get(0));
|
||||||
|
|
||||||
|
@ -1095,7 +1092,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
reader = new ZkStateReader(zkClient);
|
reader = new ZkStateReader(zkClient);
|
||||||
reader.createClusterStateWatchersAndUpdate();
|
reader.createClusterStateWatchersAndUpdate();
|
||||||
|
|
||||||
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
|
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
|
||||||
|
|
||||||
final int MAX_COLLECTIONS = 10, MAX_CORES = 10, MAX_STATE_CHANGES = 20000;
|
final int MAX_COLLECTIONS = 10, MAX_CORES = 10, MAX_STATE_CHANGES = 20000;
|
||||||
|
|
||||||
|
@ -1118,8 +1115,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, "node1",
|
ZkStateReader.CORE_NODE_NAME_PROP, "node1",
|
||||||
ZkStateReader.COLLECTION_PROP, "perf" + j,
|
ZkStateReader.COLLECTION_PROP, "perf" + j,
|
||||||
ZkStateReader.NUM_SHARDS_PROP, "1",
|
ZkStateReader.NUM_SHARDS_PROP, "1");
|
||||||
ZkStateReader.BASE_URL_PROP, "http://" + "node1" + "/solr/");
|
|
||||||
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
|
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
|
||||||
q.offer(Utils.toJSON(m));
|
q.offer(Utils.toJSON(m));
|
||||||
if (j >= MAX_COLLECTIONS - 1) j = 0;
|
if (j >= MAX_COLLECTIONS - 1) j = 0;
|
||||||
|
@ -1212,8 +1208,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
"createNodeSet", "");
|
"createNodeSet", "");
|
||||||
queue.offer(Utils.toJSON(m));
|
queue.offer(Utils.toJSON(m));
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
|
@ -1221,8 +1216,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
|
ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
|
||||||
queue.offer(Utils.toJSON(m));
|
queue.offer(Utils.toJSON(m));
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
ZkStateReader.NODE_NAME_PROP, "node1_",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.CORE_NAME_PROP, "core2",
|
ZkStateReader.CORE_NAME_PROP, "core2",
|
||||||
|
@ -1235,8 +1229,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
//submit to proper queue
|
//submit to proper queue
|
||||||
queue = overseers.get(0).getStateUpdateQueue();
|
queue = overseers.get(0).getStateUpdateQueue();
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.CORE_NAME_PROP, "core3",
|
ZkStateReader.CORE_NAME_PROP, "core3",
|
||||||
|
@ -1276,9 +1269,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
createCollection("c1", 1);
|
createCollection("c1", 1);
|
||||||
|
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.COLLECTION_PROP, "c1",
|
ZkStateReader.COLLECTION_PROP, "c1",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, "core_node1",
|
ZkStateReader.CORE_NODE_NAME_PROP, "core_node1",
|
||||||
|
@ -1291,9 +1283,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
verifyReplicaStatus(reader, "c1", "shard1", "core_node1", Replica.State.DOWN);
|
verifyReplicaStatus(reader, "c1", "shard1", "core_node1", Replica.State.DOWN);
|
||||||
|
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.COLLECTION_PROP, "c1",
|
ZkStateReader.COLLECTION_PROP, "c1",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
ZkStateReader.ROLES_PROP, "",
|
ZkStateReader.ROLES_PROP, "",
|
||||||
|
@ -1303,9 +1294,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
|
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.COLLECTION_PROP, "c1",
|
ZkStateReader.COLLECTION_PROP, "c1",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
ZkStateReader.ROLES_PROP, "",
|
ZkStateReader.ROLES_PROP, "",
|
||||||
|
@ -1340,9 +1330,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.ADDREPLICA.toLower(),
|
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.ADDREPLICA.toLower(),
|
||||||
"collection", testCollectionName,
|
"collection", testCollectionName,
|
||||||
ZkStateReader.SHARD_ID_PROP, "x",
|
ZkStateReader.SHARD_ID_PROP, "x",
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, "core_node1",
|
ZkStateReader.CORE_NODE_NAME_PROP, "core_node1",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.CORE_NAME_PROP, "core1",
|
ZkStateReader.CORE_NAME_PROP, "core1",
|
||||||
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString()
|
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString()
|
||||||
);
|
);
|
||||||
|
@ -1491,9 +1480,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
for (int ss = 1; ss <= numShards; ++ss) {
|
for (int ss = 1; ss <= numShards; ++ss) {
|
||||||
final int N = (numReplicas-rr)*numShards + ss;
|
final int N = (numReplicas-rr)*numShards + ss;
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard"+ss,
|
ZkStateReader.SHARD_ID_PROP, "shard"+ss,
|
||||||
ZkStateReader.NODE_NAME_PROP, "node"+N,
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.CORE_NAME_PROP, "core"+N,
|
ZkStateReader.CORE_NAME_PROP, "core"+N,
|
||||||
ZkStateReader.CORE_NODE_NAME_PROP, "core_node"+N,
|
ZkStateReader.CORE_NODE_NAME_PROP, "core_node"+N,
|
||||||
|
@ -1516,9 +1504,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||||
for (int ss = 1; ss <= numShards; ++ss) {
|
for (int ss = 1; ss <= numShards; ++ss) {
|
||||||
final int N = (numReplicas-rr)*numShards + ss;
|
final int N = (numReplicas-rr)*numShards + ss;
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
|
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard"+ss,
|
ZkStateReader.SHARD_ID_PROP, "shard"+ss,
|
||||||
ZkStateReader.NODE_NAME_PROP, "node"+N,
|
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
|
||||||
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
ZkStateReader.COLLECTION_PROP, COLLECTION,
|
||||||
ZkStateReader.CORE_NAME_PROP, "core"+N,
|
ZkStateReader.CORE_NAME_PROP, "core"+N,
|
||||||
ZkStateReader.ROLES_PROP, "",
|
ZkStateReader.ROLES_PROP, "",
|
||||||
|
|
|
@ -35,8 +35,6 @@ import org.apache.solr.common.cloud.Slice;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* See SOLR-9504
|
* See SOLR-9504
|
||||||
*/
|
*/
|
||||||
|
@ -73,7 +71,7 @@ public class TestLeaderElectionWithEmptyReplica extends SolrCloudTestCase {
|
||||||
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
|
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
|
||||||
for (JettySolrRunner jettySolrRunner : jettySolrRunners) {
|
for (JettySolrRunner jettySolrRunner : jettySolrRunners) {
|
||||||
int port = jettySolrRunner.getBaseUrl().getPort();
|
int port = jettySolrRunner.getBaseUrl().getPort();
|
||||||
if (replica.getStr(BASE_URL_PROP).contains(":" + port)) {
|
if (replica.getBaseUrl().contains(":" + port)) {
|
||||||
replicaJetty = jettySolrRunner;
|
replicaJetty = jettySolrRunner;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.solr.client.solrj.impl.HttpClientUtil;
|
||||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
||||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||||
import org.apache.solr.client.solrj.request.CoreAdminRequest;
|
import org.apache.solr.client.solrj.request.CoreAdminRequest;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
|
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
|
||||||
import org.apache.solr.util.SSLTestConfig;
|
import org.apache.solr.util.SSLTestConfig;
|
||||||
|
@ -104,6 +105,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
|
||||||
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
|
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
|
||||||
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
||||||
System.setProperty(ZkStateReader.URL_SCHEME, "http");
|
System.setProperty(ZkStateReader.URL_SCHEME, "http");
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
|
||||||
checkClusterWithNodeReplacement(sslConfig);
|
checkClusterWithNodeReplacement(sslConfig);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -115,6 +117,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
|
||||||
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
|
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
|
||||||
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
||||||
System.setProperty(ZkStateReader.URL_SCHEME, "http");
|
System.setProperty(ZkStateReader.URL_SCHEME, "http");
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
|
||||||
checkClusterWithNodeReplacement(sslConfig);
|
checkClusterWithNodeReplacement(sslConfig);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -112,7 +112,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
|
||||||
DocCollection b1x1 = clusterState.getCollection("b1x1");
|
DocCollection b1x1 = clusterState.getCollection("b1x1");
|
||||||
Collection<Replica> replicas = b1x1.getSlice("shard1").getReplicas();
|
Collection<Replica> replicas = b1x1.getSlice("shard1").getReplicas();
|
||||||
assertEquals(1, replicas.size());
|
assertEquals(1, replicas.size());
|
||||||
String baseUrl = replicas.iterator().next().getStr(ZkStateReader.BASE_URL_PROP);
|
String baseUrl = replicas.iterator().next().getBaseUrl();
|
||||||
if (!baseUrl.endsWith("/")) baseUrl += "/";
|
if (!baseUrl.endsWith("/")) baseUrl += "/";
|
||||||
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2", 2000, 5000)) {
|
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2", 2000, 5000)) {
|
||||||
|
|
||||||
|
@ -171,7 +171,6 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
|
||||||
|
|
||||||
//Simulate a replica being in down state.
|
//Simulate a replica being in down state.
|
||||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
|
||||||
ZkStateReader.BASE_URL_PROP, notLeader.getStr(ZkStateReader.BASE_URL_PROP),
|
|
||||||
ZkStateReader.NODE_NAME_PROP, notLeader.getStr(ZkStateReader.NODE_NAME_PROP),
|
ZkStateReader.NODE_NAME_PROP, notLeader.getStr(ZkStateReader.NODE_NAME_PROP),
|
||||||
ZkStateReader.COLLECTION_PROP, "football",
|
ZkStateReader.COLLECTION_PROP, "football",
|
||||||
ZkStateReader.SHARD_ID_PROP, "shard1",
|
ZkStateReader.SHARD_ID_PROP, "shard1",
|
||||||
|
@ -189,7 +188,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
|
||||||
|
|
||||||
//Query against the node which hosts the down replica
|
//Query against the node which hosts the down replica
|
||||||
|
|
||||||
String baseUrl = notLeader.getStr(ZkStateReader.BASE_URL_PROP);
|
String baseUrl = notLeader.getBaseUrl();
|
||||||
if (!baseUrl.endsWith("/")) baseUrl += "/";
|
if (!baseUrl.endsWith("/")) baseUrl += "/";
|
||||||
String path = baseUrl + "football";
|
String path = baseUrl + "football";
|
||||||
log.info("Firing queries against path={}", path);
|
log.info("Firing queries against path={}", path);
|
||||||
|
|
|
@ -602,8 +602,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
|
||||||
Replica newReplica = grabNewReplica(response, getCollectionState(collectionName));
|
Replica newReplica = grabNewReplica(response, getCollectionState(collectionName));
|
||||||
|
|
||||||
assertEquals("Replica should be created on the right node",
|
assertEquals("Replica should be created on the right node",
|
||||||
cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)),
|
cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)), newReplica.getBaseUrl());
|
||||||
newReplica.getStr(ZkStateReader.BASE_URL_PROP));
|
|
||||||
|
|
||||||
Path instancePath = createTempDir();
|
Path instancePath = createTempDir();
|
||||||
response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
|
response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
|
||||||
|
@ -612,7 +611,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
|
||||||
newReplica = grabNewReplica(response, getCollectionState(collectionName));
|
newReplica = grabNewReplica(response, getCollectionState(collectionName));
|
||||||
assertNotNull(newReplica);
|
assertNotNull(newReplica);
|
||||||
|
|
||||||
try (HttpSolrClient coreclient = getHttpSolrClient(newReplica.getStr(ZkStateReader.BASE_URL_PROP))) {
|
try (HttpSolrClient coreclient = getHttpSolrClient(newReplica.getBaseUrl())) {
|
||||||
CoreAdminResponse status = CoreAdminRequest.getStatus(newReplica.getStr("core"), coreclient);
|
CoreAdminResponse status = CoreAdminRequest.getStatus(newReplica.getStr("core"), coreclient);
|
||||||
NamedList<Object> coreStatus = status.getCoreStatus(newReplica.getStr("core"));
|
NamedList<Object> coreStatus = status.getCoreStatus(newReplica.getStr("core"));
|
||||||
String instanceDirStr = (String) coreStatus.get("instanceDir");
|
String instanceDirStr = (String) coreStatus.get("instanceDir");
|
||||||
|
|
|
@ -74,7 +74,6 @@ import org.junit.Test;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
|
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
|
||||||
|
|
||||||
@Slow
|
@Slow
|
||||||
|
@ -187,7 +186,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
||||||
boolean restarted = false;
|
boolean restarted = false;
|
||||||
for (JettySolrRunner jetty : jettys) {
|
for (JettySolrRunner jetty : jettys) {
|
||||||
int port = jetty.getBaseUrl().getPort();
|
int port = jetty.getBaseUrl().getPort();
|
||||||
if (replica.getStr(BASE_URL_PROP).contains(":" + port)) {
|
if (replica.getBaseUrl().contains(":" + port)) {
|
||||||
stoppedNodeName = jetty.getNodeName();
|
stoppedNodeName = jetty.getNodeName();
|
||||||
jetty.stop();
|
jetty.stop();
|
||||||
jetty.start();
|
jetty.start();
|
||||||
|
@ -206,7 +205,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
||||||
CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(collectionName, SHARD1_0);
|
CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(collectionName, SHARD1_0);
|
||||||
// use control client because less chances of it being the node being restarted
|
// use control client because less chances of it being the node being restarted
|
||||||
// this is to avoid flakiness of test because of NoHttpResponseExceptions
|
// this is to avoid flakiness of test because of NoHttpResponseExceptions
|
||||||
String control_collection = client.getZkStateReader().getClusterState().getCollection("control_collection").getReplicas().get(0).getStr(BASE_URL_PROP);
|
String control_collection = client.getZkStateReader().getClusterState().getCollection("control_collection").getReplicas().get(0).getBaseUrl();
|
||||||
try (HttpSolrClient control = new HttpSolrClient.Builder(control_collection).withHttpClient(client.getLbClient().getHttpClient()).build()) {
|
try (HttpSolrClient control = new HttpSolrClient.Builder(control_collection).withHttpClient(client.getLbClient().getHttpClient()).build()) {
|
||||||
state = addReplica.processAndWait(control, 30);
|
state = addReplica.processAndWait(control, 30);
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,8 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.solr.core.snapshots;
|
package org.apache.solr.core.snapshots;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -27,8 +25,8 @@ import java.util.Optional;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.lucene.util.TestUtil;
|
|
||||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||||
|
import org.apache.lucene.util.TestUtil;
|
||||||
import org.apache.solr.SolrTestCaseJ4;
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
import org.apache.solr.client.solrj.SolrClient;
|
import org.apache.solr.client.solrj.SolrClient;
|
||||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||||
|
@ -42,7 +40,6 @@ import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Replica.State;
|
import org.apache.solr.common.cloud.Replica.State;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.common.util.NamedList;
|
import org.apache.solr.common.util.NamedList;
|
||||||
import org.apache.solr.core.snapshots.CollectionSnapshotMetaData.CoreSnapshotMetaData;
|
import org.apache.solr.core.snapshots.CollectionSnapshotMetaData.CoreSnapshotMetaData;
|
||||||
import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
|
import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
|
||||||
|
@ -146,8 +143,8 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
|
||||||
continue; // We know that the snapshot is not created for this replica.
|
continue; // We know that the snapshot is not created for this replica.
|
||||||
}
|
}
|
||||||
|
|
||||||
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
|
String replicaBaseUrl = replica.getBaseUrl();
|
||||||
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
String coreName = replica.getCoreName();
|
||||||
|
|
||||||
assertTrue(snapshotByCoreName.containsKey(coreName));
|
assertTrue(snapshotByCoreName.containsKey(coreName));
|
||||||
CoreSnapshotMetaData coreSnapshot = snapshotByCoreName.get(coreName);
|
CoreSnapshotMetaData coreSnapshot = snapshotByCoreName.get(coreName);
|
||||||
|
@ -256,8 +253,8 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
|
||||||
continue; // We know that the snapshot was not created for this replica.
|
continue; // We know that the snapshot was not created for this replica.
|
||||||
}
|
}
|
||||||
|
|
||||||
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
|
String replicaBaseUrl = replica.getBaseUrl();
|
||||||
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
String coreName = replica.getCoreName();
|
||||||
|
|
||||||
try (SolrClient adminClient = getHttpSolrClient(replicaBaseUrl)) {
|
try (SolrClient adminClient = getHttpSolrClient(replicaBaseUrl)) {
|
||||||
Collection<SnapshotMetaData> snapshots = listCoreSnapshots(adminClient, coreName);
|
Collection<SnapshotMetaData> snapshots = listCoreSnapshots(adminClient, coreName);
|
||||||
|
|
|
@ -56,8 +56,6 @@ import org.junit.Test;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
|
|
||||||
@SolrTestCaseJ4.SuppressSSL // Currently unknown why SSL does not work with this test
|
@SolrTestCaseJ4.SuppressSSL // Currently unknown why SSL does not work with this test
|
||||||
@Slow
|
@Slow
|
||||||
public class TestSolrCoreSnapshots extends SolrCloudTestCase {
|
public class TestSolrCoreSnapshots extends SolrCloudTestCase {
|
||||||
|
@ -97,8 +95,8 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
|
||||||
assertEquals(1, shard.getReplicas().size());
|
assertEquals(1, shard.getReplicas().size());
|
||||||
Replica replica = shard.getReplicas().iterator().next();
|
Replica replica = shard.getReplicas().iterator().next();
|
||||||
|
|
||||||
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
|
String replicaBaseUrl = replica.getBaseUrl();
|
||||||
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
String coreName = replica.getCoreName();
|
||||||
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
|
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
|
||||||
String commitName = TestUtil.randomSimpleString(random(), 1, 5);
|
String commitName = TestUtil.randomSimpleString(random(), 1, 5);
|
||||||
String duplicateName = commitName.concat("_duplicate");
|
String duplicateName = commitName.concat("_duplicate");
|
||||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||||
import org.apache.solr.common.MapWriter;
|
import org.apache.solr.common.MapWriter;
|
||||||
import org.apache.solr.common.cloud.DocCollection;
|
import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.common.util.StrUtils;
|
import org.apache.solr.common.util.StrUtils;
|
||||||
import org.apache.solr.util.RTimer;
|
import org.apache.solr.util.RTimer;
|
||||||
import org.apache.solr.util.SimplePostTool;
|
import org.apache.solr.util.SimplePostTool;
|
||||||
|
@ -66,7 +65,7 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase {
|
||||||
DocCollection sysColl = cloudClient.getZkStateReader().getClusterState().getCollection(".system");
|
DocCollection sysColl = cloudClient.getZkStateReader().getClusterState().getCollection(".system");
|
||||||
Replica replica = sysColl.getActiveSlicesMap().values().iterator().next().getLeader();
|
Replica replica = sysColl.getActiveSlicesMap().values().iterator().next().getLeader();
|
||||||
|
|
||||||
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
|
String baseUrl = replica.getBaseUrl();
|
||||||
String url = baseUrl + "/.system/config/requestHandler";
|
String url = baseUrl + "/.system/config/requestHandler";
|
||||||
MapWriter map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
|
MapWriter map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
|
||||||
assertNotNull(map);
|
assertNotNull(map);
|
||||||
|
|
|
@ -96,7 +96,7 @@ public class TestConfigReload extends AbstractFullDistribZkTestBase {
|
||||||
List<String> urls = new ArrayList<>();
|
List<String> urls = new ArrayList<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas())
|
for (Replica replica : slice.getReplicas())
|
||||||
urls.add(""+replica.get(ZkStateReader.BASE_URL_PROP) + "/"+replica.get(ZkStateReader.CORE_NAME_PROP));
|
urls.add(""+replica.getBaseUrl() + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
||||||
}
|
}
|
||||||
HashSet<String> succeeded = new HashSet<>();
|
HashSet<String> succeeded = new HashSet<>();
|
||||||
|
|
||||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.solr.common.SolrInputDocument;
|
||||||
import org.apache.solr.common.cloud.DocCollection;
|
import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.common.params.CoreAdminParams;
|
import org.apache.solr.common.params.CoreAdminParams;
|
||||||
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
|
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
|
||||||
import org.apache.solr.util.BadHdfsThreadsFilter;
|
import org.apache.solr.util.BadHdfsThreadsFilter;
|
||||||
|
@ -56,8 +55,6 @@ import org.junit.Test;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
|
||||||
|
|
||||||
@ThreadLeakFilters(defaultFilters = true, filters = {
|
@ThreadLeakFilters(defaultFilters = true, filters = {
|
||||||
SolrIgnoredThreadsFilter.class,
|
SolrIgnoredThreadsFilter.class,
|
||||||
QuickPatchThreadsFilter.class,
|
QuickPatchThreadsFilter.class,
|
||||||
|
@ -179,8 +176,8 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
|
||||||
assertEquals(1, shard.getReplicas().size());
|
assertEquals(1, shard.getReplicas().size());
|
||||||
Replica replica = shard.getReplicas().iterator().next();
|
Replica replica = shard.getReplicas().iterator().next();
|
||||||
|
|
||||||
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
|
String replicaBaseUrl = replica.getBaseUrl();
|
||||||
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
String coreName = replica.getCoreName();
|
||||||
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
|
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
|
||||||
|
|
||||||
boolean testViaReplicationHandler = random().nextBoolean();
|
boolean testViaReplicationHandler = random().nextBoolean();
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class TestReqParamsAPI extends SolrCloudTestCase {
|
||||||
List<String> urls = new ArrayList<>();
|
List<String> urls = new ArrayList<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas())
|
for (Replica replica : slice.getReplicas())
|
||||||
urls.add("" + replica.get(ZkStateReader.BASE_URL_PROP) + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
urls.add("" + replica.getBaseUrl() + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
RestTestHarness writeHarness = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
RestTestHarness writeHarness = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class TestSolrConfigHandlerCloud extends AbstractFullDistribZkTestBase {
|
||||||
List<String> urls = new ArrayList<>();
|
List<String> urls = new ArrayList<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas())
|
for (Replica replica : slice.getReplicas())
|
||||||
urls.add(""+replica.get(ZkStateReader.BASE_URL_PROP) + "/"+replica.get(ZkStateReader.CORE_NAME_PROP));
|
urls.add(""+replica.getBaseUrl() + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
||||||
}
|
}
|
||||||
return urls.get(random().nextInt(urls.size()));
|
return urls.get(random().nextInt(urls.size()));
|
||||||
}
|
}
|
||||||
|
@ -95,7 +95,7 @@ public class TestSolrConfigHandlerCloud extends AbstractFullDistribZkTestBase {
|
||||||
List<String> urls = new ArrayList<>();
|
List<String> urls = new ArrayList<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas())
|
for (Replica replica : slice.getReplicas())
|
||||||
urls.add(""+replica.get(ZkStateReader.BASE_URL_PROP) + "/"+replica.get(ZkStateReader.CORE_NAME_PROP));
|
urls.add(""+replica.getBaseUrl() + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
RestTestHarness writeHarness = randomRestTestHarness();
|
RestTestHarness writeHarness = randomRestTestHarness();
|
||||||
|
|
|
@ -146,7 +146,7 @@ public class TestSolrConfigHandlerConcurrent extends AbstractFullDistribZkTestBa
|
||||||
List<String> urls = new ArrayList<>();
|
List<String> urls = new ArrayList<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas())
|
for (Replica replica : slice.getReplicas())
|
||||||
urls.add(""+replica.get(ZkStateReader.BASE_URL_PROP) + "/"+replica.get(ZkStateReader.CORE_NAME_PROP));
|
urls.add(""+replica.getBaseUrl() + "/" + replica.get(ZkStateReader.CORE_NAME_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.mockito.Mockito;
|
||||||
/**
|
/**
|
||||||
* Tests for {@link CloudReplicaSource}
|
* Tests for {@link CloudReplicaSource}
|
||||||
*/
|
*/
|
||||||
|
@SolrTestCaseJ4.SuppressSSL // lots of assumptions about http: in this test
|
||||||
public class CloudReplicaSourceTest extends SolrTestCaseJ4 {
|
public class CloudReplicaSourceTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.index.BaseTestCheckIndex;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.util.QuickPatchThreadsFilter;
|
import org.apache.lucene.util.QuickPatchThreadsFilter;
|
||||||
import org.apache.solr.SolrIgnoredThreadsFilter;
|
import org.apache.solr.SolrIgnoredThreadsFilter;
|
||||||
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
import org.apache.solr.client.solrj.SolrClient;
|
import org.apache.solr.client.solrj.SolrClient;
|
||||||
import org.apache.solr.client.solrj.SolrQuery;
|
import org.apache.solr.client.solrj.SolrQuery;
|
||||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||||
|
@ -47,6 +48,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
|
||||||
QuickPatchThreadsFilter.class,
|
QuickPatchThreadsFilter.class,
|
||||||
BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
|
BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
|
||||||
})
|
})
|
||||||
|
@SolrTestCaseJ4.SuppressSSL
|
||||||
// commented out on: 24-Dec-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 12-Jun-2018
|
// commented out on: 24-Dec-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 12-Jun-2018
|
||||||
public class CheckHdfsIndexTest extends AbstractFullDistribZkTestBase {
|
public class CheckHdfsIndexTest extends AbstractFullDistribZkTestBase {
|
||||||
private static MiniDFSCluster dfsCluster;
|
private static MiniDFSCluster dfsCluster;
|
||||||
|
|
|
@ -375,11 +375,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
|
||||||
}
|
}
|
||||||
ArrayList<Node> nodes = new ArrayList<>();
|
ArrayList<Node> nodes = new ArrayList<>();
|
||||||
|
|
||||||
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "");
|
|
||||||
|
|
||||||
final AtomicInteger retries = new AtomicInteger();
|
final AtomicInteger retries = new AtomicInteger();
|
||||||
nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
Node retryNode = new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
|
Node retryNode = new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
|
||||||
@Override
|
@Override
|
||||||
public boolean checkRetry(Error err) {
|
public boolean checkRetry(Error err) {
|
||||||
|
@ -439,11 +436,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
|
||||||
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
|
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
|
||||||
ArrayList<Node> nodes = new ArrayList<>();
|
ArrayList<Node> nodes = new ArrayList<>();
|
||||||
|
|
||||||
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "");
|
|
||||||
|
|
||||||
final AtomicInteger retries = new AtomicInteger();
|
final AtomicInteger retries = new AtomicInteger();
|
||||||
nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
if (nodeType == NodeType.FORWARD) {
|
if (nodeType == NodeType.FORWARD) {
|
||||||
nodes.add(new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
nodes.add(new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
||||||
@Override
|
@Override
|
||||||
|
@ -594,11 +588,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
|
||||||
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
|
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
|
||||||
ArrayList<Node> nodes = new ArrayList<>();
|
ArrayList<Node> nodes = new ArrayList<>();
|
||||||
|
|
||||||
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "");
|
|
||||||
|
|
||||||
final AtomicInteger retries = new AtomicInteger();
|
final AtomicInteger retries = new AtomicInteger();
|
||||||
nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
Node retryNode;
|
Node retryNode;
|
||||||
if (nodeType == NodeType.FORWARD) {
|
if (nodeType == NodeType.FORWARD) {
|
||||||
retryNode = new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
retryNode = new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
||||||
|
@ -711,11 +702,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
|
||||||
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
|
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
|
||||||
ArrayList<Node> nodes = new ArrayList<>();
|
ArrayList<Node> nodes = new ArrayList<>();
|
||||||
|
|
||||||
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "");
|
|
||||||
|
|
||||||
final AtomicInteger retries = new AtomicInteger();
|
final AtomicInteger retries = new AtomicInteger();
|
||||||
nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
ForwardNode retryNode = new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
ForwardNode retryNode = new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
|
||||||
@Override
|
@Override
|
||||||
public boolean checkRetry(Error err) {
|
public boolean checkRetry(Error err) {
|
||||||
|
@ -758,11 +746,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
|
||||||
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
|
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
|
||||||
ArrayList<Node> nodes = new ArrayList<>();
|
ArrayList<Node> nodes = new ArrayList<>();
|
||||||
|
|
||||||
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "");
|
|
||||||
|
|
||||||
final AtomicInteger retries = new AtomicInteger();
|
final AtomicInteger retries = new AtomicInteger();
|
||||||
nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
|
||||||
Node retryNode = new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
|
Node retryNode = new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
|
||||||
@Override
|
@Override
|
||||||
public boolean checkRetry(Error err) {
|
public boolean checkRetry(Error err) {
|
||||||
|
|
|
@ -71,6 +71,7 @@ import org.apache.solr.common.cloud.DocRouter;
|
||||||
import org.apache.solr.common.cloud.ImplicitDocRouter;
|
import org.apache.solr.common.cloud.ImplicitDocRouter;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||||
|
@ -83,7 +84,6 @@ import org.apache.solr.common.util.NamedList;
|
||||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||||
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
||||||
import org.apache.solr.common.util.StrUtils;
|
import org.apache.solr.common.util.StrUtils;
|
||||||
import org.apache.solr.common.util.Utils;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.slf4j.MDC;
|
import org.slf4j.MDC;
|
||||||
|
@ -1093,14 +1093,12 @@ public abstract class BaseCloudSolrClient extends SolrClient {
|
||||||
if (!liveNodes.isEmpty()) {
|
if (!liveNodes.isEmpty()) {
|
||||||
List<String> liveNodesList = new ArrayList<>(liveNodes);
|
List<String> liveNodesList = new ArrayList<>(liveNodes);
|
||||||
Collections.shuffle(liveNodesList, rand);
|
Collections.shuffle(liveNodesList, rand);
|
||||||
theUrlList.add(Utils.getBaseUrlForNodeName(liveNodesList.get(0),
|
theUrlList.add(UrlScheme.INSTANCE.getBaseUrlForNodeName(liveNodesList.get(0)));
|
||||||
getClusterStateProvider().getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
} else if (ADMIN_PATHS.contains(request.getPath())) {
|
} else if (ADMIN_PATHS.contains(request.getPath())) {
|
||||||
for (String liveNode : liveNodes) {
|
for (String liveNode : liveNodes) {
|
||||||
theUrlList.add(Utils.getBaseUrlForNodeName(liveNode,
|
theUrlList.add(UrlScheme.INSTANCE.getBaseUrlForNodeName(liveNode));
|
||||||
getClusterStateProvider().getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
} else { // Typical...
|
} else { // Typical...
|
||||||
|
|
|
@ -141,7 +141,7 @@ public class ClusterStateUtil {
|
||||||
// on a live node?
|
// on a live node?
|
||||||
boolean live = clusterState.liveNodesContain(replica.getNodeName());
|
boolean live = clusterState.liveNodesContain(replica.getNodeName());
|
||||||
String rcoreNodeName = replica.getName();
|
String rcoreNodeName = replica.getName();
|
||||||
String rbaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
|
String rbaseUrl = replica.getBaseUrl();
|
||||||
if (live && coreNodeName.equals(rcoreNodeName)
|
if (live && coreNodeName.equals(rcoreNodeName)
|
||||||
&& baseUrl.equals(rbaseUrl)) {
|
&& baseUrl.equals(rbaseUrl)) {
|
||||||
// found it
|
// found it
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.noggit.JSONWriter;
|
||||||
|
|
||||||
import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
|
import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
|
||||||
import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
|
import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
|
||||||
|
|
||||||
public class Replica extends ZkNodeProps implements MapWriter {
|
public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
|
|
||||||
|
@ -139,6 +140,8 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
// default to ACTIVE
|
// default to ACTIVE
|
||||||
this.state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, State.ACTIVE.toString())));
|
this.state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, State.ACTIVE.toString())));
|
||||||
validate();
|
validate();
|
||||||
|
|
||||||
|
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
|
||||||
}
|
}
|
||||||
|
|
||||||
// clone constructor
|
// clone constructor
|
||||||
|
@ -156,6 +159,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
this.propMap.putAll(props);
|
this.propMap.putAll(props);
|
||||||
}
|
}
|
||||||
validate();
|
validate();
|
||||||
|
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -177,7 +181,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
state = State.getState(String.valueOf(details.getOrDefault(ZkStateReader.STATE_PROP, "active")));
|
state = State.getState(String.valueOf(details.getOrDefault(ZkStateReader.STATE_PROP, "active")));
|
||||||
this.propMap.putAll(details);
|
this.propMap.putAll(details);
|
||||||
validate();
|
validate();
|
||||||
|
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
|
||||||
}
|
}
|
||||||
|
|
||||||
private final void validate() {
|
private final void validate() {
|
||||||
|
@ -198,8 +202,6 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
propMap.put(ZkStateReader.STATE_PROP, state.toString());
|
propMap.put(ZkStateReader.STATE_PROP, state.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
public String getCollection() {
|
public String getCollection() {
|
||||||
return collection;
|
return collection;
|
||||||
}
|
}
|
||||||
|
@ -235,11 +237,11 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getCoreUrl() {
|
public String getCoreUrl() {
|
||||||
return ZkCoreNodeProps.getCoreUrl(getStr(ZkStateReader.BASE_URL_PROP), core);
|
return ZkCoreNodeProps.getCoreUrl(getBaseUrl(), core);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getBaseUrl() {
|
public String getBaseUrl() {
|
||||||
return getStr(ZkStateReader.BASE_URL_PROP);
|
return getStr(BASE_URL_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** SolrCore name. */
|
/** SolrCore name. */
|
||||||
|
@ -295,8 +297,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Object clone() {
|
public Object clone() {
|
||||||
return new Replica(name, node, collection, shard, core, state, type,
|
return new Replica(name, node, collection, shard, core, state, type, propMap);
|
||||||
propMap);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -314,8 +315,13 @@ public class Replica extends ZkNodeProps implements MapWriter {
|
||||||
// propMap takes precedence because it's mutable and we can't control its
|
// propMap takes precedence because it's mutable and we can't control its
|
||||||
// contents, so a third party may override some declared fields
|
// contents, so a third party may override some declared fields
|
||||||
for (Map.Entry<String, Object> e : propMap.entrySet()) {
|
for (Map.Entry<String, Object> e : propMap.entrySet()) {
|
||||||
writer.put(e.getKey(), e.getValue(), p);
|
final String key = e.getKey();
|
||||||
|
// don't store the base_url as we can compute it from the node_name
|
||||||
|
if (!BASE_URL_PROP.equals(key)) {
|
||||||
|
writer.put(e.getKey(), e.getValue(), p);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.put(ZkStateReader.CORE_NAME_PROP, core, p)
|
writer.put(ZkStateReader.CORE_NAME_PROP, core, p)
|
||||||
.put(ZkStateReader.SHARD_ID_PROP, shard, p)
|
.put(ZkStateReader.SHARD_ID_PROP, shard, p)
|
||||||
.put(ZkStateReader.COLLECTION_PROP, collection, p)
|
.put(ZkStateReader.COLLECTION_PROP, collection, p)
|
||||||
|
|
|
@ -0,0 +1,73 @@
|
||||||
|
/*
|
||||||
|
* 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.common.cloud;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import org.apache.solr.common.util.Utils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Singleton access to global urlScheme, which although is stored in ZK as a cluster property
|
||||||
|
* really should be treated like a static global that is set at initialization and not altered after.
|
||||||
|
*
|
||||||
|
* Client applications should not use this class directly; it is only included in SolrJ because Replica
|
||||||
|
* and ZkNodeProps depend on it.
|
||||||
|
*/
|
||||||
|
public enum UrlScheme {
|
||||||
|
INSTANCE;
|
||||||
|
|
||||||
|
public static final String HTTP = "http";
|
||||||
|
public static final String HTTPS = "https";
|
||||||
|
public static final String HTTPS_PORT_PROP = "solr.jetty.https.port";
|
||||||
|
|
||||||
|
private volatile String urlScheme = HTTP;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the global urlScheme variable; ideally this should be immutable once set, but some tests rely on changing
|
||||||
|
* the value on-the-fly.
|
||||||
|
* @param urlScheme The new URL scheme, either http or https.
|
||||||
|
*/
|
||||||
|
public void setUrlScheme(final String urlScheme) {
|
||||||
|
if (HTTP.equals(urlScheme) || HTTPS.equals(urlScheme)) {
|
||||||
|
this.urlScheme = urlScheme;
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Invalid urlScheme: "+urlScheme);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getBaseUrlForNodeName(String nodeName) {
|
||||||
|
Objects.requireNonNull(nodeName,"node_name must not be null");
|
||||||
|
return Utils.getBaseUrlForNodeName(nodeName, urlScheme);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given a URL string with or without a scheme, return a new URL with the correct scheme applied.
|
||||||
|
* @param url A URL to change the scheme (http|https)
|
||||||
|
* @return A new URL with the correct scheme
|
||||||
|
*/
|
||||||
|
public String applyUrlScheme(final String url) {
|
||||||
|
Objects.requireNonNull(url, "URL must not be null!");
|
||||||
|
|
||||||
|
// heal an incorrect scheme if needed, otherwise return null indicating no change
|
||||||
|
final int at = url.indexOf("://");
|
||||||
|
return (at == -1) ? (urlScheme + "://" + url) : urlScheme + url.substring(at);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getUrlScheme() {
|
||||||
|
return urlScheme;
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,15 +16,17 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.solr.common.cloud;
|
package org.apache.solr.common.cloud;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
public class ZkCoreNodeProps {
|
public class ZkCoreNodeProps {
|
||||||
private ZkNodeProps nodeProps;
|
private final ZkNodeProps nodeProps;
|
||||||
|
|
||||||
public ZkCoreNodeProps(ZkNodeProps nodeProps) {
|
public ZkCoreNodeProps(ZkNodeProps nodeProps) {
|
||||||
this.nodeProps = nodeProps;
|
this.nodeProps = nodeProps;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getCoreUrl() {
|
public String getCoreUrl() {
|
||||||
return getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), nodeProps.getStr(ZkStateReader.CORE_NAME_PROP));
|
return getCoreUrl(this.nodeProps);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getNodeName() {
|
public String getNodeName() {
|
||||||
|
@ -36,22 +38,35 @@ public class ZkCoreNodeProps {
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getBaseUrl() {
|
public String getBaseUrl() {
|
||||||
return nodeProps.getStr(ZkStateReader.BASE_URL_PROP);
|
return getBaseUrl(this.nodeProps);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getCoreName() {
|
public String getCoreName() {
|
||||||
return nodeProps.getStr(ZkStateReader.CORE_NAME_PROP);
|
return nodeProps.getStr(ZkStateReader.CORE_NAME_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String getBaseUrl(ZkNodeProps nodeProps) {
|
||||||
|
String baseUrl = null;
|
||||||
|
final String nodeName = nodeProps.getStr(ZkStateReader.NODE_NAME_PROP);
|
||||||
|
if (nodeName != null) {
|
||||||
|
baseUrl = UrlScheme.INSTANCE.getBaseUrlForNodeName(nodeName);
|
||||||
|
} else if (nodeProps.containsKey(ZkStateReader.BASE_URL_PROP)) {
|
||||||
|
baseUrl = UrlScheme.INSTANCE.applyUrlScheme(nodeProps.getStr(ZkStateReader.BASE_URL_PROP));
|
||||||
|
}
|
||||||
|
return baseUrl;
|
||||||
|
}
|
||||||
|
|
||||||
public static String getCoreUrl(ZkNodeProps nodeProps) {
|
public static String getCoreUrl(ZkNodeProps nodeProps) {
|
||||||
return getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), nodeProps.getStr(ZkStateReader.CORE_NAME_PROP));
|
String baseUrl = getBaseUrl(nodeProps);
|
||||||
|
return baseUrl != null ? getCoreUrl(baseUrl, nodeProps.getStr(ZkStateReader.CORE_NAME_PROP)) : null;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static String getCoreUrl(String baseUrl, String coreName) {
|
public static String getCoreUrl(String baseUrl, String coreName) {
|
||||||
|
Objects.requireNonNull(baseUrl,"baseUrl must not be null");
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
sb.append(baseUrl);
|
sb.append(baseUrl);
|
||||||
if (!baseUrl.endsWith("/")) sb.append("/");
|
if (!baseUrl.endsWith("/")) sb.append("/");
|
||||||
sb.append(coreName);
|
sb.append(coreName != null ? coreName : "");
|
||||||
if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
|
if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
@ -68,6 +83,4 @@ public class ZkCoreNodeProps {
|
||||||
public boolean isLeader() {
|
public boolean isLeader() {
|
||||||
return nodeProps.containsKey(ZkStateReader.LEADER_PROP);
|
return nodeProps.containsKey(ZkStateReader.LEADER_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,7 +17,12 @@
|
||||||
package org.apache.solr.common.cloud;
|
package org.apache.solr.common.cloud;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.*;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.solr.common.util.JavaBinCodec;
|
import org.apache.solr.common.util.JavaBinCodec;
|
||||||
import org.apache.solr.common.util.Utils;
|
import org.apache.solr.common.util.Utils;
|
||||||
|
@ -37,6 +42,13 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
*/
|
*/
|
||||||
public ZkNodeProps(Map<String,Object> propMap) {
|
public ZkNodeProps(Map<String,Object> propMap) {
|
||||||
this.propMap = propMap;
|
this.propMap = propMap;
|
||||||
|
|
||||||
|
// don't store base_url if we have a node_name to recompute from when we read back from ZK
|
||||||
|
// sub-classes that know they need a base_url (Replica) can eagerly compute in their ctor
|
||||||
|
if (this.propMap.containsKey(ZkStateReader.NODE_NAME_PROP)) {
|
||||||
|
this.propMap.remove(ZkStateReader.BASE_URL_PROP);
|
||||||
|
}
|
||||||
|
|
||||||
// TODO: store an unmodifiable map, but in a way that guarantees not to wrap more than once.
|
// TODO: store an unmodifiable map, but in a way that guarantees not to wrap more than once.
|
||||||
// Always wrapping introduces a memory leak.
|
// Always wrapping introduces a memory leak.
|
||||||
}
|
}
|
||||||
|
@ -90,7 +102,7 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings({"unchecked"})
|
@SuppressWarnings({"unchecked"})
|
||||||
public static ZkNodeProps load(byte[] bytes) {
|
public static ZkNodeProps load(byte[] bytes) {
|
||||||
Map<String, Object> props = null;
|
Map<String, Object> props;
|
||||||
if (bytes[0] == 2) {
|
if (bytes[0] == 2) {
|
||||||
try (JavaBinCodec jbc = new JavaBinCodec()) {
|
try (JavaBinCodec jbc = new JavaBinCodec()) {
|
||||||
props = (Map<String, Object>) jbc.unmarshal(bytes);
|
props = (Map<String, Object>) jbc.unmarshal(bytes);
|
||||||
|
@ -105,15 +117,26 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void write(JSONWriter jsonWriter) {
|
public void write(JSONWriter jsonWriter) {
|
||||||
jsonWriter.write(propMap);
|
// don't write out the base_url if we have a node_name
|
||||||
|
if (propMap.containsKey(ZkStateReader.BASE_URL_PROP) && propMap.containsKey(ZkStateReader.NODE_NAME_PROP)) {
|
||||||
|
final Map<String,Object> filtered = new HashMap<>();
|
||||||
|
// stream / collect is no good here as the Collector doesn't like null values
|
||||||
|
propMap.forEach((key, value) -> {
|
||||||
|
if (!ZkStateReader.BASE_URL_PROP.equals(key)) {
|
||||||
|
filtered.put(key, value);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
jsonWriter.write(filtered);
|
||||||
|
} else {
|
||||||
|
jsonWriter.write(propMap);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a string property value.
|
* Get a string property value.
|
||||||
*/
|
*/
|
||||||
public String getStr(String key) {
|
public String getStr(String key) {
|
||||||
Object o = propMap.get(key);
|
return getStr(key, null);
|
||||||
return o == null ? null : o.toString();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -127,8 +150,17 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
/**
|
/**
|
||||||
* Get a string property value.
|
* Get a string property value.
|
||||||
*/
|
*/
|
||||||
public String getStr(String key,String def) {
|
public String getStr(String key, String def) {
|
||||||
Object o = propMap.get(key);
|
Object o = propMap.get(key);
|
||||||
|
|
||||||
|
// TODO: This "hack" should not be needed but keeping it here b/c we removed the base_url from the map in the ctor
|
||||||
|
if (o == null && def == null && ZkStateReader.BASE_URL_PROP.equals(key)) {
|
||||||
|
final String nodeName = (String)propMap.get(ZkStateReader.NODE_NAME_PROP);
|
||||||
|
if (nodeName != null) {
|
||||||
|
o = UrlScheme.INSTANCE.getBaseUrlForNodeName(nodeName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return o == null ? def : o.toString();
|
return o == null ? def : o.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -139,14 +171,6 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return toJSONString(this);
|
return toJSONString(this);
|
||||||
/***
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
Set<Entry<String,Object>> entries = propMap.entrySet();
|
|
||||||
for(Entry<String,Object> entry : entries) {
|
|
||||||
sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
|
|
||||||
}
|
|
||||||
return sb.toString();
|
|
||||||
***/
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -155,7 +179,7 @@ public class ZkNodeProps implements JSONWriter.Writable {
|
||||||
public boolean containsKey(String key) {
|
public boolean containsKey(String key) {
|
||||||
return propMap.containsKey(key);
|
return propMap.containsKey(key);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean getBool(String key, boolean b) {
|
public boolean getBool(String key, boolean b) {
|
||||||
Object o = propMap.get(key);
|
Object o = propMap.get(key);
|
||||||
if (o == null) return b;
|
if (o == null) return b;
|
||||||
|
|
|
@ -51,7 +51,11 @@ import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.SolrException.ErrorCode;
|
import org.apache.solr.common.SolrException.ErrorCode;
|
||||||
import org.apache.solr.common.params.CollectionAdminParams;
|
import org.apache.solr.common.params.CollectionAdminParams;
|
||||||
import org.apache.solr.common.params.CoreAdminParams;
|
import org.apache.solr.common.params.CoreAdminParams;
|
||||||
import org.apache.solr.common.util.*;
|
import org.apache.solr.common.util.ExecutorUtil;
|
||||||
|
import org.apache.solr.common.util.ObjectReleaseTracker;
|
||||||
|
import org.apache.solr.common.util.Pair;
|
||||||
|
import org.apache.solr.common.util.SolrNamedThreadFactory;
|
||||||
|
import org.apache.solr.common.util.Utils;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||||
import org.apache.zookeeper.WatchedEvent;
|
import org.apache.zookeeper.WatchedEvent;
|
||||||
|
@ -63,6 +67,7 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static java.util.Collections.EMPTY_MAP;
|
import static java.util.Collections.EMPTY_MAP;
|
||||||
import static java.util.Collections.emptySortedSet;
|
import static java.util.Collections.emptySortedSet;
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTP;
|
||||||
import static org.apache.solr.common.util.Utils.fromJSON;
|
import static org.apache.solr.common.util.Utils.fromJSON;
|
||||||
|
|
||||||
public class ZkStateReader implements SolrCloseable {
|
public class ZkStateReader implements SolrCloseable {
|
||||||
|
@ -798,7 +803,11 @@ public class ZkStateReader implements SolrCloseable {
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getLeaderUrl(String collection, String shard, int timeout) throws InterruptedException {
|
public String getLeaderUrl(String collection, String shard, int timeout) throws InterruptedException {
|
||||||
ZkCoreNodeProps props = new ZkCoreNodeProps(getLeaderRetry(collection, shard, timeout));
|
Replica replica = getLeaderRetry(collection, shard, timeout);
|
||||||
|
if (replica == null || replica.getBaseUrl() == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
ZkCoreNodeProps props = new ZkCoreNodeProps(replica);
|
||||||
return props.getCoreUrl();
|
return props.getCoreUrl();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1003,6 +1012,9 @@ public class ZkStateReader implements SolrCloseable {
|
||||||
this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
|
this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
|
||||||
log.debug("Loaded cluster properties: {}", this.clusterProperties);
|
log.debug("Loaded cluster properties: {}", this.clusterProperties);
|
||||||
|
|
||||||
|
// Make the urlScheme globally accessible
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(getClusterProperty(ZkStateReader.URL_SCHEME, HTTP));
|
||||||
|
|
||||||
for (ClusterPropertiesListener listener : clusterPropertiesListeners) {
|
for (ClusterPropertiesListener listener : clusterPropertiesListeners) {
|
||||||
listener.onChange(getClusterProperties());
|
listener.onChange(getClusterProperties());
|
||||||
}
|
}
|
||||||
|
|
|
@ -61,7 +61,6 @@ import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.DocRouter;
|
import org.apache.solr.common.cloud.DocRouter;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.common.params.CommonParams;
|
import org.apache.solr.common.params.CommonParams;
|
||||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||||
import org.apache.solr.common.params.ShardParams;
|
import org.apache.solr.common.params.ShardParams;
|
||||||
|
@ -80,7 +79,7 @@ import org.junit.rules.ExpectedException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.apache.solr.client.solrj.impl.BaseCloudSolrClient.*;
|
import static org.apache.solr.client.solrj.impl.BaseCloudSolrClient.RouteResponse;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -333,7 +332,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
|
||||||
Map<String, Long> requestCountsMap = Maps.newHashMap();
|
Map<String, Long> requestCountsMap = Maps.newHashMap();
|
||||||
for (Slice slice : col.getSlices()) {
|
for (Slice slice : col.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
String baseURL = replica.getBaseUrl();
|
||||||
requestCountsMap.put(baseURL, getNumRequests(baseURL, "routing_collection"));
|
requestCountsMap.put(baseURL, getNumRequests(baseURL, "routing_collection"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -344,7 +343,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
|
||||||
Set<String> expectedBaseURLs = Sets.newHashSet();
|
Set<String> expectedBaseURLs = Sets.newHashSet();
|
||||||
for (Slice expectedSlice : expectedSlices) {
|
for (Slice expectedSlice : expectedSlices) {
|
||||||
for (Replica replica : expectedSlice.getReplicas()) {
|
for (Replica replica : expectedSlice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
String baseURL = replica.getBaseUrl();
|
||||||
expectedBaseURLs.add(baseURL);
|
expectedBaseURLs.add(baseURL);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -391,7 +390,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
|
||||||
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
|
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
|
||||||
for (Slice slice : col.getSlices()) {
|
for (Slice slice : col.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
String baseURL = replica.getBaseUrl();
|
||||||
|
|
||||||
Long prevNumRequests = requestCountsMap.get(baseURL);
|
Long prevNumRequests = requestCountsMap.get(baseURL);
|
||||||
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
|
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
|
||||||
|
@ -700,7 +699,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
|
||||||
SolrQuery q = new SolrQuery().setQuery("*:*");
|
SolrQuery q = new SolrQuery().setQuery("*:*");
|
||||||
BaseHttpSolrClient.RemoteSolrException sse = null;
|
BaseHttpSolrClient.RemoteSolrException sse = null;
|
||||||
|
|
||||||
final String url = r.getStr(ZkStateReader.BASE_URL_PROP) + "/" + COLLECTION;
|
final String url = r.getBaseUrl() + "/" + COLLECTION;
|
||||||
try (HttpSolrClient solrClient = getHttpSolrClient(url)) {
|
try (HttpSolrClient solrClient = getHttpSolrClient(url)) {
|
||||||
|
|
||||||
if (log.isInfoEnabled()) {
|
if (log.isInfoEnabled()) {
|
||||||
|
@ -727,7 +726,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
|
||||||
Set<String> allNodesOfColl = new HashSet<>();
|
Set<String> allNodesOfColl = new HashSet<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP));
|
allNodesOfColl.add(replica.getBaseUrl());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
String theNode = null;
|
String theNode = null;
|
||||||
|
|
|
@ -49,8 +49,8 @@ import org.apache.solr.client.solrj.request.QueryRequest;
|
||||||
import org.apache.solr.client.solrj.request.UpdateRequest;
|
import org.apache.solr.client.solrj.request.UpdateRequest;
|
||||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||||
import org.apache.solr.client.solrj.response.RequestStatusState;
|
import org.apache.solr.client.solrj.response.RequestStatusState;
|
||||||
import org.apache.solr.client.solrj.response.UpdateResponse;
|
|
||||||
import org.apache.solr.client.solrj.response.SolrPingResponse;
|
import org.apache.solr.client.solrj.response.SolrPingResponse;
|
||||||
|
import org.apache.solr.client.solrj.response.UpdateResponse;
|
||||||
import org.apache.solr.cloud.AbstractDistribZkTestBase;
|
import org.apache.solr.cloud.AbstractDistribZkTestBase;
|
||||||
import org.apache.solr.cloud.SolrCloudTestCase;
|
import org.apache.solr.cloud.SolrCloudTestCase;
|
||||||
import org.apache.solr.common.SolrDocument;
|
import org.apache.solr.common.SolrDocument;
|
||||||
|
@ -62,7 +62,6 @@ import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.DocRouter;
|
import org.apache.solr.common.cloud.DocRouter;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
|
||||||
import org.apache.solr.common.params.CommonParams;
|
import org.apache.solr.common.params.CommonParams;
|
||||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||||
import org.apache.solr.common.params.ShardParams;
|
import org.apache.solr.common.params.ShardParams;
|
||||||
|
@ -319,7 +318,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
|
||||||
Map<String, Long> requestCountsMap = Maps.newHashMap();
|
Map<String, Long> requestCountsMap = Maps.newHashMap();
|
||||||
for (Slice slice : col.getSlices()) {
|
for (Slice slice : col.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
String baseURL = replica.getBaseUrl();
|
||||||
requestCountsMap.put(baseURL, getNumRequests(baseURL, "routing_collection"));
|
requestCountsMap.put(baseURL, getNumRequests(baseURL, "routing_collection"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -330,8 +329,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
|
||||||
Set<String> expectedBaseURLs = Sets.newHashSet();
|
Set<String> expectedBaseURLs = Sets.newHashSet();
|
||||||
for (Slice expectedSlice : expectedSlices) {
|
for (Slice expectedSlice : expectedSlices) {
|
||||||
for (Replica replica : expectedSlice.getReplicas()) {
|
for (Replica replica : expectedSlice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
expectedBaseURLs.add(replica.getBaseUrl());
|
||||||
expectedBaseURLs.add(baseURL);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -377,7 +375,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
|
||||||
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
|
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
|
||||||
for (Slice slice : col.getSlices()) {
|
for (Slice slice : col.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
|
String baseURL = replica.getBaseUrl();
|
||||||
|
|
||||||
Long prevNumRequests = requestCountsMap.get(baseURL);
|
Long prevNumRequests = requestCountsMap.get(baseURL);
|
||||||
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
|
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
|
||||||
|
@ -684,7 +682,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
|
||||||
SolrQuery q = new SolrQuery().setQuery("*:*");
|
SolrQuery q = new SolrQuery().setQuery("*:*");
|
||||||
BaseHttpSolrClient.RemoteSolrException sse = null;
|
BaseHttpSolrClient.RemoteSolrException sse = null;
|
||||||
|
|
||||||
final String url = r.getStr(ZkStateReader.BASE_URL_PROP) + "/" + COLLECTION;
|
final String url = r.getBaseUrl() + "/" + COLLECTION;
|
||||||
try (HttpSolrClient solrClient = getHttpSolrClient(url)) {
|
try (HttpSolrClient solrClient = getHttpSolrClient(url)) {
|
||||||
|
|
||||||
if (log.isInfoEnabled()) {
|
if (log.isInfoEnabled()) {
|
||||||
|
@ -711,7 +709,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
|
||||||
Set<String> allNodesOfColl = new HashSet<>();
|
Set<String> allNodesOfColl = new HashSet<>();
|
||||||
for (Slice slice : coll.getSlices()) {
|
for (Slice slice : coll.getSlices()) {
|
||||||
for (Replica replica : slice.getReplicas()) {
|
for (Replica replica : slice.getReplicas()) {
|
||||||
allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP));
|
allNodesOfColl.add(replica.getBaseUrl());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
String theNode = null;
|
String theNode = null;
|
||||||
|
|
|
@ -21,11 +21,13 @@ import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.solr.SolrTestCaseJ4;
|
import org.apache.solr.SolrTestCaseJ4;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.common.params.ShardParams;
|
import org.apache.solr.common.params.ShardParams;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@SolrTestCaseJ4.SuppressSSL // this test is all about http://
|
||||||
public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -33,11 +35,11 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
List<Replica> replicas = getBasicReplicaList();
|
List<Replica> replicas = getBasicReplicaList();
|
||||||
|
|
||||||
// replicaLocation rule
|
// replicaLocation rule
|
||||||
List<PreferenceRule> rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_LOCATION + ":http://host2:8983");
|
List<PreferenceRule> rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_LOCATION + ":http://node2:8983");
|
||||||
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
||||||
replicas.sort(comparator);
|
replicas.sort(comparator);
|
||||||
assertEquals("node2", replicas.get(0).getNodeName());
|
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node1", replicas.get(1).getNodeName());
|
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -49,8 +51,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
||||||
|
|
||||||
replicas.sort(comparator);
|
replicas.sort(comparator);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(1).getNodeName());
|
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
|
||||||
|
|
||||||
// reversed rule
|
// reversed rule
|
||||||
rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
|
rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
|
||||||
|
@ -58,8 +60,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
comparator = new NodePreferenceRulesComparator(rules, null);
|
comparator = new NodePreferenceRulesComparator(rules, null);
|
||||||
|
|
||||||
replicas.sort(comparator);
|
replicas.sort(comparator);
|
||||||
assertEquals("node2", replicas.get(0).getNodeName());
|
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node1", replicas.get(1).getNodeName());
|
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
|
@ -71,8 +73,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node4",
|
"node4",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node4:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node4",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "TLOG"
|
ZkStateReader.REPLICA_TYPE, "TLOG"
|
||||||
),"collection1","shard1"
|
),"collection1","shard1"
|
||||||
|
@ -82,14 +83,14 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
List<PreferenceRule> rules = PreferenceRule.from(
|
List<PreferenceRule> rules = PreferenceRule.from(
|
||||||
ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":NRT," +
|
ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":NRT," +
|
||||||
ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
|
ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
|
||||||
ShardParams.SHARDS_PREFERENCE_REPLICA_LOCATION + ":http://host2_2");
|
ShardParams.SHARDS_PREFERENCE_REPLICA_LOCATION + ":http://node4");
|
||||||
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
|
||||||
|
|
||||||
replicas.sort(comparator);
|
replicas.sort(comparator);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node4", replicas.get(1).getNodeName());
|
assertEquals("node4", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(2).getNodeName());
|
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(3).getNodeName());
|
assertEquals("node3", getHost(replicas.get(3).getNodeName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = IllegalArgumentException.class)
|
@Test(expected = IllegalArgumentException.class)
|
||||||
|
@ -116,13 +117,14 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private static List<Replica> getBasicReplicaList() {
|
private static List<Replica> getBasicReplicaList() {
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
|
||||||
|
|
||||||
List<Replica> replicas = new ArrayList<Replica>();
|
List<Replica> replicas = new ArrayList<Replica>();
|
||||||
replicas.add(
|
replicas.add(
|
||||||
new Replica(
|
new Replica(
|
||||||
"node1",
|
"node1",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host1:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node1:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "NRT"
|
ZkStateReader.REPLICA_TYPE, "NRT"
|
||||||
),"collection1","shard1"
|
),"collection1","shard1"
|
||||||
|
@ -132,8 +134,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node2",
|
"node2",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node2:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node2",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "TLOG"
|
ZkStateReader.REPLICA_TYPE, "TLOG"
|
||||||
),"collection1","shard1"
|
),"collection1","shard1"
|
||||||
|
@ -143,8 +144,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node3",
|
"node3",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node3:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node3",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "PULL"
|
ZkStateReader.REPLICA_TYPE, "PULL"
|
||||||
),"collection1","shard1"
|
),"collection1","shard1"
|
||||||
|
@ -152,4 +152,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
|
||||||
);
|
);
|
||||||
return replicas;
|
return replicas;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private String getHost(final String nodeName) {
|
||||||
|
final int colonAt = nodeName.indexOf(':');
|
||||||
|
return colonAt != -1 ? nodeName.substring(0,colonAt) : nodeName.substring(0, nodeName.indexOf('_'));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,30 +42,35 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
|
|
||||||
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
|
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(1).getNodeName());
|
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(2).getNodeName());
|
assertEquals("node3", getHost(replicas.get(2).getNodeName()));
|
||||||
|
|
||||||
params.set("routingPreference", "1");
|
params.set("routingPreference", "1");
|
||||||
rlt = generator.getReplicaListTransformer(params);
|
rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node2", replicas.get(0).getNodeName());
|
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(1).getNodeName());
|
assertEquals("node3", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node1", replicas.get(2).getNodeName());
|
assertEquals("node1", getHost(replicas.get(2).getNodeName()));
|
||||||
|
|
||||||
params.set("routingPreference", "2");
|
params.set("routingPreference", "2");
|
||||||
rlt = generator.getReplicaListTransformer(params);
|
rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node3", replicas.get(0).getNodeName());
|
assertEquals("node3", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node1", replicas.get(1).getNodeName());
|
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(2).getNodeName());
|
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
|
||||||
|
|
||||||
params.set("routingPreference", "3");
|
params.set("routingPreference", "3");
|
||||||
rlt = generator.getReplicaListTransformer(params);
|
rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(1).getNodeName());
|
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(2).getNodeName());
|
assertEquals("node3", getHost(replicas.get(2).getNodeName()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getHost(final String nodeName) {
|
||||||
|
final int colonAt = nodeName.indexOf(':');
|
||||||
|
return colonAt != -1 ? nodeName.substring(0,colonAt) : nodeName.substring(0, nodeName.indexOf('_'));
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
|
@ -80,8 +85,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node4",
|
"node4",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node4:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node4",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "TLOG"
|
ZkStateReader.REPLICA_TYPE, "TLOG"
|
||||||
), "c1","s1"
|
), "c1","s1"
|
||||||
|
@ -93,8 +97,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node5",
|
"node5",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node5:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node5",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "PULL"
|
ZkStateReader.REPLICA_TYPE, "PULL"
|
||||||
), "c1","s1"
|
), "c1","s1"
|
||||||
|
@ -110,20 +113,20 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
params.add(ShardParams.SHARDS_PREFERENCE, rulesParam);
|
params.add(ShardParams.SHARDS_PREFERENCE, rulesParam);
|
||||||
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
|
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(1).getNodeName());
|
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node4", replicas.get(2).getNodeName());
|
assertEquals("node4", getHost(replicas.get(2).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(3).getNodeName());
|
assertEquals("node3", getHost(replicas.get(3).getNodeName()));
|
||||||
assertEquals("node5", replicas.get(4).getNodeName());
|
assertEquals("node5", getHost(replicas.get(4).getNodeName()));
|
||||||
|
|
||||||
params.set("routingPreference", "1");
|
params.set("routingPreference", "1");
|
||||||
rlt = generator.getReplicaListTransformer(params);
|
rlt = generator.getReplicaListTransformer(params);
|
||||||
rlt.transform(replicas);
|
rlt.transform(replicas);
|
||||||
assertEquals("node1", replicas.get(0).getNodeName());
|
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
|
||||||
assertEquals("node4", replicas.get(1).getNodeName());
|
assertEquals("node4", getHost(replicas.get(1).getNodeName()));
|
||||||
assertEquals("node2", replicas.get(2).getNodeName());
|
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
|
||||||
assertEquals("node5", replicas.get(3).getNodeName());
|
assertEquals("node5", getHost(replicas.get(3).getNodeName()));
|
||||||
assertEquals("node3", replicas.get(4).getNodeName());
|
assertEquals("node3", getHost(replicas.get(4).getNodeName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
|
@ -133,8 +136,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node1",
|
"node1",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host1:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node1:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "NRT"
|
ZkStateReader.REPLICA_TYPE, "NRT"
|
||||||
),"c1","s1"
|
),"c1","s1"
|
||||||
|
@ -144,8 +146,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node2",
|
"node2",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node2:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node2",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "TLOG"
|
ZkStateReader.REPLICA_TYPE, "TLOG"
|
||||||
),"c1","s1"
|
),"c1","s1"
|
||||||
|
@ -155,8 +156,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
|
||||||
new Replica(
|
new Replica(
|
||||||
"node3",
|
"node3",
|
||||||
map(
|
map(
|
||||||
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
|
ZkStateReader.NODE_NAME_PROP, "node3:8983_solr",
|
||||||
ZkStateReader.NODE_NAME_PROP, "node3",
|
|
||||||
ZkStateReader.CORE_NAME_PROP, "collection1",
|
ZkStateReader.CORE_NAME_PROP, "collection1",
|
||||||
ZkStateReader.REPLICA_TYPE, "PULL"
|
ZkStateReader.REPLICA_TYPE, "PULL"
|
||||||
),"c1","s1"
|
),"c1","s1"
|
||||||
|
|
|
@ -40,7 +40,7 @@ public class ShufflingReplicaListTransformerTest extends SolrTestCase {
|
||||||
Map<String, Object> propMap = new HashMap<>();
|
Map<String, Object> propMap = new HashMap<>();
|
||||||
propMap.put("core", "core" + counter);
|
propMap.put("core", "core" + counter);
|
||||||
propMap.put("type", "NRT");
|
propMap.put("type", "NRT");
|
||||||
propMap.put("node_name", "node" + counter);
|
propMap.put("node_name", "node" + counter + "_");
|
||||||
counter++;
|
counter++;
|
||||||
replicas.add(new Replica(url, propMap, "c1", "s1"));
|
replicas.add(new Replica(url, propMap, "c1", "s1"));
|
||||||
}
|
}
|
||||||
|
@ -76,7 +76,7 @@ public class ShufflingReplicaListTransformerTest extends SolrTestCase {
|
||||||
|
|
||||||
private final void maybeAddUrl(final List<String> urls, final String url) {
|
private final void maybeAddUrl(final List<String> urls, final String url) {
|
||||||
if (random().nextBoolean()) {
|
if (random().nextBoolean()) {
|
||||||
urls.add(url);
|
urls.add(url + "_");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,56 @@
|
||||||
|
/*
|
||||||
|
* 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.common.cloud;
|
||||||
|
|
||||||
|
import org.apache.solr.SolrTestCase;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTP;
|
||||||
|
import static org.apache.solr.common.cloud.UrlScheme.HTTPS;
|
||||||
|
|
||||||
|
public class UrlSchemeTest extends SolrTestCase {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testApplyUrlScheme() {
|
||||||
|
|
||||||
|
final UrlScheme t = UrlScheme.INSTANCE;
|
||||||
|
t.setUrlScheme(HTTPS);
|
||||||
|
|
||||||
|
// mock a SolrZkClient with some live nodes and cluster props set.
|
||||||
|
String liveNode1 = "192.168.1.1:8983_solr";
|
||||||
|
String liveNode2 = "127.0.0.1:8983_solr";
|
||||||
|
String liveNode3 = "127.0.0.1_";
|
||||||
|
String liveNode4 = "127.0.0.1:61631_l_%2Fig";
|
||||||
|
|
||||||
|
assertEquals("https://192.168.1.1:8983/solr", t.getBaseUrlForNodeName(liveNode1));
|
||||||
|
assertEquals("https://127.0.0.1:8983/solr", t.getBaseUrlForNodeName(liveNode2));
|
||||||
|
assertEquals("https://127.0.0.1", t.getBaseUrlForNodeName(liveNode3));
|
||||||
|
assertEquals("https://127.0.0.1:61631/l_/ig", t.getBaseUrlForNodeName(liveNode4));
|
||||||
|
// heal wrong scheme too
|
||||||
|
assertEquals("https://127.0.0.1:8983/solr", t.applyUrlScheme("127.0.0.1:8983/solr"));
|
||||||
|
assertEquals("https://127.0.0.1:8983/solr", t.applyUrlScheme("http://127.0.0.1:8983/solr"));
|
||||||
|
|
||||||
|
t.setUrlScheme(HTTP);
|
||||||
|
assertEquals("http://192.168.1.1:8983/solr", t.getBaseUrlForNodeName(liveNode1));
|
||||||
|
assertEquals("http://127.0.0.1:8983/solr", t.getBaseUrlForNodeName(liveNode2));
|
||||||
|
assertEquals("http://127.0.0.1", t.getBaseUrlForNodeName(liveNode3));
|
||||||
|
assertEquals("http://127.0.0.1:61631/l_/ig", t.getBaseUrlForNodeName(liveNode4));
|
||||||
|
// heal wrong scheme too
|
||||||
|
assertEquals("http://127.0.0.1:8983/solr", t.applyUrlScheme("https://127.0.0.1:8983/solr"));
|
||||||
|
}
|
||||||
|
}
|
|
@ -101,6 +101,7 @@ import org.apache.solr.common.SolrDocumentList;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.SolrInputDocument;
|
import org.apache.solr.common.SolrInputDocument;
|
||||||
import org.apache.solr.common.SolrInputField;
|
import org.apache.solr.common.SolrInputField;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.params.CommonParams;
|
import org.apache.solr.common.params.CommonParams;
|
||||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||||
import org.apache.solr.common.params.MultiMapSolrParams;
|
import org.apache.solr.common.params.MultiMapSolrParams;
|
||||||
|
@ -160,6 +161,7 @@ import org.xml.sax.SAXException;
|
||||||
|
|
||||||
import static java.util.Objects.requireNonNull;
|
import static java.util.Objects.requireNonNull;
|
||||||
import static org.apache.solr.cloud.SolrZkServer.ZK_WHITELIST_PROPERTY;
|
import static org.apache.solr.cloud.SolrZkServer.ZK_WHITELIST_PROPERTY;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.URL_SCHEME;
|
||||||
import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
|
import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
|
||||||
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
||||||
|
|
||||||
|
@ -300,7 +302,10 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
|
||||||
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
|
||||||
if(isSSLMode()) {
|
if(isSSLMode()) {
|
||||||
// SolrCloud tests should usually clear this
|
// SolrCloud tests should usually clear this
|
||||||
System.setProperty("urlScheme", "https");
|
System.setProperty(URL_SCHEME, UrlScheme.HTTPS);
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTPS);
|
||||||
|
} else {
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -340,7 +345,8 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
|
||||||
System.clearProperty("tests.shardhandler.randomSeed");
|
System.clearProperty("tests.shardhandler.randomSeed");
|
||||||
System.clearProperty("enable.update.log");
|
System.clearProperty("enable.update.log");
|
||||||
System.clearProperty("useCompoundFile");
|
System.clearProperty("useCompoundFile");
|
||||||
System.clearProperty("urlScheme");
|
System.clearProperty(URL_SCHEME);
|
||||||
|
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
|
||||||
System.clearProperty("solr.cloud.wait-for-updates-with-stale-state-pause");
|
System.clearProperty("solr.cloud.wait-for-updates-with-stale-state-pause");
|
||||||
System.clearProperty("solr.zkclienttmeout");
|
System.clearProperty("solr.zkclienttmeout");
|
||||||
System.clearProperty(ZK_WHITELIST_PROPERTY);
|
System.clearProperty(ZK_WHITELIST_PROPERTY);
|
||||||
|
|
|
@ -16,6 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.solr.cloud;
|
package org.apache.solr.cloud;
|
||||||
|
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.URL_SCHEME;
|
||||||
import static org.apache.solr.common.util.Utils.makeMap;
|
import static org.apache.solr.common.util.Utils.makeMap;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
@ -72,6 +73,7 @@ import org.apache.solr.common.cloud.ClusterState;
|
||||||
import org.apache.solr.common.cloud.DocCollection;
|
import org.apache.solr.common.cloud.DocCollection;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
import org.apache.solr.common.cloud.UrlScheme;
|
||||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
import org.apache.solr.common.cloud.ZkStateReader;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
|
@ -255,17 +257,17 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isSSLMode()) {
|
if (isSSLMode()) {
|
||||||
System.clearProperty("urlScheme");
|
System.clearProperty(URL_SCHEME);
|
||||||
try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
|
try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
|
||||||
AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT)) {
|
AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT)) {
|
||||||
try {
|
try {
|
||||||
zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
|
zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
|
||||||
Utils.toJSON(Collections.singletonMap("urlScheme", "https")),
|
Utils.toJSON(Collections.singletonMap(URL_SCHEME, "https")),
|
||||||
CreateMode.PERSISTENT, true);
|
CreateMode.PERSISTENT, true);
|
||||||
} catch (KeeperException.NodeExistsException e) {
|
} catch (KeeperException.NodeExistsException e) {
|
||||||
ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(ZkStateReader.CLUSTER_PROPS,
|
ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(ZkStateReader.CLUSTER_PROPS,
|
||||||
null, null, true));
|
null, null, true));
|
||||||
zkStateReader.getZkClient().setData(ZkStateReader.CLUSTER_PROPS, Utils.toJSON(props.plus("urlScheme", "https")), true);
|
zkStateReader.getZkClient().setData(ZkStateReader.CLUSTER_PROPS, Utils.toJSON(props.plus(URL_SCHEME, "https")), true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -776,7 +778,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
}
|
}
|
||||||
|
|
||||||
protected SocketProxy getProxyForReplica(Replica replica) throws Exception {
|
protected SocketProxy getProxyForReplica(Replica replica) throws Exception {
|
||||||
String replicaBaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
|
String replicaBaseUrl = replica.getBaseUrl();
|
||||||
assertNotNull(replicaBaseUrl);
|
assertNotNull(replicaBaseUrl);
|
||||||
|
|
||||||
List<JettySolrRunner> runners = new ArrayList<>(jettys);
|
List<JettySolrRunner> runners = new ArrayList<>(jettys);
|
||||||
|
@ -838,7 +840,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
int port = new URI(((HttpSolrClient) client).getBaseURL())
|
int port = new URI(((HttpSolrClient) client).getBaseURL())
|
||||||
.getPort();
|
.getPort();
|
||||||
|
|
||||||
if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
|
if (replica.getBaseUrl().contains(":" + port)) {
|
||||||
CloudSolrServerClient csc = new CloudSolrServerClient();
|
CloudSolrServerClient csc = new CloudSolrServerClient();
|
||||||
csc.solrClient = client;
|
csc.solrClient = client;
|
||||||
csc.port = port;
|
csc.port = port;
|
||||||
|
@ -864,7 +866,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
Set<Entry<String,Replica>> entries = slice.getReplicasMap().entrySet();
|
Set<Entry<String,Replica>> entries = slice.getReplicasMap().entrySet();
|
||||||
for (Entry<String,Replica> entry : entries) {
|
for (Entry<String,Replica> entry : entries) {
|
||||||
Replica replica = entry.getValue();
|
Replica replica = entry.getValue();
|
||||||
if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
|
if (replica.getBaseUrl().contains(":" + port)) {
|
||||||
List<CloudJettyRunner> list = shardToJetty.get(slice.getName());
|
List<CloudJettyRunner> list = shardToJetty.get(slice.getName());
|
||||||
if (list == null) {
|
if (list == null) {
|
||||||
list = new ArrayList<>();
|
list = new ArrayList<>();
|
||||||
|
@ -876,7 +878,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
cjr.info = replica;
|
cjr.info = replica;
|
||||||
cjr.nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
|
cjr.nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
|
||||||
cjr.coreNodeName = entry.getKey();
|
cjr.coreNodeName = entry.getKey();
|
||||||
cjr.url = replica.getStr(ZkStateReader.BASE_URL_PROP) + "/" + replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
// TODO: no trailing slash on end desired, so replica.getCoreUrl is not applicable here
|
||||||
|
cjr.url = replica.getBaseUrl() + "/" + replica.getStr(ZkStateReader.CORE_NAME_PROP);
|
||||||
cjr.client = findClientByPort(port, theClients);
|
cjr.client = findClientByPort(port, theClients);
|
||||||
list.add(cjr);
|
list.add(cjr);
|
||||||
if (isLeader) {
|
if (isLeader) {
|
||||||
|
@ -2000,8 +2003,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
||||||
Set<Map.Entry<String,Replica>> shardEntries = shards.entrySet();
|
Set<Map.Entry<String,Replica>> shardEntries = shards.entrySet();
|
||||||
for (Map.Entry<String,Replica> shardEntry : shardEntries) {
|
for (Map.Entry<String,Replica> shardEntry : shardEntries) {
|
||||||
final ZkNodeProps node = shardEntry.getValue();
|
final ZkNodeProps node = shardEntry.getValue();
|
||||||
if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP))) {
|
final String nodeName = node.getStr(ZkStateReader.NODE_NAME_PROP);
|
||||||
return ZkCoreNodeProps.getCoreUrl(node.getStr(ZkStateReader.BASE_URL_PROP), collection); //new ZkCoreNodeProps(node).getCoreUrl();
|
if (clusterState.liveNodesContain(nodeName)) {
|
||||||
|
return ZkCoreNodeProps.getCoreUrl(UrlScheme.INSTANCE.getBaseUrlForNodeName(nodeName), collection);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue