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:
Timothy Potter 2020-11-30 12:05:48 -07:00 committed by GitHub
parent cb5ba42bd7
commit a0492840ee
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
56 changed files with 480 additions and 299 deletions

View File

@ -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
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 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -41,7 +41,6 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
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.CoreDescriptor;
import org.apache.solr.core.SolrResourceLoader;
@ -77,7 +76,7 @@ public class CloudUtil {
for (Replica replica : slice.getReplicas()) {
String cnn = replica.getName();
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
String baseUrl = replica.getBaseUrl();
log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
if (thisCnn != null && thisCnn.equals(cnn)

View File

@ -344,16 +344,11 @@ public class RecoveryStrategy implements Runnable, Closeable {
// though
try {
CloudDescriptor cloudDesc = this.coreDescriptor.getCloudDescriptor();
ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(
cloudDesc.getCollectionName(), cloudDesc.getShardId());
final String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
final String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(cloudDesc.getCollectionName(), cloudDesc.getShardId());
final String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderprops);
final String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
boolean isLeader = leaderUrl.equals(ourUrl); // TODO: We can probably delete most of this code if we say this
boolean isLeader = ourUrl.equals(leaderUrl); // TODO: We can probably delete most of this code if we say this
// strategy can only be used for pull replicas
if (isLeader && !cloudDesc.isLeader()) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Cloud state still says we are leader.");

View File

@ -165,7 +165,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
zkStateReader.getClusterState().getCollection(collection).getSlice(shardId).getReplicas().size() < 2) {
Replica leader = zkStateReader.getLeader(collection, shardId);
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))) {
isAlreadyLeader = true;
}
@ -174,7 +174,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
ZkNodeProps m = ZkNodeProps.fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
ZkStateReader.SHARD_ID_PROP, shardId,
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.STATE_PROP, Replica.State.ACTIVE.toString());
assert zkController != null;

View File

@ -19,7 +19,6 @@ package org.apache.solr.cloud;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.lang.invoke.MethodHandles;
import java.net.InetAddress;
import java.net.NetworkInterface;
@ -103,7 +102,6 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
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.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
@ -924,6 +922,8 @@ public class ZkController implements Closeable {
try {
createClusterZkNodes(zkClient);
zkStateReader.createClusterStateWatchersAndUpdate();
// this must happen after zkStateReader has initialized the cluster props
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
checkForExistingEphemeralNode();
@ -1103,6 +1103,7 @@ public class ZkController implements Closeable {
if (zkRunOnly) {
return;
}
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
log.info("Register node as live in ZooKeeper:{}", nodePath);
@ -1401,8 +1402,7 @@ public class ZkController implements Closeable {
byte[] data = zkClient.getData(
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
true);
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(
ZkNodeProps.load(data));
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
return leaderProps;
} catch (InterruptedException e) {
throw e;
@ -1442,7 +1442,6 @@ public class ZkController implements Closeable {
Map<String, Object> props = new HashMap<>();
// 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.NODE_NAME_PROP, getNodeName());
props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
@ -1541,7 +1540,6 @@ public class ZkController implements Closeable {
Map<String,Object> props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, "state");
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.ROLES_PROP, cd.getCloudDescriptor().getRoles());
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
@ -1646,7 +1644,6 @@ public class ZkController implements Closeable {
OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.NODE_NAME_PROP, getNodeName(),
ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
ZkStateReader.BASE_URL_PROP, getBaseUrl(),
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
overseerJobQueue.offer(Utils.toJSON(m));
}
@ -2131,12 +2128,7 @@ public class ZkController implements Closeable {
static String generateNodeName(final String hostName,
final String hostPort,
final String hostContext) {
try {
return hostName + ':' + hostPort + '_' +
URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), "UTF-8");
} catch (UnsupportedEncodingException e) {
throw new Error("JVM Does not seem to support UTF-8", e);
}
return hostName + ':' + hostPort + '_' + URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), StandardCharsets.UTF_8);
}
/**
@ -2197,7 +2189,6 @@ public class ZkController implements Closeable {
String coreNodeName = params.get(CORE_NODE_NAME_PROP);
String coreName = params.get(CORE_NAME_PROP);
String electionNode = params.get(ELECTION_NODE_PROP);
String baseUrl = params.get(BASE_URL_PROP);
try {
MDCLoggingContext.setCoreDescriptor(cc, cc.getCoreDescriptor(coreName));
@ -2209,7 +2200,8 @@ public class ZkController implements Closeable {
ElectionContext prevContext = electionContexts.get(contextKey);
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();
ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
@ -2225,7 +2217,6 @@ public class ZkController implements Closeable {
Replica.Type replicaType = core.getCoreDescriptor().getCloudDescriptor().getReplicaType();
if (replicaType == Type.TLOG) {
String leaderUrl = getLeader(core.getCoreDescriptor().getCloudDescriptor(), cloudConfig.getLeaderVoteWait());
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
if (!leaderUrl.equals(ourUrl)) {
// 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)

View File

@ -210,7 +210,6 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
ZkStateReader.CORE_NAME_PROP, createReplica.coreName,
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(createReplica.node),
ZkStateReader.NODE_NAME_PROP, createReplica.node,
ZkStateReader.REPLICA_TYPE, createReplica.replicaType.name());
if (createReplica.coreNodeName != null) {

View File

@ -208,7 +208,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
ZkStateReader.SHARD_ID_PROP, replicaPosition.shard,
ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.BASE_URL_PROP, baseUrl,
ZkStateReader.NODE_NAME_PROP, nodeName,
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));

View File

@ -62,6 +62,7 @@ import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
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.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
@ -91,11 +92,11 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
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.CORE_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.NODE_NAME_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.REJOIN_AT_HEAD_PROP;
@ -304,7 +305,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
private void processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
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();
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_NODE_NAME_PROP, message.getStr(CORE_NODE_NAME_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();
sreq.nodeName = message.getStr(ZkStateReader.CORE_NAME_PROP);
// yes, they must use same admin handler path everywhere...
@ -426,10 +427,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkNodeProps m = new ZkNodeProps(
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
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.CORE_NODE_NAME_PROP, replicaName,
ZkStateReader.BASE_URL_PROP, replica.getStr(ZkStateReader.BASE_URL_PROP));
ZkStateReader.CORE_NODE_NAME_PROP, replicaName);
overseer.offerStateUpdate(Utils.toJSON(m));
}

View File

@ -457,7 +457,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.CORE_NAME_PROP, solrCoreName,
ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
ZkStateReader.NODE_NAME_PROP, subShardNodeName,
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));

View File

@ -77,7 +77,6 @@ public class SliceMutator {
Replica replica = new Replica(coreNodeName,
makeMap(
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.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice);
@ -87,7 +86,6 @@ public class SliceMutator {
public ZkWriteCommand removeReplica(ClusterState clusterState, ZkNodeProps message) {
final String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_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;
DocCollection coll = clusterState.getCollectionOrNull(collection);
@ -97,10 +95,10 @@ public class SliceMutator {
}
Map<String, Slice> newSlices = new LinkedHashMap<>();
final String nodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
for (Slice slice : coll.getSlices()) {
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();
newReplicas.remove(cnn);
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(),collection);
@ -112,15 +110,7 @@ public class SliceMutator {
}
public ZkWriteCommand setShardLeader(ClusterState clusterState, ZkNodeProps message) {
StringBuilder sb = new StringBuilder();
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 leaderUrl = ZkCoreNodeProps.getCoreUrl(message);
String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
DocCollection coll = clusterState.getCollectionOrNull(collectionName);
@ -137,7 +127,7 @@ public class SliceMutator {
final Map<String, Replica> newReplicas = new LinkedHashMap<>();
for (Replica replica : slice.getReplicas()) {
// 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)) {
replica = new ReplicaMutator(cloudManager).unsetLeader(replica);

View File

@ -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.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.
@ -205,7 +204,7 @@ public class BlobRepository {
*/
ByteBuffer fetchBlob(String key) {
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);
}

View File

@ -36,16 +36,22 @@ import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterProperties;
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.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
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}.
* 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);
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
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) {
// we are part of an ensemble and we are not uploading the config - pause to give the config time
// to get up

View File

@ -34,6 +34,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
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.ZkStateReader;
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");
}
if (requestedShards == null || requestedShards.isEmpty()) {
return collection;
return postProcessCollectionJSON(collection);
} else {
Map<String, Object> shards = (Map<String, Object>) collection.get("shards");
Map<String, Object> selected = new HashMap<>();
@ -207,12 +208,10 @@ public class ClusterStatus {
selected.put(selectedShard, shards.get(selectedShard));
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
* 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;
}
}

View File

@ -109,6 +109,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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;
/**
@ -190,7 +191,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
.getOrDefault("history", Collections.emptyMap());
args.putAll(props);
overseerUrlScheme = cloudManager.getClusterStateProvider().getClusterProperty("urlScheme", "http");
overseerUrlScheme = cloudManager.getClusterStateProvider().getClusterProperty(URL_SCHEME, "http");
} else {
overseerUrlScheme = "http";
}

View File

@ -409,7 +409,7 @@ class RebalanceLeaders {
propMap.put(QUEUE_OPERATION, REBALANCELEADERS.toLower());
propMap.put(CORE_NAME_PROP, core);
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(ELECTION_NODE_PROP, electionNode);
String asyncId = REBALANCELEADERS.toLower() + "_" + core + "_" + Math.abs(System.nanoTime());

View File

@ -59,7 +59,6 @@ class RequestSyncShardOp implements CoreAdminHandler.CoreAdminOp {
syncStrategy = new SyncStrategy(core.getCoreContainer());
Map<String, Object> props = new HashMap<>();
props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
props.put(ZkStateReader.CORE_NAME_PROP, cname);
props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());

View File

@ -537,16 +537,18 @@ public final class ZookeeperInfoHandler extends RequestHandlerBase {
if (childDataStr != null) {
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) {
// verify this collection matches the filtered state
if (page.matchesStatusFilter((Map<String, Object>) collectionState, liveNodes)) {
if (page.matchesStatusFilter(collectionState, liveNodes)) {
matchesStatusFilter.add(collection);
collectionStates.put(collection, collectionState);
collectionStates.put(collection, ClusterStatus.postProcessCollectionJSON(collectionState));
}
} else {
collectionStates.put(collection, collectionState);
collectionStates.put(collection, ClusterStatus.postProcessCollectionJSON(collectionState));
}
}
}

View File

@ -116,7 +116,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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.CORE_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
continue;
}
if (replica.getStr(BASE_URL_PROP).equals(cores.getZkController().getBaseUrl())) {
if (replica.getBaseUrl().equals(cores.getZkController().getBaseUrl())) {
// don't count a local core
continue;
}
if (origCorename != null) {
coreUrl = replica.getStr(BASE_URL_PROP) + "/" + origCorename;
coreUrl = replica.getBaseUrl() + "/" + origCorename;
} else {
coreUrl = replica.getCoreUrl();
if (coreUrl.endsWith("/")) {

View File

@ -104,6 +104,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
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.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
@ -2717,6 +2718,7 @@ public class SolrCLI implements CLIO {
protected void runImpl(CommandLine cli) throws Exception {
this.urlScheme = cli.getOptionValue("urlScheme", "http");
UrlScheme.INSTANCE.setUrlScheme(this.urlScheme);
serverDir = new File(cli.getOptionValue("serverDir"));
if (!serverDir.isDirectory())

View File

@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
*/
@Slow
@SolrTestCaseJ4.SuppressPointFields(bugUrl="Test explicitly compares Trie to Points, randomization defeats the point")
@SolrTestCaseJ4.SuppressSSL
public class TestRandomDVFaceting extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

View File

@ -29,6 +29,7 @@ import org.junit.Test;
/**
* Tests for {@link ClusterStateMockUtil}
*/
@SolrTestCaseJ4.SuppressSSL // tests expect http scheme
public class ClusterStateMockUtilTest extends SolrTestCaseJ4 {
@Test

View File

@ -241,8 +241,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.CORE_NODE_NAME_PROP, replica.getName(),
ZkStateReader.BASE_URL_PROP, replica.getBaseUrl());
ZkStateReader.CORE_NODE_NAME_PROP, replica.getName());
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
@ -307,8 +306,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
ZkStateReader.CORE_NAME_PROP, replica1.getCoreName(),
ZkStateReader.NODE_NAME_PROP, replica1.getNodeName(),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.CORE_NODE_NAME_PROP, replica1.getName(),
ZkStateReader.BASE_URL_PROP, replica1.getBaseUrl());
ZkStateReader.CORE_NODE_NAME_PROP, replica1.getName());
cluster.getOpenOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
boolean replicaDeleted = false;

View File

@ -32,6 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.OnReconnect;
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.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -205,21 +206,22 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
public void testBasic() throws Exception {
LeaderElector elector = new LeaderElector(zkClient);
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);
ElectionContext context = new ShardLeaderElectionContextBase(elector,
"shard2", "collection1", "dummynode1", props, zkController);
elector.setup(context);
elector.joinElection(context, false);
assertEquals("http://127.0.0.1/solr/",
assertEquals(UrlScheme.INSTANCE.getUrlScheme() + "://127.0.0.1/solr/",
getLeaderUrl("collection1", "shard2"));
}
@Test
public void testCancelElection() throws Exception {
UrlScheme u = UrlScheme.INSTANCE;
LeaderElector first = new LeaderElector(zkClient);
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);
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
"slice1", "collection2", "dummynode1", props, zkController);
@ -227,21 +229,25 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
first.joinElection(firstContext, false);
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);
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);
ElectionContext context = new ShardLeaderElectionContextBase(second,
"slice1", "collection2", "dummynode2", props, zkController);
second.setup(context);
second.joinElection(context, false);
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();
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)
@ -384,7 +390,12 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
for (int i = 1; i <= numShards; i ++) {
// if this test fails, getLeaderUrl will more likely throw an exception and fail the test,
// 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 {
// cleanup any threads still running
@ -414,6 +425,11 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
private int getLeaderThread() throws KeeperException, InterruptedException {
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("/", ""));
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud;
import java.io.IOException;
import java.util.List;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.SolrTestCaseJ4Test;
import org.apache.solr.cloud.overseer.NodeMutator;
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.junit.Test;
@SolrTestCaseJ4.SuppressSSL // tests compare for http:
public class NodeMutatorTest extends SolrTestCaseJ4Test {
private static final String NODE3 = "baseUrl3_";

View File

@ -102,6 +102,7 @@ import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer;
@Slow
@SolrTestCaseJ4.SuppressSSL
public class OverseerTest extends SolrTestCaseJ4 {
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.COLLECTION_PROP, collection,
ZkStateReader.SHARD_ID_PROP, shard,
ZkStateReader.NUM_SHARDS_PROP, Integer.toString(numShards),
ZkStateReader.BASE_URL_PROP, "http://" + nodeName + "/solr/");
ZkStateReader.NUM_SHARDS_PROP, Integer.toString(numShards));
ZkDistributedQueue q = overseer.getStateUpdateQueue();
q.offer(Utils.toJSON(m));
}
@ -233,15 +233,14 @@ public class OverseerTest extends SolrTestCaseJ4 {
zkClient.makePath("/collections/" + collection + "/leader_elect/"
+ shardId + "/election", true);
} catch (NodeExistsException nee) {}
ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
"http://" + nodeName + "/solr/", ZkStateReader.NODE_NAME_PROP,
nodeName, ZkStateReader.CORE_NAME_PROP, coreName,
ZkNodeProps props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, nodeName,
ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.SHARD_ID_PROP, shardId,
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
LeaderElector elector = new LeaderElector(zkClient);
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
elector, shardId, collection, nodeName + "_" + coreName, props,
elector, shardId, collection, nodeName + coreName, props,
MockSolrSource.makeSimpleMock(overseer, zkStateReader, null));
elector.setup(ctx);
electionContext.put(coreName, ctx);
@ -405,7 +404,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
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
@ -449,7 +448,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
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;
mockController.createCollection(COLLECTION, 3);
@ -516,7 +515,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
try (ZkStateReader reader = new ZkStateReader(zkClient)) {
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)) {
@ -527,7 +526,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
}
}
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);
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
@ -591,8 +590,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
createCollection(COLLECTION, 1);
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.SHARD_ID_PROP, "shard1",
ZkStateReader.CORE_NAME_PROP, "core1",
@ -607,8 +605,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
//publish node state (active)
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.SHARD_ID_PROP, "shard1",
ZkStateReader.CORE_NAME_PROP, "core1",
@ -664,7 +661,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
reader = new ZkStateReader(zkClient);
reader.createClusterStateWatchersAndUpdate();
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
overseerClient = electNewOverseer(server.getZkAddress());
@ -726,7 +723,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
reader = new ZkStateReader(zkClient);
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);
if (overseers.size() > 0) {
@ -905,7 +902,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
for (int i = 0; i < atLeast(4); i++) {
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);
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);
while (!timeout.hasTimedOut()) {
@ -1037,7 +1034,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
reader = new ZkStateReader(zkClient);
reader.createClusterStateWatchersAndUpdate();
mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
mockController = new MockZKController(server.getZkAddress(), "127.0.0.1_solr", overseers);
overseerClient = electNewOverseer(server.getZkAddress());
@ -1053,7 +1050,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
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));
@ -1095,7 +1092,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
reader = new ZkStateReader(zkClient);
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;
@ -1118,8 +1115,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ZkStateReader.SHARD_ID_PROP, "shard1",
ZkStateReader.CORE_NODE_NAME_PROP, "node1",
ZkStateReader.COLLECTION_PROP, "perf" + j,
ZkStateReader.NUM_SHARDS_PROP, "1",
ZkStateReader.BASE_URL_PROP, "http://" + "node1" + "/solr/");
ZkStateReader.NUM_SHARDS_PROP, "1");
ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
q.offer(Utils.toJSON(m));
if (j >= MAX_COLLECTIONS - 1) j = 0;
@ -1212,8 +1208,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
"createNodeSet", "");
queue.offer(Utils.toJSON(m));
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, "127.0.0.1_solr",
ZkStateReader.SHARD_ID_PROP, "shard1",
ZkStateReader.COLLECTION_PROP, COLLECTION,
ZkStateReader.CORE_NAME_PROP, "core1",
@ -1221,8 +1216,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
queue.offer(Utils.toJSON(m));
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.COLLECTION_PROP, COLLECTION,
ZkStateReader.CORE_NAME_PROP, "core2",
@ -1235,8 +1229,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
//submit to proper queue
queue = overseers.get(0).getStateUpdateQueue();
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, "127.0.0.1_solr",
ZkStateReader.SHARD_ID_PROP, "shard1",
ZkStateReader.COLLECTION_PROP, COLLECTION,
ZkStateReader.CORE_NAME_PROP, "core3",
@ -1276,9 +1269,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
createCollection("c1", 1);
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.NODE_NAME_PROP, "node1",
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
ZkStateReader.COLLECTION_PROP, "c1",
ZkStateReader.CORE_NAME_PROP, "core1",
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);
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.NODE_NAME_PROP, "node1",
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
ZkStateReader.COLLECTION_PROP, "c1",
ZkStateReader.CORE_NAME_PROP, "core1",
ZkStateReader.ROLES_PROP, "",
@ -1303,9 +1294,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
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.NODE_NAME_PROP, "node1",
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
ZkStateReader.COLLECTION_PROP, "c1",
ZkStateReader.CORE_NAME_PROP, "core1",
ZkStateReader.ROLES_PROP, "",
@ -1340,9 +1330,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.ADDREPLICA.toLower(),
"collection", testCollectionName,
ZkStateReader.SHARD_ID_PROP, "x",
ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
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.STATE_PROP, Replica.State.DOWN.toString()
);
@ -1491,9 +1480,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
for (int ss = 1; ss <= numShards; ++ss) {
final int N = (numReplicas-rr)*numShards + ss;
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.NODE_NAME_PROP, "node"+N,
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
ZkStateReader.COLLECTION_PROP, COLLECTION,
ZkStateReader.CORE_NAME_PROP, "core"+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) {
final int N = (numReplicas-rr)*numShards + ss;
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.NODE_NAME_PROP, "node"+N,
ZkStateReader.NODE_NAME_PROP, "127.0.0.1_solr",
ZkStateReader.COLLECTION_PROP, COLLECTION,
ZkStateReader.CORE_NAME_PROP, "core"+N,
ZkStateReader.ROLES_PROP, "",

View File

@ -35,8 +35,6 @@ import org.apache.solr.common.cloud.Slice;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
/**
* See SOLR-9504
*/
@ -73,7 +71,7 @@ public class TestLeaderElectionWithEmptyReplica extends SolrCloudTestCase {
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
for (JettySolrRunner jettySolrRunner : jettySolrRunners) {
int port = jettySolrRunner.getBaseUrl().getPort();
if (replica.getStr(BASE_URL_PROP).contains(":" + port)) {
if (replica.getBaseUrl().contains(":" + port)) {
replicaJetty = jettySolrRunner;
break;
}

View File

@ -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.request.CollectionAdminRequest;
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.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.util.SSLTestConfig;
@ -104,6 +105,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
System.setProperty(ZkStateReader.URL_SCHEME, "http");
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
checkClusterWithNodeReplacement(sslConfig);
}
@ -115,6 +117,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
HttpClientUtil.setSocketFactoryRegistryProvider(sslConfig.buildClientSocketFactoryRegistryProvider());
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
System.setProperty(ZkStateReader.URL_SCHEME, "http");
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
checkClusterWithNodeReplacement(sslConfig);
}

View File

@ -112,7 +112,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
DocCollection b1x1 = clusterState.getCollection("b1x1");
Collection<Replica> replicas = b1x1.getSlice("shard1").getReplicas();
assertEquals(1, replicas.size());
String baseUrl = replicas.iterator().next().getStr(ZkStateReader.BASE_URL_PROP);
String baseUrl = replicas.iterator().next().getBaseUrl();
if (!baseUrl.endsWith("/")) baseUrl += "/";
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2", 2000, 5000)) {
@ -171,7 +171,6 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
//Simulate a replica being in down state.
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.COLLECTION_PROP, "football",
ZkStateReader.SHARD_ID_PROP, "shard1",
@ -189,7 +188,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
//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 += "/";
String path = baseUrl + "football";
log.info("Firing queries against path={}", path);

View File

@ -602,8 +602,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
Replica newReplica = grabNewReplica(response, getCollectionState(collectionName));
assertEquals("Replica should be created on the right node",
cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)),
newReplica.getStr(ZkStateReader.BASE_URL_PROP));
cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)), newReplica.getBaseUrl());
Path instancePath = createTempDir();
response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
@ -612,7 +611,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
newReplica = grabNewReplica(response, getCollectionState(collectionName));
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);
NamedList<Object> coreStatus = status.getCoreStatus(newReplica.getStr("core"));
String instanceDirStr = (String) coreStatus.get("instanceDir");

View File

@ -74,7 +74,6 @@ import org.junit.Test;
import org.slf4j.Logger;
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;
@Slow
@ -187,7 +186,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
boolean restarted = false;
for (JettySolrRunner jetty : jettys) {
int port = jetty.getBaseUrl().getPort();
if (replica.getStr(BASE_URL_PROP).contains(":" + port)) {
if (replica.getBaseUrl().contains(":" + port)) {
stoppedNodeName = jetty.getNodeName();
jetty.stop();
jetty.start();
@ -206,7 +205,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(collectionName, SHARD1_0);
// use control client because less chances of it being the node being restarted
// 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()) {
state = addReplica.processAndWait(control, 30);
}

View File

@ -16,8 +16,6 @@
*/
package org.apache.solr.core.snapshots;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
@ -27,8 +25,8 @@ import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrClient;
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.State;
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.core.snapshots.CollectionSnapshotMetaData.CoreSnapshotMetaData;
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.
}
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String replicaBaseUrl = replica.getBaseUrl();
String coreName = replica.getCoreName();
assertTrue(snapshotByCoreName.containsKey(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.
}
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String replicaBaseUrl = replica.getBaseUrl();
String coreName = replica.getCoreName();
try (SolrClient adminClient = getHttpSolrClient(replicaBaseUrl)) {
Collection<SnapshotMetaData> snapshots = listCoreSnapshots(adminClient, coreName);

View File

@ -56,8 +56,6 @@ import org.junit.Test;
import org.slf4j.Logger;
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
@Slow
public class TestSolrCoreSnapshots extends SolrCloudTestCase {
@ -97,8 +95,8 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
assertEquals(1, shard.getReplicas().size());
Replica replica = shard.getReplicas().iterator().next();
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String replicaBaseUrl = replica.getBaseUrl();
String coreName = replica.getCoreName();
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
String commitName = TestUtil.randomSimpleString(random(), 1, 5);
String duplicateName = commitName.concat("_duplicate");

View File

@ -39,7 +39,6 @@ import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.DocCollection;
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.util.RTimer;
import org.apache.solr.util.SimplePostTool;
@ -66,7 +65,7 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase {
DocCollection sysColl = cloudClient.getZkStateReader().getClusterState().getCollection(".system");
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";
MapWriter map = TestSolrConfigHandlerConcurrent.getAsMap(url, cloudClient);
assertNotNull(map);

View File

@ -96,7 +96,7 @@ public class TestConfigReload extends AbstractFullDistribZkTestBase {
List<String> urls = new ArrayList<>();
for (Slice slice : coll.getSlices()) {
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<>();

View File

@ -46,7 +46,6 @@ import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
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.CoreAdminAction;
import org.apache.solr.util.BadHdfsThreadsFilter;
@ -56,8 +55,6 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
@ThreadLeakFilters(defaultFilters = true, filters = {
SolrIgnoredThreadsFilter.class,
QuickPatchThreadsFilter.class,
@ -179,8 +176,8 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
assertEquals(1, shard.getReplicas().size());
Replica replica = shard.getReplicas().iterator().next();
String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String replicaBaseUrl = replica.getBaseUrl();
String coreName = replica.getCoreName();
String backupName = TestUtil.randomSimpleString(random(), 1, 5);
boolean testViaReplicationHandler = random().nextBoolean();

View File

@ -85,7 +85,7 @@ public class TestReqParamsAPI extends SolrCloudTestCase {
List<String> urls = new ArrayList<>();
for (Slice slice : coll.getSlices()) {
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()));

View File

@ -85,7 +85,7 @@ public class TestSolrConfigHandlerCloud extends AbstractFullDistribZkTestBase {
List<String> urls = new ArrayList<>();
for (Slice slice : coll.getSlices()) {
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()));
}
@ -95,7 +95,7 @@ public class TestSolrConfigHandlerCloud extends AbstractFullDistribZkTestBase {
List<String> urls = new ArrayList<>();
for (Slice slice : coll.getSlices()) {
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();

View File

@ -146,7 +146,7 @@ public class TestSolrConfigHandlerConcurrent extends AbstractFullDistribZkTestBa
List<String> urls = new ArrayList<>();
for (Slice slice : coll.getSlices()) {
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));
}

View File

@ -31,6 +31,7 @@ import org.mockito.Mockito;
/**
* Tests for {@link CloudReplicaSource}
*/
@SolrTestCaseJ4.SuppressSSL // lots of assumptions about http: in this test
public class CloudReplicaSourceTest extends SolrTestCaseJ4 {
@BeforeClass

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.BaseTestCheckIndex;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.QuickPatchThreadsFilter;
import org.apache.solr.SolrIgnoredThreadsFilter;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
@ -47,6 +48,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
QuickPatchThreadsFilter.class,
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
public class CheckHdfsIndexTest extends AbstractFullDistribZkTestBase {
private static MiniDFSCluster dfsCluster;

View File

@ -375,11 +375,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
}
ArrayList<Node> nodes = new ArrayList<>();
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
ZkStateReader.CORE_NAME_PROP, "");
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) {
@Override
public boolean checkRetry(Error err) {
@ -439,11 +436,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
ArrayList<Node> nodes = new ArrayList<>();
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
ZkStateReader.CORE_NAME_PROP, "");
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) {
nodes.add(new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
@Override
@ -594,11 +588,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
streamingClients.setExp(Exp.CONNECT_EXCEPTION);
ArrayList<Node> nodes = new ArrayList<>();
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
ZkStateReader.CORE_NAME_PROP, "");
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;
if (nodeType == NodeType.FORWARD) {
retryNode = new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
@ -711,11 +702,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
ArrayList<Node> nodes = new ArrayList<>();
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
ZkStateReader.CORE_NAME_PROP, "");
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) {
@Override
public boolean checkRetry(Error err) {
@ -758,11 +746,8 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
streamingClients.setExp(Exp.SOCKET_EXCEPTION);
ArrayList<Node> nodes = new ArrayList<>();
ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
ZkStateReader.CORE_NAME_PROP, "");
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) {
@Override
public boolean checkRetry(Error err) {

View File

@ -71,6 +71,7 @@ import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.Replica;
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.ZkStateReader;
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.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
@ -1093,14 +1093,12 @@ public abstract class BaseCloudSolrClient extends SolrClient {
if (!liveNodes.isEmpty()) {
List<String> liveNodesList = new ArrayList<>(liveNodes);
Collections.shuffle(liveNodesList, rand);
theUrlList.add(Utils.getBaseUrlForNodeName(liveNodesList.get(0),
getClusterStateProvider().getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
theUrlList.add(UrlScheme.INSTANCE.getBaseUrlForNodeName(liveNodesList.get(0)));
}
} else if (ADMIN_PATHS.contains(request.getPath())) {
for (String liveNode : liveNodes) {
theUrlList.add(Utils.getBaseUrlForNodeName(liveNode,
getClusterStateProvider().getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
theUrlList.add(UrlScheme.INSTANCE.getBaseUrlForNodeName(liveNode));
}
} else { // Typical...

View File

@ -141,7 +141,7 @@ public class ClusterStateUtil {
// on a live node?
boolean live = clusterState.liveNodesContain(replica.getNodeName());
String rcoreNodeName = replica.getName();
String rbaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
String rbaseUrl = replica.getBaseUrl();
if (live && coreNodeName.equals(rcoreNodeName)
&& baseUrl.equals(rbaseUrl)) {
// found it

View File

@ -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.dedupeKeyPredicate;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
public class Replica extends ZkNodeProps implements MapWriter {
@ -139,6 +140,8 @@ public class Replica extends ZkNodeProps implements MapWriter {
// default to ACTIVE
this.state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, State.ACTIVE.toString())));
validate();
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
}
// clone constructor
@ -156,6 +159,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
this.propMap.putAll(props);
}
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")));
this.propMap.putAll(details);
validate();
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
}
private final void validate() {
@ -198,8 +202,6 @@ public class Replica extends ZkNodeProps implements MapWriter {
propMap.put(ZkStateReader.STATE_PROP, state.toString());
}
public String getCollection() {
return collection;
}
@ -235,11 +237,11 @@ public class Replica extends ZkNodeProps implements MapWriter {
}
public String getCoreUrl() {
return ZkCoreNodeProps.getCoreUrl(getStr(ZkStateReader.BASE_URL_PROP), core);
return ZkCoreNodeProps.getCoreUrl(getBaseUrl(), core);
}
public String getBaseUrl() {
return getStr(ZkStateReader.BASE_URL_PROP);
return getStr(BASE_URL_PROP);
}
/** SolrCore name. */
@ -295,8 +297,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
}
public Object clone() {
return new Replica(name, node, collection, shard, core, state, type,
propMap);
return new Replica(name, node, collection, shard, core, state, type, propMap);
}
@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
// contents, so a third party may override some declared fields
for (Map.Entry<String, Object> e : propMap.entrySet()) {
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)
.put(ZkStateReader.SHARD_ID_PROP, shard, p)
.put(ZkStateReader.COLLECTION_PROP, collection, p)

View File

@ -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;
}
}

View File

@ -16,15 +16,17 @@
*/
package org.apache.solr.common.cloud;
import java.util.Objects;
public class ZkCoreNodeProps {
private ZkNodeProps nodeProps;
private final ZkNodeProps nodeProps;
public ZkCoreNodeProps(ZkNodeProps nodeProps) {
this.nodeProps = nodeProps;
}
public String getCoreUrl() {
return getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), nodeProps.getStr(ZkStateReader.CORE_NAME_PROP));
return getCoreUrl(this.nodeProps);
}
public String getNodeName() {
@ -36,22 +38,35 @@ public class ZkCoreNodeProps {
}
public String getBaseUrl() {
return nodeProps.getStr(ZkStateReader.BASE_URL_PROP);
return getBaseUrl(this.nodeProps);
}
public String getCoreName() {
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) {
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) {
Objects.requireNonNull(baseUrl,"baseUrl must not be null");
StringBuilder sb = new StringBuilder();
sb.append(baseUrl);
if (!baseUrl.endsWith("/")) sb.append("/");
sb.append(coreName);
sb.append(coreName != null ? coreName : "");
if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
return sb.toString();
}
@ -68,6 +83,4 @@ public class ZkCoreNodeProps {
public boolean isLeader() {
return nodeProps.containsKey(ZkStateReader.LEADER_PROP);
}
}

View File

@ -17,7 +17,12 @@
package org.apache.solr.common.cloud;
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.Utils;
@ -37,6 +42,13 @@ public class ZkNodeProps implements JSONWriter.Writable {
*/
public ZkNodeProps(Map<String,Object> 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.
// Always wrapping introduces a memory leak.
}
@ -90,7 +102,7 @@ public class ZkNodeProps implements JSONWriter.Writable {
*/
@SuppressWarnings({"unchecked"})
public static ZkNodeProps load(byte[] bytes) {
Map<String, Object> props = null;
Map<String, Object> props;
if (bytes[0] == 2) {
try (JavaBinCodec jbc = new JavaBinCodec()) {
props = (Map<String, Object>) jbc.unmarshal(bytes);
@ -105,15 +117,26 @@ public class ZkNodeProps implements JSONWriter.Writable {
@Override
public void write(JSONWriter jsonWriter) {
// 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.
*/
public String getStr(String key) {
Object o = propMap.get(key);
return o == null ? null : o.toString();
return getStr(key, null);
}
/**
@ -127,8 +150,17 @@ public class ZkNodeProps implements JSONWriter.Writable {
/**
* Get a string property value.
*/
public String getStr(String key,String def) {
public String getStr(String key, String def) {
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();
}
@ -139,14 +171,6 @@ public class ZkNodeProps implements JSONWriter.Writable {
@Override
public String toString() {
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();
***/
}
/**

View File

@ -51,7 +51,11 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.params.CollectionAdminParams;
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.NoNodeException;
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.emptySortedSet;
import static org.apache.solr.common.cloud.UrlScheme.HTTP;
import static org.apache.solr.common.util.Utils.fromJSON;
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 {
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();
}
@ -1003,6 +1012,9 @@ public class ZkStateReader implements SolrCloseable {
this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
log.debug("Loaded cluster properties: {}", this.clusterProperties);
// Make the urlScheme globally accessible
UrlScheme.INSTANCE.setUrlScheme(getClusterProperty(ZkStateReader.URL_SCHEME, HTTP));
for (ClusterPropertiesListener listener : clusterPropertiesListeners) {
listener.onChange(getClusterProperties());
}

View File

@ -61,7 +61,6 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
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.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
@ -80,7 +79,7 @@ import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
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();
for (Slice slice : col.getSlices()) {
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"));
}
}
@ -344,7 +343,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
Set<String> expectedBaseURLs = Sets.newHashSet();
for (Slice expectedSlice : expectedSlices) {
for (Replica replica : expectedSlice.getReplicas()) {
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
String baseURL = replica.getBaseUrl();
expectedBaseURLs.add(baseURL);
}
}
@ -391,7 +390,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
for (Slice slice : col.getSlices()) {
for (Replica replica : slice.getReplicas()) {
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
String baseURL = replica.getBaseUrl();
Long prevNumRequests = requestCountsMap.get(baseURL);
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
@ -700,7 +699,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
SolrQuery q = new SolrQuery().setQuery("*:*");
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)) {
if (log.isInfoEnabled()) {
@ -727,7 +726,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
Set<String> allNodesOfColl = new HashSet<>();
for (Slice slice : coll.getSlices()) {
for (Replica replica : slice.getReplicas()) {
allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP));
allNodesOfColl.add(replica.getBaseUrl());
}
}
String theNode = null;

View File

@ -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.response.QueryResponse;
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.UpdateResponse;
import org.apache.solr.cloud.AbstractDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
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.Replica;
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.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
@ -319,7 +318,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
Map<String, Long> requestCountsMap = Maps.newHashMap();
for (Slice slice : col.getSlices()) {
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"));
}
}
@ -330,8 +329,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
Set<String> expectedBaseURLs = Sets.newHashSet();
for (Slice expectedSlice : expectedSlices) {
for (Replica replica : expectedSlice.getReplicas()) {
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
expectedBaseURLs.add(baseURL);
expectedBaseURLs.add(replica.getBaseUrl());
}
}
@ -377,7 +375,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
Map<String, Long> numRequestsToUnexpectedUrls = Maps.newHashMap();
for (Slice slice : col.getSlices()) {
for (Replica replica : slice.getReplicas()) {
String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP);
String baseURL = replica.getBaseUrl();
Long prevNumRequests = requestCountsMap.get(baseURL);
Long curNumRequests = getNumRequests(baseURL, "routing_collection");
@ -684,7 +682,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
SolrQuery q = new SolrQuery().setQuery("*:*");
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)) {
if (log.isInfoEnabled()) {
@ -711,7 +709,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
Set<String> allNodesOfColl = new HashSet<>();
for (Slice slice : coll.getSlices()) {
for (Replica replica : slice.getReplicas()) {
allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP));
allNodesOfColl.add(replica.getBaseUrl());
}
}
String theNode = null;

View File

@ -21,11 +21,13 @@ import java.util.ArrayList;
import java.util.List;
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.ZkStateReader;
import org.apache.solr.common.params.ShardParams;
import org.junit.Test;
@SolrTestCaseJ4.SuppressSSL // this test is all about http://
public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
@Test
@ -33,11 +35,11 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
List<Replica> replicas = getBasicReplicaList();
// 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);
replicas.sort(comparator);
assertEquals("node2", replicas.get(0).getNodeName());
assertEquals("node1", replicas.get(1).getNodeName());
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
}
@ -49,8 +51,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
replicas.sort(comparator);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node2", replicas.get(1).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
// reversed rule
rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
@ -58,8 +60,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
comparator = new NodePreferenceRulesComparator(rules, null);
replicas.sort(comparator);
assertEquals("node2", replicas.get(0).getNodeName());
assertEquals("node1", replicas.get(1).getNodeName());
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
}
@SuppressWarnings("unchecked")
@ -71,8 +73,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
new Replica(
"node4",
map(
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node4",
ZkStateReader.NODE_NAME_PROP, "node4:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
),"collection1","shard1"
@ -82,14 +83,14 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
List<PreferenceRule> rules = PreferenceRule.from(
ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":NRT," +
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);
replicas.sort(comparator);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node4", replicas.get(1).getNodeName());
assertEquals("node2", replicas.get(2).getNodeName());
assertEquals("node3", replicas.get(3).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node4", getHost(replicas.get(1).getNodeName()));
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
assertEquals("node3", getHost(replicas.get(3).getNodeName()));
}
@Test(expected = IllegalArgumentException.class)
@ -116,13 +117,14 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
@SuppressWarnings("unchecked")
private static List<Replica> getBasicReplicaList() {
UrlScheme.INSTANCE.setUrlScheme(UrlScheme.HTTP);
List<Replica> replicas = new ArrayList<Replica>();
replicas.add(
new Replica(
"node1",
map(
ZkStateReader.BASE_URL_PROP, "http://host1:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node1",
ZkStateReader.NODE_NAME_PROP, "node1:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "NRT"
),"collection1","shard1"
@ -132,8 +134,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
new Replica(
"node2",
map(
ZkStateReader.BASE_URL_PROP, "http://host2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node2",
ZkStateReader.NODE_NAME_PROP, "node2:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
),"collection1","shard1"
@ -143,8 +144,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
new Replica(
"node3",
map(
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node3",
ZkStateReader.NODE_NAME_PROP, "node3:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "PULL"
),"collection1","shard1"
@ -152,4 +152,9 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
);
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('_'));
}
}

View File

@ -42,30 +42,35 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node2", replicas.get(1).getNodeName());
assertEquals("node3", replicas.get(2).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
assertEquals("node3", getHost(replicas.get(2).getNodeName()));
params.set("routingPreference", "1");
rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node2", replicas.get(0).getNodeName());
assertEquals("node3", replicas.get(1).getNodeName());
assertEquals("node1", replicas.get(2).getNodeName());
assertEquals("node2", getHost(replicas.get(0).getNodeName()));
assertEquals("node3", getHost(replicas.get(1).getNodeName()));
assertEquals("node1", getHost(replicas.get(2).getNodeName()));
params.set("routingPreference", "2");
rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node3", replicas.get(0).getNodeName());
assertEquals("node1", replicas.get(1).getNodeName());
assertEquals("node2", replicas.get(2).getNodeName());
assertEquals("node3", getHost(replicas.get(0).getNodeName()));
assertEquals("node1", getHost(replicas.get(1).getNodeName()));
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
params.set("routingPreference", "3");
rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node2", replicas.get(1).getNodeName());
assertEquals("node3", replicas.get(2).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node2", getHost(replicas.get(1).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")
@ -80,8 +85,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
new Replica(
"node4",
map(
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node4",
ZkStateReader.NODE_NAME_PROP, "node4:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
), "c1","s1"
@ -93,8 +97,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
new Replica(
"node5",
map(
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node5",
ZkStateReader.NODE_NAME_PROP, "node5:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "PULL"
), "c1","s1"
@ -110,20 +113,20 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
params.add(ShardParams.SHARDS_PREFERENCE, rulesParam);
ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node2", replicas.get(1).getNodeName());
assertEquals("node4", replicas.get(2).getNodeName());
assertEquals("node3", replicas.get(3).getNodeName());
assertEquals("node5", replicas.get(4).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node2", getHost(replicas.get(1).getNodeName()));
assertEquals("node4", getHost(replicas.get(2).getNodeName()));
assertEquals("node3", getHost(replicas.get(3).getNodeName()));
assertEquals("node5", getHost(replicas.get(4).getNodeName()));
params.set("routingPreference", "1");
rlt = generator.getReplicaListTransformer(params);
rlt.transform(replicas);
assertEquals("node1", replicas.get(0).getNodeName());
assertEquals("node4", replicas.get(1).getNodeName());
assertEquals("node2", replicas.get(2).getNodeName());
assertEquals("node5", replicas.get(3).getNodeName());
assertEquals("node3", replicas.get(4).getNodeName());
assertEquals("node1", getHost(replicas.get(0).getNodeName()));
assertEquals("node4", getHost(replicas.get(1).getNodeName()));
assertEquals("node2", getHost(replicas.get(2).getNodeName()));
assertEquals("node5", getHost(replicas.get(3).getNodeName()));
assertEquals("node3", getHost(replicas.get(4).getNodeName()));
}
@SuppressWarnings("unchecked")
@ -133,8 +136,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
new Replica(
"node1",
map(
ZkStateReader.BASE_URL_PROP, "http://host1:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node1",
ZkStateReader.NODE_NAME_PROP, "node1:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "NRT"
),"c1","s1"
@ -144,8 +146,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
new Replica(
"node2",
map(
ZkStateReader.BASE_URL_PROP, "http://host2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node2",
ZkStateReader.NODE_NAME_PROP, "node2:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
),"c1","s1"
@ -155,8 +156,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
new Replica(
"node3",
map(
ZkStateReader.BASE_URL_PROP, "http://host2_2:8983/solr",
ZkStateReader.NODE_NAME_PROP, "node3",
ZkStateReader.NODE_NAME_PROP, "node3:8983_solr",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "PULL"
),"c1","s1"

View File

@ -40,7 +40,7 @@ public class ShufflingReplicaListTransformerTest extends SolrTestCase {
Map<String, Object> propMap = new HashMap<>();
propMap.put("core", "core" + counter);
propMap.put("type", "NRT");
propMap.put("node_name", "node" + counter);
propMap.put("node_name", "node" + counter + "_");
counter++;
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) {
if (random().nextBoolean()) {
urls.add(url);
urls.add(url + "_");
}
}

View File

@ -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"));
}
}

View File

@ -101,6 +101,7 @@ import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
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.ModifiableSolrParams;
import org.apache.solr.common.params.MultiMapSolrParams;
@ -160,6 +161,7 @@ import org.xml.sax.SAXException;
import static java.util.Objects.requireNonNull;
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.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
@ -300,7 +302,10 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
if(isSSLMode()) {
// 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("enable.update.log");
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.zkclienttmeout");
System.clearProperty(ZK_WHITELIST_PROPERTY);

View File

@ -16,6 +16,7 @@
*/
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 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.Replica;
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.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -255,17 +257,17 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
}
if (isSSLMode()) {
System.clearProperty("urlScheme");
System.clearProperty(URL_SCHEME);
try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT)) {
try {
zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
Utils.toJSON(Collections.singletonMap("urlScheme", "https")),
Utils.toJSON(Collections.singletonMap(URL_SCHEME, "https")),
CreateMode.PERSISTENT, true);
} catch (KeeperException.NodeExistsException e) {
ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(ZkStateReader.CLUSTER_PROPS,
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 {
String replicaBaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
String replicaBaseUrl = replica.getBaseUrl();
assertNotNull(replicaBaseUrl);
List<JettySolrRunner> runners = new ArrayList<>(jettys);
@ -838,7 +840,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
int port = new URI(((HttpSolrClient) client).getBaseURL())
.getPort();
if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
if (replica.getBaseUrl().contains(":" + port)) {
CloudSolrServerClient csc = new CloudSolrServerClient();
csc.solrClient = client;
csc.port = port;
@ -864,7 +866,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
Set<Entry<String,Replica>> entries = slice.getReplicasMap().entrySet();
for (Entry<String,Replica> entry : entries) {
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());
if (list == null) {
list = new ArrayList<>();
@ -876,7 +878,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
cjr.info = replica;
cjr.nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
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);
list.add(cjr);
if (isLeader) {
@ -2000,8 +2003,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
Set<Map.Entry<String,Replica>> shardEntries = shards.entrySet();
for (Map.Entry<String,Replica> shardEntry : shardEntries) {
final ZkNodeProps node = shardEntry.getValue();
if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP))) {
return ZkCoreNodeProps.getCoreUrl(node.getStr(ZkStateReader.BASE_URL_PROP), collection); //new ZkCoreNodeProps(node).getCoreUrl();
final String nodeName = node.getStr(ZkStateReader.NODE_NAME_PROP);
if (clusterState.liveNodesContain(nodeName)) {
return ZkCoreNodeProps.getCoreUrl(UrlScheme.INSTANCE.getBaseUrlForNodeName(nodeName), collection);
}
}
}