SOLR-15157: refactor Collection API to separate from Overseer and message handling abstractions (#2390)

No functional changes. In preparation of distributing the Collection API command execution.
This commit is contained in:
Ilan Ginzburg 2021-02-19 14:40:23 +01:00 committed by GitHub
parent 6deee14382
commit e7c80f6445
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 1695 additions and 1393 deletions

View File

@ -43,8 +43,8 @@ import org.apache.solr.common.params.CollectionAdminParams;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SHARD_UNIQUE;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
// Class to encapsulate processing replica properties that have at most one replica hosting a property per slice.

View File

@ -18,8 +18,8 @@
package org.apache.solr.cloud.api.collections;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
@ -45,7 +45,7 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@ -67,13 +67,13 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public class AddReplicaCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public AddReplicaCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public AddReplicaCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -94,7 +94,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
final String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
@ -141,17 +141,17 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount,
ocmh.overseer.getCoreContainer())
List<CreateReplica> createReplicas = buildReplicaPositions(ccc.getSolrCloudManager(), clusterState, collectionName, message, replicaTypesVsCount,
ccc.getCoreContainer())
.stream()
.map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, clusterState, message, replicaPosition))
.map(replicaPosition -> assignReplicaDetails(ccc.getSolrCloudManager(), clusterState, message, replicaPosition))
.collect(Collectors.toList());
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ZkStateReader zkStateReader = ocmh.zkStateReader;
ShardHandler shardHandler = ccc.getShardHandler();
ZkStateReader zkStateReader = ccc.getZkStateReader();
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (CreateReplica createReplica : createReplicas) {
assert createReplica.coreName != null;
ModifiableSolrParams params = getReplicaParams(clusterState, message, results, collectionName, coll, skipCreateReplicaInClusterState, asyncId, shardHandler, createReplica);
@ -161,14 +161,14 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
Runnable runnable = () -> {
shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
for (CreateReplica replica : createReplicas) {
ocmh.waitForCoreNodeName(collectionName, replica.node, replica.coreName);
CollectionHandlingUtils.waitForCoreNodeName(collectionName, replica.node, replica.coreName, ccc.getZkStateReader());
}
if (onComplete != null) onComplete.run();
};
if (!parallel || waitForFinalState) {
if (waitForFinalState) {
SolrCloseableLatch latch = new SolrCloseableLatch(totalReplicas, ocmh);
SolrCloseableLatch latch = new SolrCloseableLatch(totalReplicas, ccc.getCloseableToLatchOn());
ActiveReplicaWatcher watcher = new ActiveReplicaWatcher(collectionName, null,
createReplicas.stream().map(createReplica -> createReplica.coreName).collect(Collectors.toList()), latch);
try {
@ -184,7 +184,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
runnable.run();
}
} else {
ocmh.tpe.submit(runnable);
ccc.getExecutorService().submit(runnable);
}
return createReplicas.stream()
@ -198,7 +198,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, String collectionName, DocCollection coll, boolean skipCreateReplicaInClusterState, String asyncId, ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
if (!skipCreateReplicaInClusterState) {
ZkNodeProps props = new ZkNodeProps(
Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
@ -211,12 +211,12 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (createReplica.coreNodeName != null) {
props = props.plus(ZkStateReader.CORE_NODE_NAME_PROP, createReplica.coreNodeName);
}
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceAddReplica, props,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceAddReplica, props,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
try {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Exception updating Overseer state queue", e);
}
@ -225,7 +225,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.CORE_NODE_NAME,
ocmh.waitToSeeReplicasInState(collectionName, Collections.singleton(createReplica.coreName)).get(createReplica.coreName).getName());
CollectionHandlingUtils.waitToSeeReplicasInState(ccc.getZkStateReader(), ccc.getSolrCloudManager().getTimeSource(), collectionName, Collections.singleton(createReplica.coreName)).get(createReplica.coreName).getName());
String configName = zkStateReader.readConfigName(collectionName);
String routeKey = message.getStr(ShardParams._ROUTE_);
@ -262,7 +262,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (createReplica.coreNodeName != null) {
params.set(CoreAdminParams.CORE_NODE_NAME, createReplica.coreNodeName);
}
ocmh.addPropertyParams(message, params);
CollectionHandlingUtils.addPropertyParams(message, params);
return params;
}
@ -320,10 +320,10 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
String node = message.getStr(CoreAdminParams.NODE);
Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
Object createNodeSetStr = message.get(CollectionHandlingUtils.CREATE_NODE_SET);
if (createNodeSetStr == null) {
if (node != null) {
message.getProperties().put(OverseerCollectionMessageHandler.CREATE_NODE_SET, node);
message.getProperties().put(CollectionHandlingUtils.CREATE_NODE_SET, node);
createNodeSetStr = node;
}
}

View File

@ -41,12 +41,12 @@ import static org.apache.solr.common.params.CommonParams.NAME;
* means, given the current state of the alias and some information from a routed field in a document that
* may imply a need for changes, create, delete or otherwise modify collections as required.
*/
abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
abstract class AliasCmd implements CollApiCmds.CollectionApiCommand {
final OverseerCollectionMessageHandler ocmh;
protected final CollectionCommandContext ccc;
AliasCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
protected AliasCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
/**
@ -55,7 +55,7 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
*/
@SuppressWarnings({"rawtypes"})
static NamedList createCollectionAndWait(ClusterState clusterState, String aliasName, Map<String, String> aliasMetadata,
String createCollName, OverseerCollectionMessageHandler ocmh) throws Exception {
String createCollName, CollectionCommandContext ccc) throws Exception {
// Map alias metadata starting with a prefix to a create-collection API request
final ModifiableSolrParams createReqParams = new ModifiableSolrParams();
for (Map.Entry<String, String> e : aliasMetadata.entrySet()) {
@ -73,7 +73,7 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
final Map<String, Object> createMsgMap = CollectionsHandler.CollectionOperation.CREATE_OP.execute(
new LocalSolrQueryRequest(null, createReqParams),
null,
ocmh.overseer.getCoreContainer().getCollectionsHandler());
ccc.getCoreContainer().getCollectionsHandler());
createMsgMap.put(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower());
NamedList results = new NamedList();
@ -81,7 +81,7 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
// Since we are running in the Overseer here, send the message directly to the Overseer CreateCollectionCmd.
// note: there's doesn't seem to be any point in locking on the collection name, so we don't. We currently should
// already have a lock on the alias name which should be sufficient.
ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
new CreateCollectionCmd(ccc).call(clusterState, new ZkNodeProps(createMsgMap), results);
} catch (SolrException e) {
// The collection might already exist, and that's okay -- we can adopt it.
if (!e.getMessage().contains("collection already exists")) {
@ -89,11 +89,10 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
CollectionsHandler.waitForActiveCollection(createCollName, ocmh.overseer.getCoreContainer(),
new OverseerSolrResponse(results));
CollectionProperties collectionProperties = new CollectionProperties(ocmh.zkStateReader.getZkClient());
CollectionsHandler.waitForActiveCollection(createCollName, ccc.getCoreContainer(), new OverseerSolrResponse(results));
CollectionProperties collectionProperties = new CollectionProperties(ccc.getZkStateReader().getZkClient());
collectionProperties.setCollectionProperty(createCollName,ROUTED_ALIAS_NAME_CORE_PROP,aliasName);
while (!ocmh.zkStateReader.getCollectionProperties(createCollName,1000).containsKey(ROUTED_ALIAS_NAME_CORE_PROP)) {
while (!ccc.getZkStateReader().getCollectionProperties(createCollName,1000).containsKey(ROUTED_ALIAS_NAME_CORE_PROP)) {
Thread.sleep(50);
}
return results;

View File

@ -16,7 +16,7 @@
*/
package org.apache.solr.cloud.api.collections;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import java.io.IOException;
@ -231,14 +231,14 @@ public class Assign {
List<String> nodeList;
final String createNodeSetStr = message.getStr(CREATE_NODE_SET);
final List<String> createNodeList = (createNodeSetStr == null) ? null :
StrUtils.splitSmart((OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY.equals(createNodeSetStr) ?
StrUtils.splitSmart((CollectionHandlingUtils.CREATE_NODE_SET_EMPTY.equals(createNodeSetStr) ?
"" : createNodeSetStr), ",", true);
if (createNodeList != null) {
nodeList = new ArrayList<>(createNodeList);
nodeList.retainAll(liveNodes);
if (message.getBool(OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE,
OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE_DEFAULT)) {
if (message.getBool(CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE,
CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE_DEFAULT)) {
Collections.shuffle(nodeList, random);
}
} else {

View File

@ -16,7 +16,7 @@
*/
package org.apache.solr.cloud.api.collections;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
@ -57,13 +57,13 @@ import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
public class BackupCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public BackupCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public BackupCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -74,19 +74,19 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
String backupName = message.getStr(NAME);
String repo = message.getStr(CoreAdminParams.BACKUP_REPOSITORY);
boolean incremental = message.getBool(CoreAdminParams.BACKUP_INCREMENTAL, true);
String configName = ocmh.zkStateReader.readConfigName(collectionName);
String configName = ccc.getZkStateReader().readConfigName(collectionName);
BackupProperties backupProperties = BackupProperties.create(backupName, collectionName,
extCollectionName, configName);
CoreContainer cc = ocmh.overseer.getCoreContainer();
CoreContainer cc = ccc.getCoreContainer();
try (BackupRepository repository = cc.newBackupRepository(repo)) {
// Backup location
@ -94,8 +94,8 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
final URI backupUri = createAndValidateBackupPath(repository, incremental, location, backupName, collectionName);
BackupManager backupMgr = (incremental) ?
BackupManager.forIncrementalBackup(repository, ocmh.zkStateReader, backupUri) :
BackupManager.forBackup(repository, ocmh.zkStateReader, backupUri);
BackupManager.forIncrementalBackup(repository, ccc.getZkStateReader(), backupUri) :
BackupManager.forBackup(repository, ccc.getZkStateReader(), backupUri);
String strategy = message.getStr(CollectionAdminParams.INDEX_BACKUP_STRATEGY, CollectionAdminParams.COPY_FILES_STRATEGY);
switch (strategy) {
@ -105,7 +105,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
incrementalCopyIndexFiles(backupUri, collectionName, message, results, backupProperties, backupMgr);
} catch (SolrException e) {
log.error("Error happened during incremental backup for collection:{}", collectionName, e);
ocmh.cleanBackup(repository, backupUri, backupMgr.getBackupId());
CollectionHandlingUtils.cleanBackup(repository, backupUri, backupMgr.getBackupId(), ccc);
throw e;
}
} else {
@ -125,7 +125,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
//Save the collection's state. Can be part of the monolithic clusterstate.json or a individual state.json
//Since we don't want to distinguish we extract the state and back it up as a separate json
DocCollection collectionState = ocmh.zkStateReader.getClusterState().getCollection(collectionName);
DocCollection collectionState = ccc.getZkStateReader().getClusterState().getCollection(collectionName);
backupMgr.writeCollectionState(collectionName, collectionState);
backupMgr.downloadCollectionProperties(collectionName);
@ -139,7 +139,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
int maxNumBackup = message.getInt(CoreAdminParams.MAX_NUM_BACKUP_POINTS, -1);
if (incremental && maxNumBackup != -1) {
ocmh.deleteBackup(repository, backupUri, maxNumBackup, results);
CollectionHandlingUtils.deleteBackup(repository, backupUri, maxNumBackup, results, ccc);
}
}
}
@ -210,15 +210,15 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
String backupName = request.getStr(NAME);
String asyncId = request.getStr(ASYNC);
String repoName = request.getStr(CoreAdminParams.BACKUP_REPOSITORY);
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
log.info("Starting backup of collection={} with backupName={} at location={}", collectionName, backupName,
backupUri);
Optional<BackupProperties> previousProps = backupManager.tryReadBackupProperties();
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
Collection<Slice> slices = ocmh.zkStateReader.getClusterState().getCollection(collectionName).getActiveSlices();
Collection<Slice> slices = ccc.getZkStateReader().getClusterState().getCollection(collectionName).getActiveSlices();
for (Slice slice : slices) {
// Note - Actually this can return a null value when there is no leader for this shard.
Replica replica = slice.getLeader();
@ -293,12 +293,12 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
String backupName = request.getStr(NAME);
String asyncId = request.getStr(ASYNC);
String repoName = request.getStr(CoreAdminParams.BACKUP_REPOSITORY);
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
String commitName = request.getStr(CoreAdminParams.COMMIT_NAME);
Optional<CollectionSnapshotMetaData> snapshotMeta = Optional.empty();
if (commitName != null) {
SolrZkClient zkClient = ocmh.zkStateReader.getZkClient();
SolrZkClient zkClient = ccc.getZkStateReader().getZkClient();
snapshotMeta = SolrSnapshotManager.getCollectionLevelSnapshot(zkClient, collectionName, commitName);
if (!snapshotMeta.isPresent()) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Snapshot with name " + commitName
@ -318,8 +318,8 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
shardsToConsider = snapshotMeta.get().getShards();
}
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
for (Slice slice : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getActiveSlices()) {
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (Slice slice : ccc.getZkStateReader().getClusterState().getCollection(collectionName).getActiveSlices()) {
Replica replica = null;
if (snapshotMeta.isPresent()) {

View File

@ -0,0 +1,279 @@
/*
* 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.cloud.api.collections;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
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.CollectionAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
/**
* This class contains "smaller" Collection API commands implementation as well as the interface implemented by all commands.
* Previously these implementations in {@link OverseerCollectionMessageHandler} were relying on methods implementing the
* functional interface.
*/
public class CollApiCmds {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
/**
* Interface implemented by all Collection API commands. Collection API commands are defined in classes whose names ends in {@code Cmd}.
*/
protected interface CollectionApiCommand {
void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception;
}
static public class MockOperationCmd implements CollectionApiCommand {
@SuppressForbidden(reason = "Needs currentTimeMillis for mock requests")
@SuppressWarnings({"unchecked"})
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws InterruptedException {
//only for test purposes
Thread.sleep(message.getInt("sleep", 1));
if (log.isInfoEnabled()) {
log.info("MOCK_TASK_EXECUTED time {} data {}", System.currentTimeMillis(), Utils.toJSONString(message));
}
results.add("MOCK_FINISHED", System.currentTimeMillis());
}
}
static public class ReloadCollectionCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public ReloadCollectionCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@SuppressWarnings({"unchecked"})
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RELOAD.toString());
String asyncId = message.getStr(ASYNC);
CollectionHandlingUtils.collectionCmd(message, params, results, Replica.State.ACTIVE, asyncId, Collections.emptySet(), ccc, clusterState);
}
}
static public class RebalanceLeadersCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public RebalanceLeadersCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@SuppressWarnings("unchecked")
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results)
throws Exception {
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, CORE_NAME_PROP, ELECTION_NODE_PROP,
CORE_NODE_NAME_PROP, NODE_NAME_PROP, REJOIN_AT_HEAD_PROP);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
params.set(SHARD_ID_PROP, message.getStr(SHARD_ID_PROP));
params.set(REJOIN_AT_HEAD_PROP, message.getStr(REJOIN_AT_HEAD_PROP));
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REJOINLEADERELECTION.toString());
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(NODE_NAME_PROP, message.getStr(NODE_NAME_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...
params.set("qt", ccc.getAdminPath());
sreq.purpose = ShardRequest.PURPOSE_PRIVATE;
sreq.shards = new String[]{baseUrl};
sreq.actualShards = sreq.shards;
sreq.params = params;
ShardHandler shardHandler = ccc.getShardHandler();
shardHandler.submit(sreq, baseUrl, sreq.params);
}
}
static public class AddReplicaPropCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public AddReplicaPropCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@SuppressWarnings("unchecked")
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP, PROPERTY_VALUE_PROP);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, ADDREPLICAPROP.toLower());
propMap.putAll(message.getProperties());
ZkNodeProps m = new ZkNodeProps(propMap);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ReplicaAddReplicaProperty, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
}
static public class DeleteReplicaPropCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public DeleteReplicaPropCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results)
throws Exception {
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, DELETEREPLICAPROP.toLower());
propMap.putAll(message.getProperties());
ZkNodeProps m = new ZkNodeProps(propMap);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ReplicaDeleteReplicaProperty, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
}
static public class BalanceShardsUniqueCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public BalanceShardsUniqueCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
"' parameters are required for the BALANCESHARDUNIQUE operation, no action taken");
}
Map<String, Object> m = new HashMap<>();
m.put(Overseer.QUEUE_OPERATION, BALANCESHARDUNIQUE.toLower());
m.putAll(message.getProperties());
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.BalanceShardsUnique, new ZkNodeProps(m),
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
}
static public class ModifyCollectionCmd implements CollectionApiCommand {
private final CollectionCommandContext ccc;
public ModifyCollectionCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
//the rest of the processing is based on writing cluster state properties
//remove the property here to avoid any errors down the pipeline due to this property appearing
String configName = (String) message.getProperties().remove(CollectionAdminParams.COLL_CONF);
if (configName != null) {
CollectionHandlingUtils.validateConfigOrThrowSolrException(ccc.getSolrCloudManager(), configName);
CollectionHandlingUtils.createConfNode(ccc.getSolrCloudManager().getDistribStateManager(), configName, collectionName);
new ReloadCollectionCmd(ccc).call(clusterState, new ZkNodeProps(NAME, collectionName), results);
}
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// Apply the state update right away. The wait will still be useful for the change to be visible in the local cluster state (watchers have fired).
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, message,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ccc.offerStateUpdate(Utils.toJSON(message));
}
try {
ccc.getZkStateReader().waitForState(collectionName, 30, TimeUnit.SECONDS, c -> {
if (c == null) return false;
for (Map.Entry<String, Object> updateEntry : message.getProperties().entrySet()) {
String updateKey = updateEntry.getKey();
if (!updateKey.equals(ZkStateReader.COLLECTION_PROP)
&& !updateKey.equals(Overseer.QUEUE_OPERATION)
&& updateEntry.getValue() != null // handled below in a separate conditional
&& !updateEntry.getValue().equals(c.get(updateKey))) {
return false;
}
if (updateEntry.getValue() == null && c.containsKey(updateKey)) {
return false;
}
}
return true;
});
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
log.debug("modifyCollection(ClusterState={}, ZkNodeProps={}, NamedList={})", clusterState, message, results, e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to modify collection", e);
}
// if switching to/from read-only mode reload the collection
if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
new ReloadCollectionCmd(ccc).call(clusterState, new ZkNodeProps(NAME, collectionName), results);
}
}
}
}

View File

@ -0,0 +1,90 @@
/*
* 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.cloud.api.collections;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.zookeeper.KeeperException;
import java.util.concurrent.ExecutorService;
/**
* Data passed to Collection API command execution, to allow calls from either the {@link OverseerCollectionMessageHandler}
* when commands are executed on the Overseer, or - in a future change - allow Collection API commands to be executed in a
* distributed way, unrelated to and not depending upon Overseer abstractions (including overseer collection message handling).
*/
public interface CollectionCommandContext {
ShardHandler getShardHandler();
SolrCloudManager getSolrCloudManager();
ZkStateReader getZkStateReader();
DistributedClusterStateUpdater getDistributedClusterStateUpdater();
CoreContainer getCoreContainer();
default ShardRequestTracker asyncRequestTracker(String asyncId) {
return new ShardRequestTracker(asyncId, getAdminPath(), getZkStateReader(), getShardHandler().getShardHandlerFactory());
}
/**
* admin path passed to Overseer is a constant, including in tests.
*/
default String getAdminPath() {
return CommonParams.CORES_HANDLER_PATH;
}
SolrCloseable getCloseableToLatchOn();
ExecutorService getExecutorService();
/**
* This method enables the commands to enqueue to the overseer cluster state update. This should only be used when the command
* is running in the Overseer (and will throw an exception if called when Collection API is distributed)
*/
default void offerStateUpdate(byte[] data) throws KeeperException, InterruptedException {
throw new IllegalStateException("Bug! offerStateUpdate() should not be called when distributed state updates are enabled");
}
default String getOverseerId() {
throw new IllegalStateException("Bug! getOverseerId() default implementation should never be called");
}
/**
* Command delegating to Overseer to retrieve cluster state update stats from a Collection API call. This does not make
* sense when cluster state updates are distributed given Overseer does not see them and can't collect stats.
*/
default Stats getOverseerStats() {
throw new IllegalStateException("Bug! getOverseerStats() should not be called when distributed state updates are enabled");
}
/**
* Method used by Per Replica States implementation to force the cluster state updater to immediately reload a collection from Zookeeper.
* This method is not used when cluster state updates are distributed.
*/
default void submitIntraProcessMessage(Overseer.Message message) {
throw new IllegalStateException("Bug! submitIntraProcessMessage() should not be called when distributed state updates are enabled");
}
}

View File

@ -0,0 +1,592 @@
/*
* 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.cloud.api.collections;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.*;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.backup.BackupId;
import org.apache.solr.core.backup.repository.BackupRepository;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.Utils.makeMap;
/**
* This class contains helper methods used by commands of the Collection API. Previously these methods were in
* {@link OverseerCollectionMessageHandler} and were refactored out to (eventually) allow Collection API commands to be
* executed outside the context of the Overseer.
*/
public class CollectionHandlingUtils {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String NUM_SLICES = "numShards";
public static final boolean CREATE_NODE_SET_SHUFFLE_DEFAULT = true;
public static final String CREATE_NODE_SET_SHUFFLE = CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
public static final String CREATE_NODE_SET_EMPTY = "EMPTY";
public static final String CREATE_NODE_SET = CollectionAdminParams.CREATE_NODE_SET_PARAM;
public static final String ROUTER = "router";
public static final String SHARDS_PROP = "shards";
public static final String REQUESTID = "requestid";
public static final String ONLY_IF_DOWN = "onlyIfDown";
public static final String SHARD_UNIQUE = "shardUnique";
public static final String ONLY_ACTIVE_NODES = "onlyactivenodes";
static final String SKIP_CREATE_REPLICA_IN_CLUSTER_STATE = "skipCreateReplicaInClusterState";
public static final Map<String, Object> COLLECTION_PROPS_AND_DEFAULTS = Collections.unmodifiableMap(makeMap(
ROUTER, DocRouter.DEFAULT_NAME,
ZkStateReader.REPLICATION_FACTOR, "1",
ZkStateReader.NRT_REPLICAS, "1",
ZkStateReader.TLOG_REPLICAS, "0",
DocCollection.PER_REPLICA_STATE, null,
ZkStateReader.PULL_REPLICAS, "0"));
protected static final Random RANDOM;
static {
// We try to make things reproducible in the context of our tests by initializing the random instance
// based on the current seed
String seed = System.getProperty("tests.seed");
if (seed == null) {
RANDOM = new Random();
} else {
RANDOM = new Random(seed.hashCode());
}
}
static boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms, ZkStateReader zkStateReader) throws InterruptedException {
try {
zkStateReader.waitForState(collectionName, timeoutms, TimeUnit.MILLISECONDS, (c) -> {
if (c == null)
return true;
Slice slice = c.getSlice(shard);
if(slice == null || slice.getReplica(replicaName) == null) {
return true;
}
return false;
});
} catch (TimeoutException e) {
return false;
}
return true;
}
static void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core, CollectionCommandContext ccc) throws Exception {
ZkNodeProps m = new ZkNodeProps(
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
ZkStateReader.CORE_NAME_PROP, core,
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.CORE_NODE_NAME_PROP, replicaName);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceRemoveReplica, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
static void checkRequired(ZkNodeProps message, String... props) {
for (String prop : props) {
if (message.get(prop) == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, StrUtils.join(Arrays.asList(props), ',') + " are required params");
}
}
}
static void checkResults(String label, NamedList<Object> results, boolean failureIsFatal) throws SolrException {
Object failure = results.get("failure");
if (failure == null) {
failure = results.get("error");
}
if (failure != null) {
String msg = "Error: " + label + ": " + Utils.toJSONString(results);
if (failureIsFatal) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
} else {
log.error(msg);
}
}
}
@SuppressWarnings({"unchecked"})
static void commit(@SuppressWarnings({"rawtypes"}) NamedList results, String slice, Replica parentShardLeader) {
log.debug("Calling soft commit to make sub shard updates visible");
String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
// HttpShardHandler is hard coded to send a QueryRequest hence we go direct
// and we force open a searcher so that we have documents to show upon switching states
UpdateResponse updateResponse = null;
try {
updateResponse = softCommit(coreUrl);
CollectionHandlingUtils.processResponse(results, null, coreUrl, updateResponse, slice, Collections.emptySet());
} catch (Exception e) {
CollectionHandlingUtils.processResponse(results, e, coreUrl, updateResponse, slice, Collections.emptySet());
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to call distrib softCommit on: " + coreUrl, e);
}
}
static UpdateResponse softCommit(String url) throws SolrServerException, IOException {
try (HttpSolrClient client = new HttpSolrClient.Builder(url)
.withConnectionTimeout(30000)
.withSocketTimeout(120000)
.build()) {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true, true);
return ureq.process(client);
}
}
public static String waitForCoreNodeName(String collectionName, String msgNodeName, String msgCore, ZkStateReader zkStateReader) {
try {
DocCollection collection = zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c ->
ClusterStateMutator.getAssignedCoreNodeName(c, msgNodeName, msgCore) != null
);
return ClusterStateMutator.getAssignedCoreNodeName(collection, msgNodeName, msgCore);
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed waiting for coreNodeName", e);
}
}
static ClusterState waitForNewShard(String collectionName, String sliceName, ZkStateReader zkStateReader) throws KeeperException, InterruptedException {
log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
try {
zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c -> {
return c != null && c.getSlice(sliceName) != null;
});
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed waiting for new slice", e);
}
return zkStateReader.getClusterState();
}
static void addPropertyParams(ZkNodeProps message, ModifiableSolrParams params) {
// Now add the property.key=value pairs
for (String key : message.keySet()) {
if (key.startsWith(CollectionAdminParams.PROPERTY_PREFIX)) {
params.set(key, message.getStr(key));
}
}
}
static void addPropertyParams(ZkNodeProps message, Map<String, Object> map) {
// Now add the property.key=value pairs
for (String key : message.keySet()) {
if (key.startsWith(CollectionAdminParams.PROPERTY_PREFIX)) {
map.put(key, message.getStr(key));
}
}
}
static void cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results, CollectionCommandContext ccc) throws Exception {
log.error("Cleaning up collection [{}].", collectionName);
Map<String, Object> props = makeMap(
Overseer.QUEUE_OPERATION, DELETE.toLower(),
NAME, collectionName);
new DeleteCollectionCmd(ccc).call(ccc.getZkStateReader().getClusterState(), new ZkNodeProps(props), results);
}
static Map<String, Replica> waitToSeeReplicasInState(ZkStateReader zkStateReader, TimeSource timeSource, String collectionName, Collection<String> coreNames) throws InterruptedException {
assert coreNames.size() > 0;
Map<String, Replica> results = new ConcurrentHashMap<>();
long maxWait = Long.getLong("solr.waitToSeeReplicasInStateTimeoutSeconds", 120); // could be a big cluster
try {
zkStateReader.waitForState(collectionName, maxWait, TimeUnit.SECONDS, c -> {
if (c == null) return false;
// We write into a ConcurrentHashMap, which will be ok if called multiple times by multiple threads
c.getSlices().stream().flatMap(slice -> slice.getReplicas().stream())
.filter(r -> coreNames.contains(r.getCoreName())) // Only the elements that were asked for...
.forEach(r -> results.putIfAbsent(r.getCoreName(), r)); // ...get added to the map
log.debug("Expecting {} cores, found {}", coreNames, results);
return results.size() == coreNames.size();
});
} catch (TimeoutException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e.getMessage(), e);
}
return results;
}
@SuppressWarnings({"rawtypes"})
static void cleanBackup(BackupRepository repository, URI backupUri, BackupId backupId, CollectionCommandContext ccc) throws Exception {
new DeleteBackupCmd(ccc).deleteBackupIds(backupUri, repository, Collections.singleton(backupId), new NamedList());
}
static void deleteBackup(BackupRepository repository, URI backupPath, int maxNumBackup,
@SuppressWarnings({"rawtypes"}) NamedList results, CollectionCommandContext ccc) throws Exception {
new DeleteBackupCmd(ccc).keepNumberOfBackup(repository, backupPath, maxNumBackup, results);
}
static List<ZkNodeProps> addReplica(ClusterState clusterState, ZkNodeProps message,
@SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete, CollectionCommandContext ccc)
throws Exception {
return new AddReplicaCmd(ccc).addReplica(clusterState, message, results, onComplete);
}
static void validateConfigOrThrowSolrException(SolrCloudManager cloudManager, String configName) throws IOException, KeeperException, InterruptedException {
boolean isValid = cloudManager.getDistribStateManager().hasData(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
if (!isValid) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not find the specified config set: " + configName);
}
}
/**
* This doesn't validate the config (path) itself and is just responsible for creating the confNode.
* That check should be done before the config node is created.
*/
public static void createConfNode(DistribStateManager stateManager, String configName, String coll) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException {
if (configName != null) {
String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
log.debug("creating collections conf node {} ", collDir);
byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
if (stateManager.hasData(collDir)) {
stateManager.setData(collDir, data, -1);
} else {
stateManager.makePath(collDir, data, CreateMode.PERSISTENT, false);
}
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,"Unable to get config name");
}
}
/**
* Send request to all replicas of a collection
* @return List of replicas which is not live for receiving the request
*/
static List<Replica> collectionCmd(ZkNodeProps message, ModifiableSolrParams params,
NamedList<Object> results, Replica.State stateMatcher, String asyncId, Set<String> okayExceptions,
CollectionCommandContext ccc, ClusterState clusterState) {
log.info("Executing Collection Cmd={}, asyncId={}", params, asyncId);
String collectionName = message.getStr(NAME);
ShardHandler shardHandler = ccc.getShardHandler();
DocCollection coll = clusterState.getCollection(collectionName);
List<Replica> notLivesReplicas = new ArrayList<>();
final CollectionHandlingUtils.ShardRequestTracker shardRequestTracker = asyncRequestTracker(asyncId, ccc);
for (Slice slice : coll.getSlices()) {
notLivesReplicas.addAll(shardRequestTracker.sliceCmd(clusterState, params, stateMatcher, slice, shardHandler));
}
shardRequestTracker.processResponses(results, shardHandler, false, null, okayExceptions);
return notLivesReplicas;
}
static void processResponse(NamedList<Object> results, ShardResponse srsp, Set<String> okayExceptions) {
Throwable e = srsp.getException();
String nodeName = srsp.getNodeName();
SolrResponse solrResponse = srsp.getSolrResponse();
String shard = srsp.getShard();
processResponse(results, e, nodeName, solrResponse, shard, okayExceptions);
}
@SuppressWarnings("deprecation")
static void processResponse(NamedList<Object> results, Throwable e, String nodeName, SolrResponse solrResponse, String shard, Set<String> okayExceptions) {
String rootThrowable = null;
if (e instanceof BaseHttpSolrClient.RemoteSolrException) {
rootThrowable = ((BaseHttpSolrClient.RemoteSolrException) e).getRootThrowable();
}
if (e != null && (rootThrowable == null || !okayExceptions.contains(rootThrowable))) {
log.error("Error from shard: {}", shard, e);
addFailure(results, nodeName, e.getClass().getName() + ":" + e.getMessage());
} else {
addSuccess(results, nodeName, solrResponse.getResponse());
}
}
@SuppressWarnings("unchecked")
private static void addFailure(NamedList<Object> results, String key, Object value) {
SimpleOrderedMap<Object> failure = (SimpleOrderedMap<Object>) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap<>();
results.add("failure", failure);
}
failure.add(key, value);
}
@SuppressWarnings("unchecked")
private static void addSuccess(NamedList<Object> results, String key, Object value) {
SimpleOrderedMap<Object> success = (SimpleOrderedMap<Object>) results.get("success");
if (success == null) {
success = new SimpleOrderedMap<>();
results.add("success", success);
}
success.add(key, value);
}
private static NamedList<Object> waitForCoreAdminAsyncCallToComplete(ShardHandlerFactory shardHandlerFactory, String adminPath, ZkStateReader zkStateReader, String nodeName, String requestId) {
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTSTATUS.toString());
params.set(CoreAdminParams.REQUESTID, requestId);
int counter = 0;
ShardRequest sreq;
do {
sreq = new ShardRequest();
params.set("qt", adminPath);
sreq.purpose = 1;
String replica = zkStateReader.getBaseUrlForNodeName(nodeName);
sreq.shards = new String[] {replica};
sreq.actualShards = sreq.shards;
sreq.params = params;
shardHandler.submit(sreq, replica, sreq.params);
ShardResponse srsp;
do {
srsp = shardHandler.takeCompletedOrError();
if (srsp != null) {
NamedList<Object> results = new NamedList<>();
processResponse(results, srsp, Collections.emptySet());
if (srsp.getSolrResponse().getResponse() == null) {
NamedList<Object> response = new NamedList<>();
response.add("STATUS", "failed");
return response;
}
String r = (String) srsp.getSolrResponse().getResponse().get("STATUS");
if (r.equals("running")) {
log.debug("The task is still RUNNING, continuing to wait.");
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
continue;
} else if (r.equals("completed")) {
log.debug("The task is COMPLETED, returning");
return srsp.getSolrResponse().getResponse();
} else if (r.equals("failed")) {
// TODO: Improve this. Get more information.
log.debug("The task is FAILED, returning");
return srsp.getSolrResponse().getResponse();
} else if (r.equals("notfound")) {
log.debug("The task is notfound, retry");
if (counter++ < 5) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
break;
}
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid status request for requestId: " + requestId + "" + srsp.getSolrResponse().getResponse().get("STATUS") +
"retried " + counter + "times");
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid status request " + srsp.getSolrResponse().getResponse().get("STATUS"));
}
}
} while (srsp != null);
} while(true);
}
public static ShardRequestTracker syncRequestTracker(CollectionCommandContext ccc) {
return asyncRequestTracker(null, ccc);
}
public static ShardRequestTracker asyncRequestTracker(String asyncId, CollectionCommandContext ccc) {
return new ShardRequestTracker(asyncId, ccc.getAdminPath(), ccc.getZkStateReader(), ccc.getShardHandler().getShardHandlerFactory());
}
public static class ShardRequestTracker {
/*
* backward compatibility reasons, add the response with the async ID as top level.
* This can be removed in Solr 9
*/
@Deprecated
static boolean INCLUDE_TOP_LEVEL_RESPONSE = true;
private final String asyncId;
private final String adminPath;
private final ZkStateReader zkStateReader;
private final ShardHandlerFactory shardHandlerFactory;
private final NamedList<String> shardAsyncIdByNode = new NamedList<String>();
public ShardRequestTracker(String asyncId, String adminPath, ZkStateReader zkStateReader, ShardHandlerFactory shardHandlerFactory) {
this.asyncId = asyncId;
this.adminPath = adminPath;
this.zkStateReader = zkStateReader;
this.shardHandlerFactory = shardHandlerFactory;
}
/**
* Send request to all replicas of a slice
* @return List of replicas which is not live for receiving the request
*/
public List<Replica> sliceCmd(ClusterState clusterState, ModifiableSolrParams params, Replica.State stateMatcher,
Slice slice, ShardHandler shardHandler) {
List<Replica> notLiveReplicas = new ArrayList<>();
for (Replica replica : slice.getReplicas()) {
if ((stateMatcher == null || Replica.State.getState(replica.getStr(ZkStateReader.STATE_PROP)) == stateMatcher)) {
if (clusterState.liveNodesContain(replica.getStr(ZkStateReader.NODE_NAME_PROP))) {
// For thread safety, only simple clone the ModifiableSolrParams
ModifiableSolrParams cloneParams = new ModifiableSolrParams();
cloneParams.add(params);
cloneParams.set(CoreAdminParams.CORE, replica.getStr(ZkStateReader.CORE_NAME_PROP));
sendShardRequest(replica.getStr(ZkStateReader.NODE_NAME_PROP), cloneParams, shardHandler);
} else {
notLiveReplicas.add(replica);
}
}
}
return notLiveReplicas;
}
public void sendShardRequest(String nodeName, ModifiableSolrParams params,
ShardHandler shardHandler) {
sendShardRequest(nodeName, params, shardHandler, adminPath, zkStateReader);
}
public void sendShardRequest(String nodeName, ModifiableSolrParams params, ShardHandler shardHandler,
String adminPath, ZkStateReader zkStateReader) {
if (asyncId != null) {
String coreAdminAsyncId = asyncId + Math.abs(System.nanoTime());
params.set(ASYNC, coreAdminAsyncId);
track(nodeName, coreAdminAsyncId);
}
ShardRequest sreq = new ShardRequest();
params.set("qt", adminPath);
sreq.purpose = 1;
String replica = zkStateReader.getBaseUrlForNodeName(nodeName);
sreq.shards = new String[] {replica};
sreq.actualShards = sreq.shards;
sreq.nodeName = nodeName;
sreq.params = params;
shardHandler.submit(sreq, replica, sreq.params);
}
void processResponses(NamedList<Object> results, ShardHandler shardHandler, boolean abortOnError, String msgOnError) {
processResponses(results, shardHandler, abortOnError, msgOnError, Collections.emptySet());
}
void processResponses(NamedList<Object> results, ShardHandler shardHandler, boolean abortOnError, String msgOnError,
Set<String> okayExceptions) {
// Processes all shard responses
ShardResponse srsp;
do {
srsp = shardHandler.takeCompletedOrError();
if (srsp != null) {
processResponse(results, srsp, okayExceptions);
Throwable exception = srsp.getException();
if (abortOnError && exception != null) {
// drain pending requests
while (srsp != null) {
srsp = shardHandler.takeCompletedOrError();
}
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msgOnError, exception);
}
}
} while (srsp != null);
// If request is async wait for the core admin to complete before returning
if (asyncId != null) {
waitForAsyncCallsToComplete(results); // TODO: Shouldn't we abort with msgOnError exception when failure?
shardAsyncIdByNode.clear();
}
}
private void waitForAsyncCallsToComplete(NamedList<Object> results) {
for (Map.Entry<String,String> nodeToAsync:shardAsyncIdByNode) {
final String node = nodeToAsync.getKey();
final String shardAsyncId = nodeToAsync.getValue();
log.debug("I am Waiting for :{}/{}", node, shardAsyncId);
NamedList<Object> reqResult = waitForCoreAdminAsyncCallToComplete(shardHandlerFactory, adminPath, zkStateReader, node, shardAsyncId);
if (INCLUDE_TOP_LEVEL_RESPONSE) {
results.add(shardAsyncId, reqResult);
}
if ("failed".equalsIgnoreCase(((String)reqResult.get("STATUS")))) {
log.error("Error from shard {}: {}", node, reqResult);
addFailure(results, node, reqResult);
} else {
addSuccess(results, node, reqResult);
}
}
}
/** @deprecated consider to make it private after {@link CreateCollectionCmd} refactoring*/
@Deprecated void track(String nodeName, String coreAdminAsyncId) {
shardAsyncIdByNode.add(nodeName, coreAdminAsyncId);
}
}
}

View File

@ -47,15 +47,15 @@ public class CreateAliasCmd extends AliasCmd {
}
@SuppressWarnings("WeakerAccess")
public CreateAliasCmd(OverseerCollectionMessageHandler ocmh) {
super(ocmh);
public CreateAliasCmd(CollectionCommandContext ccc) {
super(ccc);
}
@Override
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
final String aliasName = message.getStr(CommonParams.NAME);
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
// make sure we have the latest version of existing aliases
if (zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
zkStateReader.aliasesManager.update();
@ -142,7 +142,7 @@ public class CreateAliasCmd extends AliasCmd {
private void ensureAliasCollection(String aliasName, ZkStateReader zkStateReader, ClusterState state, Map<String, String> aliasProperties, String initialCollectionName) throws Exception {
// Create the collection
createCollectionAndWait(state, aliasName, aliasProperties, initialCollectionName, ocmh);
createCollectionAndWait(state, aliasName, aliasProperties, initialCollectionName, ccc);
validateAllCollectionsExistAndNoDuplicates(Collections.singletonList(initialCollectionName), zkStateReader);
}

View File

@ -42,7 +42,7 @@ import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.RefreshCollectionMessage;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.cloud.overseer.ZkWriteCommand;
@ -81,25 +81,25 @@ import static org.apache.solr.common.util.StrUtils.formatString;
import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
import static org.apache.solr.handler.admin.ConfigSetsHandler.getSuffixedNameForAutoGeneratedConfigSet;
public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
private final TimeSource timeSource;
private final DistribStateManager stateManager;
public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.stateManager = ocmh.cloudManager.getDistribStateManager();
this.timeSource = ocmh.cloudManager.getTimeSource();
public CreateCollectionCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
this.stateManager = ccc.getSolrCloudManager().getDistribStateManager();
this.timeSource = ccc.getSolrCloudManager().getTimeSource();
}
@Override
@SuppressWarnings({"unchecked"})
public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
if (ocmh.zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
ocmh.zkStateReader.aliasesManager.update();
if (ccc.getZkStateReader().aliasesManager != null) { // not a mock ZkStateReader
ccc.getZkStateReader().aliasesManager.update();
}
final Aliases aliases = ocmh.zkStateReader.getAliases();
final Aliases aliases = ccc.getZkStateReader().getAliases();
final String collectionName = message.getStr(NAME);
final boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
final String alias = message.getStr(ALIAS, collectionName);
@ -117,7 +117,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No config set found to associate with the collection.");
}
ocmh.validateConfigOrThrowSolrException(configName);
CollectionHandlingUtils.validateConfigOrThrowSolrException(ccc.getSolrCloudManager(), configName);
String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
@ -131,9 +131,9 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
final String async = message.getStr(ASYNC);
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
OverseerCollectionMessageHandler.createConfNode(stateManager, configName, collectionName);
CollectionHandlingUtils.createConfNode(stateManager, configName, collectionName);
Map<String,String> collectionParams = new HashMap<>();
Map<String,Object> collectionProps = message.getProperties();
@ -161,20 +161,24 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// This code directly updates Zookeeper by creating the collection state.json. It is compatible with both distributed
// cluster state updates and Overseer based cluster state updates.
ZkWriteCommand command = new ClusterStateMutator(ocmh.cloudManager).createCollection(clusterState, message);
ZkWriteCommand command = new ClusterStateMutator(ccc.getSolrCloudManager()).createCollection(clusterState, message);
byte[] data = Utils.toJSON(Collections.singletonMap(collectionName, command.collection));
ocmh.zkStateReader.getZkClient().create(collectionPath, data, CreateMode.PERSISTENT, true);
ccc.getZkStateReader().getZkClient().create(collectionPath, data, CreateMode.PERSISTENT, true);
clusterState = clusterState.copyWith(collectionName, command.collection);
newColl = command.collection;
ocmh.overseer.submit(new RefreshCollectionMessage(collectionName));
// When cluster state updates are handled by Overseer, ask it to load that collection it doesn't know about.
// When cluster state updates are distributed, ZK is the source of truth for all nodes so no reload needed.
if (!ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.submitIntraProcessMessage(new RefreshCollectionMessage(collectionName));
}
} else {
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// The message has been crafted by CollectionsHandler.CollectionOperation.CREATE_OP and defines the QUEUE_OPERATION
// to be CollectionParams.CollectionAction.CREATE.
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ClusterCreateCollection, message,
ocmh.cloudManager, ocmh.zkStateReader);
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ClusterCreateCollection, message,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
ccc.offerStateUpdate(Utils.toJSON(message));
}
// wait for a while until we see the collection
@ -182,7 +186,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
boolean created = false;
while (!waitUntil.hasTimedOut()) {
waitUntil.sleep(100);
created = ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
created = ccc.getSolrCloudManager().getClusterStateProvider().getClusterState().hasCollection(collectionName);
if (created) break;
}
if (!created) {
@ -191,17 +195,17 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// refresh cluster state (value read below comes from Zookeeper watch firing following the update done previously,
// be it by Overseer or by this thread when updates are distributed)
clusterState = ocmh.cloudManager.getClusterStateProvider().getClusterState();
clusterState = ccc.getSolrCloudManager().getClusterStateProvider().getClusterState();
newColl = clusterState.getCollection(collectionName);
}
final List<ReplicaPosition> replicaPositions;
try {
replicaPositions = buildReplicaPositions(ocmh.overseer.getCoreContainer(), ocmh.cloudManager, clusterState, newColl,
replicaPositions = buildReplicaPositions(ccc.getCoreContainer(), ccc.getSolrCloudManager(), clusterState, newColl,
message, shardNames);
} catch (Assign.AssignmentException e) {
ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
new DeleteCollectionCmd(ccc).call(clusterState, deleteMessage, results);
// unwrap the exception
throw new SolrException(ErrorCode.BAD_REQUEST, e.getMessage(), e.getCause());
}
@ -211,20 +215,20 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
return;
}
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(async);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(async, ccc);
if (log.isDebugEnabled()) {
log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}",
collectionName, shardNames, message));
}
Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
final DistributedClusterStateUpdater.StateChangeRecorder scr;
// PRS collections update Zookeeper directly, so even if we run in distributed state update,
// there's nothing to update in state.json for such collection in the loop over replica positions below.
if (!isPRS && ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (!isPRS && ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// The collection got created. Now we're adding replicas (and will update ZK only once when done adding).
scr = ocmh.getDistributedClusterStateUpdater().createStateChangeRecorder(collectionName, false);;
scr = ccc.getDistributedClusterStateUpdater().createStateChangeRecorder(collectionName, false);;
} else {
scr = null;
}
@ -232,8 +236,8 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
for (ReplicaPosition replicaPosition : replicaPositions) {
String nodeName = replicaPosition.node;
String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(),
ocmh.cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName),
String coreName = Assign.buildSolrCoreName(ccc.getSolrCloudManager().getDistribStateManager(),
ccc.getSolrCloudManager().getClusterStateProvider().getClusterState().getCollection(collectionName),
replicaPosition.shard, replicaPosition.type, true);
if (log.isDebugEnabled()) {
log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
@ -257,18 +261,19 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// to the overseer queue.
// TODO: Consider doing this for all collections, not just the PRS collections.
// TODO: consider doing this once after the loop for all replicas rather than writing state.json repeatedly
// This PRS specific code is compatible with both Overseer and distributed cluster state update strategies
ZkWriteCommand command = new SliceMutator(ocmh.cloudManager).addReplica(clusterState, props);
ZkWriteCommand command = new SliceMutator(ccc.getSolrCloudManager()).addReplica(clusterState, props);
byte[] data = Utils.toJSON(Collections.singletonMap(collectionName, command.collection));
// log.info("collection updated : {}", new String(data, StandardCharsets.UTF_8));
ocmh.zkStateReader.getZkClient().setData(collectionPath, data, true);
ccc.getZkStateReader().getZkClient().setData(collectionPath, data, true);
clusterState = clusterState.copyWith(collectionName, command.collection);
newColl = command.collection;
} else {
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
scr.record(DistributedClusterStateUpdater.MutatingCommand.SliceAddReplica, props);
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
}
}
@ -289,11 +294,11 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
params.add(ASYNC, coreAdminAsyncId);
shardRequestTracker.track(nodeName, coreAdminAsyncId);
}
ocmh.addPropertyParams(message, params);
CollectionHandlingUtils.addPropertyParams(message, params);
ShardRequest sreq = new ShardRequest();
sreq.nodeName = nodeName;
params.set("qt", ocmh.adminPath);
params.set("qt", ccc.getAdminPath());
sreq.purpose = 1;
sreq.shards = new String[]{baseUrl};
sreq.actualShards = sreq.shards;
@ -301,14 +306,18 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
coresToCreate.put(coreName, sreq);
}
if(isPRS) {
ocmh.overseer.submit(new RefreshCollectionMessage(collectionName));
// PRS collections updated ZK state.json in the loop above. When Overseer is managing cluster state updates, need to
// tell it to refresh itself to know about the replicas and be able to execute nodes shard requests regarding the replicas.
if (isPRS && !ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.submitIntraProcessMessage(new RefreshCollectionMessage(collectionName));
}
// PRS collections did their own thing and we didn't create a StateChangeRecorder for them
if (!isPRS && ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// Distributed updates don't need to do anything for PRS collections that wrote state.json directly
// For non PRS collections, distributed updates have to be executed if that's how the cluster is configured
if (!isPRS && ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// Add the replicas to the collection state (all at once after the loop above)
scr.executeStateUpdates(ocmh.cloudManager, ocmh.zkStateReader);
scr.executeStateUpdates(ccc.getSolrCloudManager(), ccc.getZkStateReader());
}
final Map<String, Replica> replicas;
@ -319,7 +328,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
.forEach(r -> replicas.putIfAbsent(r.getCoreName(), r)); // ...get added to the map
} else {
// wait for all replica entries to be created and visible in local cluster state (updated by ZK watches)
replicas = ocmh.waitToSeeReplicasInState(collectionName, coresToCreate.keySet());
replicas = CollectionHandlingUtils.waitToSeeReplicasInState(ccc.getZkStateReader(), ccc.getSolrCloudManager().getTimeSource(), collectionName, coresToCreate.keySet());
}
for (Map.Entry<String, ShardRequest> e : coresToCreate.entrySet()) {
@ -333,11 +342,11 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
boolean failure = results.get("failure") != null && ((SimpleOrderedMap)results.get("failure")).size() > 0;
if (isPRS) {
TimeOut timeout = new TimeOut(Integer.getInteger("solr.waitToSeeReplicasInStateTimeoutSeconds", 120), TimeUnit.SECONDS, timeSource); // could be a big cluster
PerReplicaStates prs = PerReplicaStates.fetch(collectionPath, ocmh.zkStateReader.getZkClient(), null);
PerReplicaStates prs = PerReplicaStates.fetch(collectionPath, ccc.getZkStateReader().getZkClient(), null);
while (!timeout.hasTimedOut()) {
if(prs.allActive()) break;
Thread.sleep(100);
prs = PerReplicaStates.fetch(collectionPath, ocmh.zkStateReader.getZkClient(), null);
prs = PerReplicaStates.fetch(collectionPath, ccc.getZkStateReader().getZkClient(), null);
}
if (prs.allActive()) {
// we have successfully found all replicas to be ACTIVE
@ -348,16 +357,16 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// of a new collection created elsewhere (which is how all collections are created).
// Note it is likely possibly to skip the the whole if (isPRS) bloc, but keeping distributed state updates as
// close in behavior to Overseer state updates for now.
if (!ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (!ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// Now ask Overseer to fetch the latest state of collection from ZK
ocmh.overseer.submit(new RefreshCollectionMessage(collectionName));
ccc.submitIntraProcessMessage(new RefreshCollectionMessage(collectionName));
}
}
if (failure) {
// Let's cleanup as we hit an exception
// We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
// element, which may be interpreted by the user as a positive ack
ocmh.cleanupCollection(collectionName, new NamedList<Object>());
CollectionHandlingUtils.cleanupCollection(collectionName, new NamedList<Object>(), ccc);
log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName);
} else {
@ -374,7 +383,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// create an alias pointing to the new collection, if different from the collectionName
if (!alias.equals(collectionName)) {
ocmh.zkStateReader.aliasesManager.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(alias, collectionName));
ccc.getZkStateReader().aliasesManager.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(alias, collectionName));
}
} catch (SolrException ex) {
@ -402,7 +411,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// but (for now) require that each core goes on a distinct node.
List<ReplicaPosition> replicaPositions;
List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, CollectionHandlingUtils.RANDOM);
if (nodeList.isEmpty()) {
log.warn("It is unusual to create a collection ({}) without cores.", collectionName);
@ -413,7 +422,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
log.warn("Specified number of replicas of {} on collection {} is higher than the number of Solr instances currently live or live and part of your {}({}). {}"
, totalNumReplicas
, collectionName
, OverseerCollectionMessageHandler.CREATE_NODE_SET
, CollectionHandlingUtils.CREATE_NODE_SET
, nodeList.size()
, "It's unusual to run two replica of the same slice on the same Solr-instance.");
}
@ -443,16 +452,16 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
public static List<String> populateShardNames(ZkNodeProps message, String router) {
List<String> shardNames = new ArrayList<>();
Integer numSlices = message.getInt(OverseerCollectionMessageHandler.NUM_SLICES, null);
Integer numSlices = message.getInt(CollectionHandlingUtils.NUM_SLICES, null);
if (ImplicitDocRouter.NAME.equals(router)) {
ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
numSlices = shardNames.size();
} else {
if (numSlices == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, OverseerCollectionMessageHandler.NUM_SLICES + " is a required param (when using CompositeId router).");
throw new SolrException(ErrorCode.BAD_REQUEST, CollectionHandlingUtils.NUM_SLICES + " is a required param (when using CompositeId router).");
}
if (numSlices <= 0) {
throw new SolrException(ErrorCode.BAD_REQUEST, OverseerCollectionMessageHandler.NUM_SLICES + " must be > 0");
throw new SolrException(ErrorCode.BAD_REQUEST, CollectionHandlingUtils.NUM_SLICES + " must be > 0");
}
ClusterStateMutator.getShardNames(numSlices, shardNames);
}
@ -466,7 +475,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// if there is only one conf, use that
List<String> configNames = null;
try {
configNames = ocmh.zkStateReader.getZkClient().getChildren(ZkConfigManager.CONFIGS_ZKNODE, null, true);
configNames = ccc.getZkStateReader().getZkClient().getChildren(ZkConfigManager.CONFIGS_ZKNODE, null, true);
if (configNames.contains(DEFAULT_CONFIGSET_NAME)) {
if (CollectionAdminParams.SYSTEM_COLL.equals(coll)) {
return coll;
@ -491,7 +500,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
* Copies the _default configset to the specified configset name (overwrites if pre-existing)
*/
private void copyDefaultConfigSetTo(List<String> configNames, String targetConfig) {
ZkConfigManager configManager = new ZkConfigManager(ocmh.zkStateReader.getZkClient());
ZkConfigManager configManager = new ZkConfigManager(ccc.getZkStateReader().getZkClient());
// if a configset named collection exists, re-use it
if (configNames.contains(targetConfig)) {

View File

@ -43,12 +43,12 @@ import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
public class CreateShardCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public CreateShardCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public CreateShardCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -65,7 +65,7 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
@ -79,19 +79,19 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// The message has been crafted by CollectionsHandler.CollectionOperation.CREATESHARD_OP and defines the QUEUE_OPERATION
// to be CollectionParams.CollectionAction.CREATESHARD.
// Likely a bug here (distributed or Overseer based) as we use the collection alias name and not the real name?
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionCreateShard, message,
ocmh.cloudManager, ocmh.zkStateReader);
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionCreateShard, message,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
// message contains extCollectionName that might be an alias. Unclear (to me) how this works in that case.
ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
ccc.offerStateUpdate(Utils.toJSON(message));
}
// wait for a while until we see the shard and update the local view of the cluster state
clusterState = ocmh.waitForNewShard(collectionName, sliceName);
clusterState = CollectionHandlingUtils.waitForNewShard(collectionName, sliceName, ccc.getZkStateReader());
String async = message.getStr(ASYNC);
ZkNodeProps addReplicasProps = new ZkNodeProps(
@ -100,17 +100,16 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.NRT_REPLICAS, String.valueOf(numNrtReplicas),
ZkStateReader.TLOG_REPLICAS, String.valueOf(numTlogReplicas),
ZkStateReader.PULL_REPLICAS, String.valueOf(numPullReplicas),
OverseerCollectionMessageHandler.CREATE_NODE_SET, message.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET),
CollectionHandlingUtils.CREATE_NODE_SET, message.getStr(CollectionHandlingUtils.CREATE_NODE_SET),
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
Map<String, Object> propertyParams = new HashMap<>();
ocmh.addPropertyParams(message, propertyParams);
CollectionHandlingUtils.addPropertyParams(message, propertyParams);
addReplicasProps = addReplicasProps.plus(propertyParams);
if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
final NamedList addResult = new NamedList();
try {
//ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, () -> {
ocmh.addReplica(clusterState, addReplicasProps, addResult, () -> {
new AddReplicaCmd(ccc).addReplica(clusterState, addReplicasProps, addResult, () -> {
Object addResultFailure = addResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
@ -131,7 +130,7 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
} catch (Assign.AssignmentException e) {
// clean up the slice that we created
ZkNodeProps deleteShard = new ZkNodeProps(COLLECTION_PROP, collectionName, SHARD_ID_PROP, sliceName, ASYNC, async);
new DeleteShardCmd(ocmh).call(clusterState, deleteShard, results);
new DeleteShardCmd(ccc).call(clusterState, deleteShard, results);
throw e;
}

View File

@ -31,7 +31,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
@ -56,12 +56,12 @@ import org.slf4j.LoggerFactory;
/**
* This class implements the functionality of creating a collection level snapshot.
*/
public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
public class CreateSnapshotCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public CreateSnapshotCmd (OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public CreateSnapshotCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -72,14 +72,14 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
String collectionName;
if (followAliases) {
collectionName = ocmh.zkStateReader.getAliases().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getZkStateReader().getAliases().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
String commitName = message.getStr(CoreAdminParams.COMMIT_NAME);
String asyncId = message.getStr(ASYNC);
SolrZkClient zkClient = ocmh.zkStateReader.getZkClient();
SolrZkClient zkClient = ccc.getZkStateReader().getZkClient();
Date creationDate = new Date();
if(SolrSnapshotManager.snapshotExists(zkClient, collectionName, commitName)) {
@ -96,10 +96,10 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList shardRequestResults = new NamedList();
Map<String, Slice> shardByCoreName = new HashMap<>();
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
for (Slice slice : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getSlices()) {
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (Slice slice : ccc.getZkStateReader().getClusterState().getCollection(collectionName).getSlices()) {
for (Replica replica : slice.getReplicas()) {
if (replica.getState() != State.ACTIVE) {
if (log.isInfoEnabled()) {
@ -169,7 +169,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
// Now that we know number of failures per shard, we can figure out
// if at-least one replica per shard was able to create a snapshot or not.
DocCollection collectionStatus = ocmh.zkStateReader.getClusterState().getCollection(collectionName);
DocCollection collectionStatus = ccc.getZkStateReader().getClusterState().getCollection(collectionName);
for (Map.Entry<String,Integer> entry : failuresByShardId.entrySet()) {
int replicaCount = collectionStatus.getSlice(entry.getKey()).getReplicas().size();
if (replicaCount <= entry.getValue()) {

View File

@ -25,18 +25,18 @@ import org.apache.solr.common.util.NamedList;
import static org.apache.solr.common.params.CommonParams.NAME;
public class DeleteAliasCmd implements OverseerCollectionMessageHandler.Cmd {
private final OverseerCollectionMessageHandler ocmh;
public class DeleteAliasCmd implements CollApiCmds.CollectionApiCommand {
private final CollectionCommandContext ccc;
public DeleteAliasCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteAliasCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String aliasName = message.getStr(NAME);
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
zkStateReader.aliasesManager.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(aliasName, null));
}

View File

@ -60,13 +60,13 @@ import static org.apache.solr.core.backup.BackupManager.START_TIME_PROP;
* The deletion can either delete a specific {@link BackupId}, delete everything except the most recent N backup
* points, or can be used to trigger a "garbage collection" of unused index files in the backup repository.
*/
public class DeleteBackupCmd implements OverseerCollectionMessageHandler.Cmd {
public class DeleteBackupCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
DeleteBackupCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteBackupCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -81,7 +81,7 @@ public class DeleteBackupCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, String.format(Locale.ROOT, "%s, %s or %s param must be provided", CoreAdminParams.BACKUP_ID, CoreAdminParams.MAX_NUM_BACKUP_POINTS,
CoreAdminParams.BACKUP_PURGE_UNUSED));
}
CoreContainer cc = ocmh.overseer.getCoreContainer();
CoreContainer cc = ccc.getCoreContainer();
try (BackupRepository repository = cc.newBackupRepository(repo)) {
URI location = repository.createURI(backupLocation);
final URI backupPath = BackupFilePaths.buildExistingBackupLocationURI(repository, location, backupName);

View File

@ -57,15 +57,15 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.DE
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
public class DeleteCollectionCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final Set<String> okayExceptions = Collections.singleton(NonExistentCoreException.class.getName());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public DeleteCollectionCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteCollectionCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -75,7 +75,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
((Runnable)o).run(); // this will ensure the collection is removed from the alias before it disappears.
}
final String extCollection = message.getStr(NAME);
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
if (zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
zkStateReader.aliasesManager.update(); // aliases may have been stale; get latest from ZK
@ -96,8 +96,8 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// verify the placement modifications caused by the deletion are allowed
DocCollection coll = state.getCollectionOrNull(collection);
if (coll != null) {
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.overseer.getCoreContainer(), state, coll);
assignStrategy.verifyDeleteCollection(ocmh.cloudManager, coll);
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer(), state, coll);
assignStrategy.verifyDeleteCollection(ccc.getSolrCloudManager(), coll);
}
final boolean deleteHistory = message.getBool(CoreAdminParams.DELETE_METRICS_HISTORY, true);
@ -119,7 +119,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
// remove collection-level metrics history
if (deleteHistory) {
MetricsHistoryHandler historyHandler = ocmh.overseer.getCoreContainer().getMetricsHistoryHandler();
MetricsHistoryHandler historyHandler = ccc.getCoreContainer().getMetricsHistoryHandler();
if (historyHandler != null) {
String registry = SolrMetricManager.getRegistryName(SolrInfoBean.Group.collection, collection);
historyHandler.removeHistory(registry);
@ -136,7 +136,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
ZkNodeProps internalMsg = message.plus(NAME, collection);
@SuppressWarnings({"unchecked"})
List<Replica> failedReplicas = ocmh.collectionCmd(internalMsg, params, results, null, asyncId, okayExceptions);
List<Replica> failedReplicas = CollectionHandlingUtils.collectionCmd(internalMsg, params, results, null, asyncId, okayExceptions, ccc, state);
for (Replica failedReplica : failedReplicas) {
boolean isSharedFS = failedReplica.getBool(ZkStateReader.SHARED_STORAGE_PROP, false) && failedReplica.get("dataDir") != null;
if (isSharedFS) {
@ -148,11 +148,11 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETE.toLower(), NAME, collection);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ClusterDeleteCollection, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ClusterDeleteCollection, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
// wait for a while until we don't see the collection
@ -160,7 +160,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// we can delete any remaining unique aliases
if (!aliasReferences.isEmpty()) {
ocmh.zkStateReader.aliasesManager.applyModificationAndExportToZk(a -> {
ccc.getZkStateReader().aliasesManager.applyModificationAndExportToZk(a -> {
for (String alias : aliasReferences) {
a = a.cloneWithCollectionAlias(alias, null);
}

View File

@ -39,29 +39,28 @@ import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
public class DeleteNodeCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public DeleteNodeCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteNodeCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@SuppressWarnings({"unchecked"})
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ocmh.checkRequired(message, "node");
CollectionHandlingUtils.checkRequired(message, "node");
String node = message.getStr("node");
List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
List<String> singleReplicas = verifyReplicaAvailability(sourceReplicas, state);
if (!singleReplicas.isEmpty()) {
results.add("failure", "Can't delete the only existing non-PULL replica(s) on node " + node + ": " + singleReplicas.toString());
} else {
cleanupReplicas(results, state, sourceReplicas, ocmh, node, message.getStr(ASYNC));
cleanupReplicas(results, state, sourceReplicas, ccc, node, message.getStr(ASYNC));
}
}
@ -97,7 +96,7 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
static void cleanupReplicas(@SuppressWarnings({"rawtypes"})NamedList results,
ClusterState clusterState,
List<ZkNodeProps> sourceReplicas,
OverseerCollectionMessageHandler ocmh,
CollectionCommandContext ccc,
String node,
String async) throws IOException, InterruptedException {
CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());
@ -110,7 +109,7 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
NamedList deleteResult = new NamedList();
try {
if (async != null) sourceReplica = sourceReplica.plus(ASYNC, async);
((DeleteReplicaCmd)ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult, () -> {
new DeleteReplicaCmd(ccc).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult, () -> {
cleanupLatch.countDown();
if (deleteResult.get("failure") != null) {
synchronized (results) {

View File

@ -34,8 +34,8 @@ import java.util.Set;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollApiCmds.CollectionApiCommand;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@ -54,12 +54,12 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DeleteReplicaCmd implements Cmd {
public class DeleteReplicaCmd implements CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public DeleteReplicaCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteReplicaCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -84,7 +84,7 @@ public class DeleteReplicaCmd implements Cmd {
}
ocmh.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP);
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP);
String extCollectionName = message.getStr(COLLECTION_PROP);
String shard = message.getStr(SHARD_ID_PROP);
String replicaName = message.getStr(REPLICA_PROP);
@ -92,7 +92,7 @@ public class DeleteReplicaCmd implements Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
@ -118,7 +118,7 @@ public class DeleteReplicaCmd implements Cmd {
Runnable onComplete,
boolean parallel)
throws KeeperException, IOException, InterruptedException {
ocmh.checkRequired(message, COLLECTION_PROP, COUNT_PROP);
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, COUNT_PROP);
int count = Integer.parseInt(message.getStr(COUNT_PROP));
String collectionName = message.getStr(COLLECTION_PROP);
String shard = message.getStr(SHARD_ID_PROP);
@ -148,14 +148,14 @@ public class DeleteReplicaCmd implements Cmd {
}
// verify that all replicas can be deleted
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.overseer.getCoreContainer(), clusterState, coll);
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer(), clusterState, coll);
for (Map.Entry<Slice, Set<String>> entry : shardToReplicasMapping.entrySet()) {
Slice shardSlice = entry.getKey();
String shardId = shardSlice.getName();
Set<String> replicaNames = entry.getValue();
Set<Replica> replicas = replicaNames.stream()
.map(name -> shardSlice.getReplica(name)).collect(Collectors.toSet());
assignStrategy.verifyDeleteReplicas(ocmh.cloudManager, coll, shardId, replicas);
assignStrategy.verifyDeleteReplicas(ccc.getSolrCloudManager(), coll, shardId, replicas);
}
for (Map.Entry<Slice, Set<String>> entry : shardToReplicasMapping.entrySet()) {
@ -245,7 +245,7 @@ public class DeleteReplicaCmd implements Cmd {
// If users are being safe and only want to remove a shard if it is down, they can specify onlyIfDown=true
// on the command.
if (Boolean.parseBoolean(message.getStr(OverseerCollectionMessageHandler.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
if (Boolean.parseBoolean(message.getStr(CollectionHandlingUtils.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Attempted to remove replica : " + coll.getName() + "/" + shardId + "/" + replicaName +
" with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'");
@ -253,11 +253,11 @@ public class DeleteReplicaCmd implements Cmd {
// verify that we are allowed to delete this replica
if (verifyPlacement) {
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.overseer.getCoreContainer(), clusterState, coll);
assignStrategy.verifyDeleteReplicas(ocmh.cloudManager, coll, shardId, Set.of(replica));
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer(), clusterState, coll);
assignStrategy.verifyDeleteReplicas(ccc.getSolrCloudManager(), coll, shardId, Set.of(replica));
}
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String asyncId = message.getStr(ASYNC);
@ -270,8 +270,8 @@ public class DeleteReplicaCmd implements Cmd {
params.set(CoreAdminParams.DELETE_DATA_DIR, message.getBool(CoreAdminParams.DELETE_DATA_DIR, true));
params.set(CoreAdminParams.DELETE_METRICS_HISTORY, message.getBool(CoreAdminParams.DELETE_METRICS_HISTORY, true));
boolean isLive = ocmh.zkStateReader.getClusterState().getLiveNodes().contains(replica.getNodeName());
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
boolean isLive = ccc.getZkStateReader().getClusterState().getLiveNodes().contains(replica.getNodeName());
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
if (isLive) {
shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
}
@ -282,12 +282,12 @@ public class DeleteReplicaCmd implements Cmd {
shardRequestTracker.processResponses(results, shardHandler, false, null);
//check if the core unload removed the corenode zk entry
if (ocmh.waitForCoreNodeGone(coll.getName(), shardId, replicaName, 30000)) return Boolean.TRUE;
if (CollectionHandlingUtils.waitForCoreNodeGone(coll.getName(), shardId, replicaName, 30000, ccc.getZkStateReader())) return Boolean.TRUE;
}
// try and ensure core info is removed from cluster state
ocmh.deleteCoreNode(coll.getName(), replicaName, replica, core);
if (ocmh.waitForCoreNodeGone(coll.getName(), shardId, replicaName, 30000)) return Boolean.TRUE;
CollectionHandlingUtils.deleteCoreNode(coll.getName(), replicaName, replica, core, ccc);
if (CollectionHandlingUtils.waitForCoreNodeGone(coll.getName(), shardId, replicaName, 30000, ccc.getZkStateReader())) return Boolean.TRUE;
return Boolean.FALSE;
} catch (Exception e) {
results.add("failure", "Could not complete delete " + e.getMessage());
@ -309,7 +309,7 @@ public class DeleteReplicaCmd implements Cmd {
}
} else {
ocmh.tpe.submit(callable);
ccc.getExecutorService().submit(callable);
}
}

View File

@ -21,7 +21,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@ -46,20 +45,18 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
public class DeleteShardCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
public DeleteShardCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
private final CollectionCommandContext ccc;
public DeleteShardCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -71,7 +68,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
@ -98,7 +95,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(sliceId, Slice.State.CONSTRUCTION.toString());
propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
ZkNodeProps m = new ZkNodeProps(propMap);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// In this DeleteShardCmd.call() method there are potentially two cluster state updates. This is the first one.
// Even though the code of this method does not wait for it to complete, it does call the Collection API before
// it issues the second state change below. The collection API will be doing its own state change(s), and these will
@ -106,10 +103,10 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
// immediately and do not group it with the one done further down.
// Once the Collection API is also distributed (and not only the cluster state updates), we will likely be able
// to batch more/all cluster state updates done by this command (DeleteShardCmd). TODO SOLR-15146
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ocmh.cloudManager, ocmh.zkStateReader);
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
@ -126,7 +123,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList deleteResult = new NamedList();
try {
((DeleteReplicaCmd)ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, replica, deleteResult, () -> {
new DeleteReplicaCmd(ccc).deleteReplica(clusterState, replica, deleteResult, () -> {
cleanupLatch.countDown();
if (deleteResult.get("failure") != null) {
synchronized (results) {
@ -156,12 +153,12 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP,
collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
ZkStateReader zkStateReader = ocmh.zkStateReader;
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionDeleteShard, m,
ocmh.cloudManager, ocmh.zkStateReader);
ZkStateReader zkStateReader = ccc.getZkStateReader();
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionDeleteShard, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
zkStateReader.waitForState(collectionName, 45, TimeUnit.SECONDS, (c) -> c.getSlice(sliceId) == null);

View File

@ -29,7 +29,7 @@ import java.util.List;
import java.util.Optional;
import java.util.Set;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
@ -54,12 +54,12 @@ import org.slf4j.LoggerFactory;
/**
* This class implements the functionality of deleting a collection level snapshot.
*/
public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
public class DeleteSnapshotCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public DeleteSnapshotCmd (OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public DeleteSnapshotCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -69,7 +69,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.zkStateReader.getAliases().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getZkStateReader().getAliases().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
@ -77,8 +77,8 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
String asyncId = message.getStr(ASYNC);
@SuppressWarnings({"rawtypes"})
NamedList shardRequestResults = new NamedList();
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
SolrZkClient zkClient = ocmh.zkStateReader.getZkClient();
ShardHandler shardHandler = ccc.getShardHandler();
SolrZkClient zkClient = ccc.getZkStateReader().getZkClient();
Optional<CollectionSnapshotMetaData> meta = SolrSnapshotManager.getCollectionLevelSnapshot(zkClient, collectionName, commitName);
if (!meta.isPresent()) { // Snapshot not found. Nothing to do.
@ -88,7 +88,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Deleting a snapshot for collection={} with commitName={}", collectionName, commitName);
Set<String> existingCores = new HashSet<>();
for (Slice s : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getSlices()) {
for (Slice s : ccc.getZkStateReader().getClusterState().getCollection(collectionName).getSlices()) {
for (Replica r : s.getReplicas()) {
existingCores.add(r.getCoreName());
}
@ -101,9 +101,9 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
log.info("Existing cores with snapshot for collection={} are {}", collectionName, existingCores);
for (Slice slice : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getSlices()) {
for (Slice slice : ccc.getZkStateReader().getClusterState().getCollection(collectionName).getSlices()) {
for (Replica replica : slice.getReplicas()) {
if (replica.getState() == State.DOWN) {
continue; // Since replica is down - no point sending a request.

View File

@ -46,8 +46,8 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
static final String INVOKED_BY_ROUTED_ALIAS = "invokedByRoutedAlias";
static final String ROUTED_ALIAS_TARGET_COL = "routedAliasTargetCol";
MaintainRoutedAliasCmd(OverseerCollectionMessageHandler ocmh) {
super(ocmh);
MaintainRoutedAliasCmd(CollectionCommandContext ccc) {
super(ccc);
}
/**
@ -105,7 +105,7 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
final String aliasName = message.getStr(NAME);
final String routeValue = message.getStr(ROUTED_ALIAS_TARGET_COL);
final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager;
final ZkStateReader.AliasesManager aliasesManager = ccc.getZkStateReader().aliasesManager;
final Aliases aliases = aliasesManager.getAliases();
final Map<String, String> aliasMetadata = aliases.getCollectionAliasProperties(aliasName);
if (aliasMetadata.isEmpty()) {
@ -117,14 +117,14 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "MaintainRoutedAlias called on non-routed alias");
}
ra.updateParsedCollectionAliases(ocmh.zkStateReader, true);
ra.updateParsedCollectionAliases(ccc.getZkStateReader(), true);
List<RoutedAlias.Action> actions = ra.calculateActions(routeValue);
for (RoutedAlias.Action action : actions) {
boolean exists = ocmh.zkStateReader.getClusterState().getCollectionOrNull(action.targetCollection) != null;
boolean exists = ccc.getZkStateReader().getClusterState().getCollectionOrNull(action.targetCollection) != null;
switch (action.actionType) {
case ENSURE_REMOVED:
if (exists) {
ocmh.tpe.submit(() -> {
ccc.getExecutorService().submit(() -> {
try {
deleteTargetCollection(clusterState, results, aliasName, aliasesManager, action);
} catch (Exception e) {
@ -147,10 +147,10 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
// take additional work (though presently they might work if the below book keeping is done by hand)
if (!ra.getCollectionList(aliases).contains(action.targetCollection)) {
addCollectionToAlias(aliasName, aliasesManager, action.targetCollection);
Map<String, String> collectionProperties = ocmh.zkStateReader
Map<String, String> collectionProperties = ccc.getZkStateReader()
.getCollectionProperties(action.targetCollection, 1000);
if (!collectionProperties.containsKey(RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP)) {
CollectionProperties props = new CollectionProperties(ocmh.zkStateReader.getZkClient());
CollectionProperties props = new CollectionProperties(ccc.getZkStateReader().getZkClient());
props.setCollectionProperty(action.targetCollection, RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, aliasName);
}
}
@ -166,7 +166,7 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
public void addTargetCollection(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String aliasName, ZkStateReader.AliasesManager aliasesManager, Map<String, String> aliasMetadata, RoutedAlias.Action action) throws Exception {
@SuppressWarnings({"rawtypes"})
NamedList createResults = createCollectionAndWait(clusterState, aliasName, aliasMetadata,
action.targetCollection, ocmh);
action.targetCollection, ccc);
if (createResults != null) {
results.add("create", createResults);
}
@ -179,6 +179,6 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
(Runnable) () -> removeCollectionFromAlias(aliasName, aliasesManager, action.targetCollection));
delProps.put(NAME, action.targetCollection);
ZkNodeProps messageDelete = new ZkNodeProps(delProps);
new DeleteCollectionCmd(ocmh).call(clusterState, messageDelete, results);
new DeleteCollectionCmd(ccc).call(clusterState, messageDelete, results);
}
}

View File

@ -24,9 +24,9 @@ import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@ -42,10 +42,8 @@ import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.TimeOut;
import org.slf4j.Logger;
@ -62,14 +60,12 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.Utils.makeMap;
public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
private final CollectionCommandContext ccc;
public MigrateCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
public MigrateCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@ -84,8 +80,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
String sourceCollectionName;
String targetCollectionName;
if (followAliases) {
sourceCollectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extSourceCollectionName);
targetCollectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extTargetCollectionName);
sourceCollectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extSourceCollectionName);
targetCollectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extTargetCollectionName);
} else {
sourceCollectionName = extSourceCollectionName;
targetCollectionName = extTargetCollectionName;
@ -142,7 +138,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
String splitKey, int timeout,
@SuppressWarnings({"rawtypes"})NamedList results, String asyncId, ZkNodeProps message) throws Exception {
String tempSourceCollectionName = "split_" + sourceSlice.getName() + "_temp_" + targetSlice.getName();
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
if (clusterState.hasCollection(tempSourceCollectionName)) {
log.info("Deleting temporary collection: {}", tempSourceCollectionName);
Map<String, Object> props = makeMap(
@ -150,7 +146,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
NAME, tempSourceCollectionName);
try {
ocmh.commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
new DeleteCollectionCmd(ccc).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
clusterState = zkStateReader.getClusterState();
} catch (Exception e) {
log.warn("Unable to clean up existing temporary collection: {}", tempSourceCollectionName, e);
@ -160,13 +156,12 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
CompositeIdRouter sourceRouter = (CompositeIdRouter) sourceCollection.getRouter();
DocRouter.Range keyHashRange = sourceRouter.keyHashRange(splitKey);
ShardHandlerFactory shardHandlerFactory = ocmh.shardHandlerFactory;
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
log.info("Hash range for split.key: {} is: {}", splitKey, keyHashRange);
// intersect source range, keyHashRange and target range
// this is the range that has to be split from source and transferred to target
DocRouter.Range splitRange = ocmh.intersect(targetSlice.getRange(), ocmh.intersect(sourceSlice.getRange(), keyHashRange));
DocRouter.Range splitRange = intersect(targetSlice.getRange(), intersect(sourceSlice.getRange(), keyHashRange));
if (splitRange == null) {
if (log.isInfoEnabled()) {
log.info("No common hashes between source shard: {} and target shard: {}", sourceSlice.getName(), targetSlice.getName());
@ -188,7 +183,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
params.set(CoreAdminParams.NAME, targetLeader.getStr("core"));
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to request node to buffer updates");
@ -202,16 +197,16 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
"targetCollection", targetCollection.getName(),
"expireAt", RoutingRule.makeExpiryAt(timeout));
log.info("Adding routing rule: {}", m);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceAddRoutingRule, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceAddRoutingRule, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
// wait for a while until we see the new rule
log.info("Waiting to see routing rule updated in clusterstate");
TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS, timeSource);
TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS, ccc.getSolrCloudManager().getTimeSource());
boolean added = false;
while (!waitUntil.hasTimedOut()) {
waitUntil.sleep(100);
@ -241,24 +236,24 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
Overseer.QUEUE_OPERATION, CREATE.toLower(),
NAME, tempSourceCollectionName,
NRT_REPLICAS, 1,
OverseerCollectionMessageHandler.NUM_SLICES, 1,
CollectionHandlingUtils.NUM_SLICES, 1,
CollectionAdminParams.COLL_CONF, configName,
OverseerCollectionMessageHandler.CREATE_NODE_SET, sourceLeader.getNodeName());
CollectionHandlingUtils.CREATE_NODE_SET, sourceLeader.getNodeName());
if (asyncId != null) {
String internalAsyncId = asyncId + Math.abs(System.nanoTime());
props.put(ASYNC, internalAsyncId);
}
log.info("Creating temporary collection: {}", props);
ocmh.commandMap.get(CREATE).call(clusterState, new ZkNodeProps(props), results);
new CreateCollectionCmd(ccc).call(clusterState, new ZkNodeProps(props), results);
// refresh cluster state
clusterState = zkStateReader.getClusterState();
Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next();
Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000);
String tempCollectionReplica1 = tempSourceLeader.getCoreName();
String coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
sourceLeader.getNodeName(), tempCollectionReplica1);
String coreNodeName = CollectionHandlingUtils.waitForCoreNodeName(tempSourceCollectionName,
sourceLeader.getNodeName(), tempCollectionReplica1, ccc.getZkStateReader());
// wait for the replicas to be seen as active on temp source leader
if (log.isInfoEnabled()) {
log.info("Asking source leader to wait for: {} to be alive on: {}", tempCollectionReplica1, sourceLeader.getNodeName());
@ -271,7 +266,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
cmd.setCheckLive(true);
cmd.setOnlyIfLeader(true);
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
final ShardRequestTracker syncRequestTracker = CollectionHandlingUtils.syncRequestTracker(ccc);
// we don't want this to happen asynchronously
syncRequestTracker.sendShardRequest(tempSourceLeader.getNodeName(), new ModifiableSolrParams(cmd.getParams()),
shardHandler);
@ -292,7 +287,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
String tempNodeName = sourceLeader.getNodeName();
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(tempNodeName, params, shardHandler);
shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to invoke SPLIT core admin command");
}
@ -300,7 +295,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
tempSourceCollectionName, targetLeader.getNodeName());
}
String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
String tempCollectionReplica2 = Assign.buildSolrCoreName(ccc.getSolrCloudManager().getDistribStateManager(),
zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
@ -318,15 +313,15 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
props.put(ASYNC, asyncId);
}
((AddReplicaCmd)ocmh.commandMap.get(ADDREPLICA)).addReplica(clusterState, new ZkNodeProps(props), results, null);
new AddReplicaCmd(ccc).addReplica(clusterState, new ZkNodeProps(props), results, null);
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
final ShardRequestTracker syncRequestTracker = CollectionHandlingUtils.syncRequestTracker(ccc);
syncRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to create replica of " +
"temporary collection in target leader node.");
}
coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
targetLeader.getNodeName(), tempCollectionReplica2);
coreNodeName = CollectionHandlingUtils.waitForCoreNodeName(tempSourceCollectionName,
targetLeader.getNodeName(), tempCollectionReplica2, ccc.getZkStateReader());
// wait for the replicas to be seen as active on temp source leader
if (log.isInfoEnabled()) {
log.info("Asking temp source leader to wait for: {} to be alive on: {}", tempCollectionReplica2, targetLeader.getNodeName());
@ -341,7 +336,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
params = new ModifiableSolrParams(cmd.getParams());
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(tempSourceLeader.getNodeName(), params, shardHandler);
shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to create temp collection" +
@ -356,7 +351,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
params.set(CoreAdminParams.SRC_CORE, tempCollectionReplica2);
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
String msg = "MIGRATE failed to merge " + tempCollectionReplica2 + " to "
@ -369,7 +364,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
params.set(CoreAdminParams.NAME, targetLeader.getStr("core"));
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to request node to apply buffered updates");
}
@ -378,9 +373,23 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
props = makeMap(
Overseer.QUEUE_OPERATION, DELETE.toLower(),
NAME, tempSourceCollectionName);
ocmh.commandMap.get(DELETE). call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
new DeleteCollectionCmd(ccc).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
} catch (Exception e) {
log.error("Unable to delete temporary collection: {}. Please remove it manually", tempSourceCollectionName, e);
}
}
DocRouter.Range intersect(DocRouter.Range a, DocRouter.Range b) {
if (a == null || b == null || !a.overlaps(b)) {
return null;
} else if (a.isSubsetOf(b))
return a;
else if (b.isSubsetOf(a))
return b;
else if (b.includes(a.max)) {
return new DocRouter.Range(b.min, a.max);
} else {
return new DocRouter.Range(a.min, b.max);
}
}
}

View File

@ -36,14 +36,13 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.TimeOut;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
@ -53,27 +52,25 @@ import static org.apache.solr.common.params.CommonAdminParams.IN_PLACE_MOVE;
import static org.apache.solr.common.params.CommonAdminParams.TIMEOUT;
import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public class MoveReplicaCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
private final CollectionCommandContext ccc;
public MoveReplicaCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
public MoveReplicaCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
moveReplica(ocmh.zkStateReader.getClusterState(), message, results);
moveReplica(ccc.getZkStateReader().getClusterState(), message, results);
}
private void moveReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
if (log.isDebugEnabled()) {
log.debug("moveReplica() : {}", Utils.toJSONString(message));
}
ocmh.checkRequired(message, COLLECTION_PROP, CollectionParams.TARGET_NODE);
CollectionHandlingUtils.checkRequired(message, COLLECTION_PROP, CollectionParams.TARGET_NODE);
String extCollection = message.getStr(COLLECTION_PROP);
String targetNode = message.getStr(CollectionParams.TARGET_NODE);
boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
@ -85,7 +82,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collection;
if (followAliases) {
collection = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollection);
collection = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollection);
} else {
collection = extCollection;
}
@ -121,7 +118,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
}
Collections.shuffle(sliceReplicas, OverseerCollectionMessageHandler.RANDOM);
Collections.shuffle(sliceReplicas, CollectionHandlingUtils.RANDOM);
replica = sliceReplicas.iterator().next();
}
@ -162,7 +159,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList deleteResult = new NamedList();
try {
ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
new DeleteReplicaCmd(ccc).deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
} catch (SolrException e) {
// assume this failed completely so there's nothing to roll back
deleteResult.add("failure", e.toString());
@ -175,9 +172,9 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
return;
}
TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS, timeSource);
TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS, ccc.getSolrCloudManager().getTimeSource());
while (!timeOut.hasTimedOut()) {
coll = ocmh.zkStateReader.getClusterState().getCollection(coll.getName());
coll = ccc.getZkStateReader().getClusterState().getCollection(coll.getName());
if (coll.getReplica(replica.getName()) != null) {
timeOut.sleep(100);
} else {
@ -208,7 +205,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList addResult = new NamedList();
try {
ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, addResult, null);
new AddReplicaCmd(ccc).addReplica(ccc.getZkStateReader().getClusterState(), addReplicasProps, addResult, null);
} catch (Exception e) {
// fatal error - try rolling back
String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
@ -218,7 +215,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
@SuppressWarnings({"rawtypes"})
NamedList rollback = new NamedList();
ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
new AddReplicaCmd(ccc).addReplica(ccc.getZkStateReader().getClusterState(), addReplicasProps, rollback, null);
if (rollback.get("failure") != null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
+ ", collection may be inconsistent: " + rollback.get("failure"));
@ -236,7 +233,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList rollback = new NamedList();
try {
ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
new AddReplicaCmd(ccc).addReplica(ccc.getZkStateReader().getClusterState(), addReplicasProps, rollback, null);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
+ ", collection may be inconsistent!", e);
@ -256,7 +253,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"unchecked"})
private void moveNormalReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String targetNode, String async,
DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
String newCoreName = Assign.buildSolrCoreName(ccc.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, coll.getName(),
SHARD_ID_PROP, slice.getName(),
@ -267,14 +264,14 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
@SuppressWarnings({"rawtypes"})
NamedList addResult = new NamedList();
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(1, ocmh);
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(1, ccc.getCloseableToLatchOn());
ActiveReplicaWatcher watcher = null;
ZkNodeProps props = ocmh.addReplica(clusterState, addReplicasProps, addResult, null).get(0);
ZkNodeProps props = new AddReplicaCmd(ccc).addReplica(clusterState, addReplicasProps, addResult, null).get(0);
log.debug("props {}", props);
if (replica.equals(slice.getLeader()) || waitForFinalState) {
watcher = new ActiveReplicaWatcher(coll.getName(), null, Collections.singletonList(newCoreName), countDownLatch);
log.debug("-- registered watcher {}", watcher);
ocmh.zkStateReader.registerCollectionStateWatcher(coll.getName(), watcher);
ccc.getZkStateReader().registerCollectionStateWatcher(coll.getName(), watcher);
}
if (addResult.get("failure") != null) {
String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
@ -282,7 +279,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
log.warn(errorString);
results.add("failure", errorString);
if (watcher != null) { // unregister
ocmh.zkStateReader.removeCollectionStateWatcher(coll.getName(), watcher);
ccc.getZkStateReader().removeCollectionStateWatcher(coll.getName(), watcher);
}
return;
}
@ -302,7 +299,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
} finally {
ocmh.zkStateReader.removeCollectionStateWatcher(coll.getName(), watcher);
ccc.getZkStateReader().removeCollectionStateWatcher(coll.getName(), watcher);
}
}
@ -314,7 +311,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList deleteResult = new NamedList();
try {
ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
new DeleteReplicaCmd(ccc).deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
} catch (SolrException e) {
deleteResult.add("failure", e.toString());
}

View File

@ -0,0 +1,96 @@
/*
* 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.cloud.api.collections;
import java.util.concurrent.ExecutorService;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.zookeeper.KeeperException;
/**
* Context passed to Collection API commands when they execute in the Overseer.
*/
public class OcmhCollectionCommandContext implements CollectionCommandContext {
private final OverseerCollectionMessageHandler ocmh;
public OcmhCollectionCommandContext(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
}
@Override
public ShardHandler getShardHandler() {
return ocmh.shardHandlerFactory.getShardHandler();
}
@Override
public SolrCloudManager getSolrCloudManager() {
return ocmh.cloudManager;
}
@Override
public CoreContainer getCoreContainer() {
return ocmh.overseer.getCoreContainer();
}
@Override
public ZkStateReader getZkStateReader() {
return ocmh.zkStateReader;
}
@Override
public DistributedClusterStateUpdater getDistributedClusterStateUpdater() {
return ocmh.overseer.getDistributedClusterStateUpdater();
}
@Override
public void offerStateUpdate(byte[] data) throws KeeperException, InterruptedException {
ocmh.overseer.offerStateUpdate(data);
}
@Override
public SolrCloseable getCloseableToLatchOn() {
return ocmh;
}
@Override
public ExecutorService getExecutorService() {
return ocmh.tpe;
}
@Override
public String getOverseerId() {
return ocmh.myId;
}
@Override
public Stats getOverseerStats() {
return ocmh.stats;
}
@Override
public void submitIntraProcessMessage(Overseer.Message message) {
ocmh.overseer.submit(message);
}
}

View File

@ -17,130 +17,51 @@
package org.apache.solr.cloud.api.collections;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.LockTree;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.OverseerMessageHandler;
import org.apache.solr.cloud.OverseerNodePrioritizer;
import org.apache.solr.cloud.OverseerSolrResponse;
import org.apache.solr.cloud.Stats;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
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.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;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
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.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.backup.BackupId;
import org.apache.solr.core.backup.repository.BackupRepository;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.Utils.makeMap;
/**
* A {@link OverseerMessageHandler} that handles Collections API related
* overseer messages.
* A {@link OverseerMessageHandler} that handles Collections API related overseer messages.<p>
*
* A lot of the content that was in this class got moved to {@link CollectionHandlingUtils} and {@link CollApiCmds}.
*/
public class OverseerCollectionMessageHandler implements OverseerMessageHandler, SolrCloseable {
public static final String NUM_SLICES = "numShards";
public static final boolean CREATE_NODE_SET_SHUFFLE_DEFAULT = true;
public static final String CREATE_NODE_SET_SHUFFLE = CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
public static final String CREATE_NODE_SET_EMPTY = "EMPTY";
public static final String CREATE_NODE_SET = CollectionAdminParams.CREATE_NODE_SET_PARAM;
public static final String ROUTER = "router";
public static final String SHARDS_PROP = "shards";
public static final String REQUESTID = "requestid";
public static final String ONLY_IF_DOWN = "onlyIfDown";
public static final String SHARD_UNIQUE = "shardUnique";
public static final String ONLY_ACTIVE_NODES = "onlyactivenodes";
static final String SKIP_CREATE_REPLICA_IN_CLUSTER_STATE = "skipCreateReplicaInClusterState";
public static final Map<String, Object> COLLECTION_PROPS_AND_DEFAULTS = Collections.unmodifiableMap(makeMap(
ROUTER, DocRouter.DEFAULT_NAME,
ZkStateReader.REPLICATION_FACTOR, "1",
ZkStateReader.NRT_REPLICAS, "1",
ZkStateReader.TLOG_REPLICAS, "0",
DocCollection.PER_REPLICA_STATE, null,
ZkStateReader.PULL_REPLICAS, "0"));
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String FAILURE_FIELD = "failure";
public static final String SUCCESS_FIELD = "success";
Overseer overseer;
HttpShardHandlerFactory shardHandlerFactory;
@ -150,7 +71,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
String myId;
Stats stats;
TimeSource timeSource;
private final DistributedClusterStateUpdater distributedClusterStateUpdater;
private final CollectionCommandContext ccc;
// Set that tracks collections that are currently being processed by a running task.
// This is used for handling mutual exclusion of the tasks.
@ -160,19 +81,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
protected static final Random RANDOM;
static {
// We try to make things reproducible in the context of our tests by initializing the random instance
// based on the current seed
String seed = System.getProperty("tests.seed");
if (seed == null) {
RANDOM = new Random();
} else {
RANDOM = new Random(seed.hashCode());
}
}
final Map<CollectionAction, Cmd> commandMap;
final private Map<CollectionAction, CollApiCmds.CollectionApiCommand> commandMap;
private volatile boolean isClosed;
@ -188,45 +97,45 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
this.myId = myId;
this.stats = stats;
this.overseer = overseer;
this.distributedClusterStateUpdater = overseer.getDistributedClusterStateUpdater();
this.cloudManager = overseer.getSolrCloudManager();
this.timeSource = cloudManager.getTimeSource();
this.isClosed = false;
commandMap = new ImmutableMap.Builder<CollectionAction, Cmd>()
.put(REPLACENODE, new ReplaceNodeCmd(this))
.put(DELETENODE, new DeleteNodeCmd(this))
.put(BACKUP, new BackupCmd(this))
.put(RESTORE, new RestoreCmd(this))
.put(DELETEBACKUP, new DeleteBackupCmd(this))
.put(CREATESNAPSHOT, new CreateSnapshotCmd(this))
.put(DELETESNAPSHOT, new DeleteSnapshotCmd(this))
.put(SPLITSHARD, new SplitShardCmd(this))
.put(ADDROLE, new OverseerRoleCmd(this, ADDROLE, overseerPrioritizer))
.put(REMOVEROLE, new OverseerRoleCmd(this, REMOVEROLE, overseerPrioritizer))
.put(MOCK_COLL_TASK, this::mockOperation)
.put(MOCK_SHARD_TASK, this::mockOperation)
.put(MOCK_REPLICA_TASK, this::mockOperation)
.put(CREATESHARD, new CreateShardCmd(this))
.put(MIGRATE, new MigrateCmd(this))
.put(CREATE, new CreateCollectionCmd(this))
.put(MODIFYCOLLECTION, this::modifyCollection)
.put(ADDREPLICAPROP, this::processReplicaAddPropertyCommand)
.put(DELETEREPLICAPROP, this::processReplicaDeletePropertyCommand)
.put(BALANCESHARDUNIQUE, this::balanceProperty)
.put(REBALANCELEADERS, this::processRebalanceLeaders)
.put(RELOAD, this::reloadCollection)
.put(DELETE, new DeleteCollectionCmd(this))
.put(CREATEALIAS, new CreateAliasCmd(this))
.put(DELETEALIAS, new DeleteAliasCmd(this))
.put(ALIASPROP, new SetAliasPropCmd(this))
.put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(this))
.put(OVERSEERSTATUS, new OverseerStatusCmd(this))
.put(DELETESHARD, new DeleteShardCmd(this))
.put(DELETEREPLICA, new DeleteReplicaCmd(this))
.put(ADDREPLICA, new AddReplicaCmd(this))
.put(MOVEREPLICA, new MoveReplicaCmd(this))
.put(REINDEXCOLLECTION, new ReindexCollectionCmd(this))
.put(RENAME, new RenameCmd(this))
ccc = new OcmhCollectionCommandContext(this);
commandMap = new ImmutableMap.Builder<CollectionAction, CollApiCmds.CollectionApiCommand>()
.put(REPLACENODE, new ReplaceNodeCmd(ccc))
.put(DELETENODE, new DeleteNodeCmd(ccc))
.put(BACKUP, new BackupCmd(ccc))
.put(RESTORE, new RestoreCmd(ccc))
.put(DELETEBACKUP, new DeleteBackupCmd(ccc))
.put(CREATESNAPSHOT, new CreateSnapshotCmd(ccc))
.put(DELETESNAPSHOT, new DeleteSnapshotCmd(ccc))
.put(SPLITSHARD, new SplitShardCmd(ccc))
.put(ADDROLE, new OverseerRoleCmd(ccc, ADDROLE, overseerPrioritizer))
.put(REMOVEROLE, new OverseerRoleCmd(ccc, REMOVEROLE, overseerPrioritizer))
.put(MOCK_COLL_TASK, new CollApiCmds.MockOperationCmd())
.put(MOCK_SHARD_TASK, new CollApiCmds.MockOperationCmd())
.put(MOCK_REPLICA_TASK, new CollApiCmds.MockOperationCmd())
.put(CREATESHARD, new CreateShardCmd(ccc))
.put(MIGRATE, new MigrateCmd(ccc))
.put(CREATE, new CreateCollectionCmd(ccc))
.put(MODIFYCOLLECTION, new CollApiCmds.ModifyCollectionCmd(ccc))
.put(ADDREPLICAPROP, new CollApiCmds.AddReplicaPropCmd(ccc))
.put(DELETEREPLICAPROP, new CollApiCmds.DeleteReplicaPropCmd(ccc))
.put(BALANCESHARDUNIQUE, new CollApiCmds.BalanceShardsUniqueCmd(ccc))
.put(REBALANCELEADERS, new CollApiCmds.RebalanceLeadersCmd(ccc))
.put(RELOAD, new CollApiCmds.ReloadCollectionCmd(ccc))
.put(DELETE, new DeleteCollectionCmd(ccc))
.put(CREATEALIAS, new CreateAliasCmd(ccc))
.put(DELETEALIAS, new DeleteAliasCmd(ccc))
.put(ALIASPROP, new SetAliasPropCmd(ccc))
.put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(ccc))
.put(OVERSEERSTATUS, new OverseerStatusCmd(ccc))
.put(DELETESHARD, new DeleteShardCmd(ccc))
.put(DELETEREPLICA, new DeleteReplicaCmd(ccc))
.put(ADDREPLICA, new AddReplicaCmd(ccc))
.put(MOVEREPLICA, new MoveReplicaCmd(ccc))
.put(REINDEXCOLLECTION, new ReindexCollectionCmd(ccc))
.put(RENAME, new RenameCmd(ccc))
.build()
;
}
@ -243,7 +152,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
NamedList results = new NamedList();
try {
CollectionAction action = getCollectionAction(operation);
Cmd command = commandMap.get(action);
CollApiCmds.CollectionApiCommand command = commandMap.get(action);
if (command != null) {
command.call(cloudManager.getClusterStateProvider().getClusterState(), message, results);
} else {
@ -270,21 +179,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return new OverseerSolrResponse(results);
}
DistributedClusterStateUpdater getDistributedClusterStateUpdater() {
return distributedClusterStateUpdater;
}
@SuppressForbidden(reason = "Needs currentTimeMillis for mock requests")
@SuppressWarnings({"unchecked"})
private void mockOperation(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws InterruptedException {
//only for test purposes
Thread.sleep(message.getInt("sleep", 1));
if (log.isInfoEnabled()) {
log.info("MOCK_TASK_EXECUTED time {} data {}", System.currentTimeMillis(), Utils.toJSONString(message));
}
results.add("MOCK_FINISHED", System.currentTimeMillis());
}
private CollectionAction getCollectionAction(String operation) {
CollectionAction action = CollectionAction.get(operation);
if (action == null) {
@ -293,541 +187,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return action;
}
@SuppressWarnings({"unchecked"})
private void reloadCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString());
String asyncId = message.getStr(ASYNC);
collectionCmd(message, params, results, Replica.State.ACTIVE, asyncId, Collections.emptySet());
}
@SuppressWarnings("unchecked")
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, NODE_NAME_PROP, REJOIN_AT_HEAD_PROP);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
params.set(SHARD_ID_PROP, message.getStr(SHARD_ID_PROP));
params.set(REJOIN_AT_HEAD_PROP, message.getStr(REJOIN_AT_HEAD_PROP));
params.set(CoreAdminParams.ACTION, CoreAdminAction.REJOINLEADERELECTION.toString());
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(NODE_NAME_PROP, message.getStr(NODE_NAME_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...
params.set("qt", adminPath);
sreq.purpose = ShardRequest.PURPOSE_PRIVATE;
sreq.shards = new String[] {baseUrl};
sreq.actualShards = sreq.shards;
sreq.params = params;
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
shardHandler.submit(sreq, baseUrl, sreq.params);
}
@SuppressWarnings("unchecked")
private void processReplicaAddPropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP, PROPERTY_VALUE_PROP);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, ADDREPLICAPROP.toLower());
propMap.putAll(message.getProperties());
ZkNodeProps m = new ZkNodeProps(propMap);
if (distributedClusterStateUpdater.isDistributedStateUpdate()) {
distributedClusterStateUpdater.doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ReplicaAddReplicaProperty, m,
cloudManager, zkStateReader);
} else {
overseer.offerStateUpdate(Utils.toJSON(m));
}
}
private void processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, DELETEREPLICAPROP.toLower());
propMap.putAll(message.getProperties());
ZkNodeProps m = new ZkNodeProps(propMap);
if (distributedClusterStateUpdater.isDistributedStateUpdate()) {
distributedClusterStateUpdater.doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.ReplicaDeleteReplicaProperty, m,
cloudManager, zkStateReader);
} else {
overseer.offerStateUpdate(Utils.toJSON(m));
}
}
private void balanceProperty(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
"' parameters are required for the BALANCESHARDUNIQUE operation, no action taken");
}
Map<String, Object> m = new HashMap<>();
m.put(Overseer.QUEUE_OPERATION, BALANCESHARDUNIQUE.toLower());
m.putAll(message.getProperties());
if (distributedClusterStateUpdater.isDistributedStateUpdate()) {
distributedClusterStateUpdater.doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.BalanceShardsUnique, new ZkNodeProps(m),
cloudManager, zkStateReader);
} else {
overseer.offerStateUpdate(Utils.toJSON(m));
}
}
/**
* Get collection status from cluster state.
* Can return collection status by given shard name.
*
*
* @param collection collection map parsed from JSON-serialized {@link ClusterState}
* @param name collection name
* @param requestedShards a set of shards to be returned in the status.
* An empty or null values indicates <b>all</b> shards.
* @return map of collection properties
*/
@SuppressWarnings("unchecked")
private Map<String, Object> getCollectionStatus(Map<String, Object> collection, String name, Set<String> requestedShards) {
if (collection == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
}
if (requestedShards == null || requestedShards.isEmpty()) {
return collection;
} else {
Map<String, Object> shards = (Map<String, Object>) collection.get("shards");
Map<String, Object> selected = new HashMap<>();
for (String selectedShard : requestedShards) {
if (!shards.containsKey(selectedShard)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection: " + name + " shard: " + selectedShard + " not found");
}
selected.put(selectedShard, shards.get(selectedShard));
collection.put("shards", selected);
}
return collection;
}
}
@SuppressWarnings("unchecked")
void deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
throws Exception {
((DeleteReplicaCmd) commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, message, results, onComplete);
}
boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
try {
zkStateReader.waitForState(collectionName, timeoutms, TimeUnit.MILLISECONDS, (c) -> {
if (c == null)
return true;
Slice slice = c.getSlice(shard);
if(slice == null || slice.getReplica(replicaName) == null) {
return true;
}
return false;
});
} catch (TimeoutException e) {
return false;
}
return true;
}
void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws Exception {
ZkNodeProps m = new ZkNodeProps(
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
ZkStateReader.CORE_NAME_PROP, core,
ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.CORE_NODE_NAME_PROP, replicaName);
if (distributedClusterStateUpdater.isDistributedStateUpdate()) {
distributedClusterStateUpdater.doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceRemoveReplica, m,
cloudManager, zkStateReader);
} else {
overseer.offerStateUpdate(Utils.toJSON(m));
}
}
void checkRequired(ZkNodeProps message, String... props) {
for (String prop : props) {
if(message.get(prop) == null){
throw new SolrException(ErrorCode.BAD_REQUEST, StrUtils.join(Arrays.asList(props),',') +" are required params" );
}
}
}
void checkResults(String label, NamedList<Object> results, boolean failureIsFatal) throws SolrException {
Object failure = results.get("failure");
if (failure == null) {
failure = results.get("error");
}
if (failure != null) {
String msg = "Error: " + label + ": " + Utils.toJSONString(results);
if (failureIsFatal) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
} else {
log.error(msg);
}
}
}
@SuppressWarnings({"unchecked"})
void commit(@SuppressWarnings({"rawtypes"})NamedList results, String slice, Replica parentShardLeader) {
log.debug("Calling soft commit to make sub shard updates visible");
String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
// HttpShardHandler is hard coded to send a QueryRequest hence we go direct
// and we force open a searcher so that we have documents to show upon switching states
UpdateResponse updateResponse = null;
try {
updateResponse = softCommit(coreUrl);
processResponse(results, null, coreUrl, updateResponse, slice, Collections.emptySet());
} catch (Exception e) {
processResponse(results, e, coreUrl, updateResponse, slice, Collections.emptySet());
throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to call distrib softCommit on: " + coreUrl, e);
}
}
static UpdateResponse softCommit(String url) throws SolrServerException, IOException {
try (HttpSolrClient client = new HttpSolrClient.Builder(url)
.withConnectionTimeout(30000)
.withSocketTimeout(120000)
.build()) {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true, true);
return ureq.process(client);
}
}
String waitForCoreNodeName(String collectionName, String msgNodeName, String msgCore) {
try {
DocCollection collection = zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c ->
ClusterStateMutator.getAssignedCoreNodeName(c, msgNodeName, msgCore) != null
);
return ClusterStateMutator.getAssignedCoreNodeName(collection, msgNodeName, msgCore);
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
throw new SolrException(ErrorCode.SERVER_ERROR, "Failed waiting for coreNodeName", e);
}
}
ClusterState waitForNewShard(String collectionName, String sliceName) throws KeeperException, InterruptedException {
log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
try {
zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c -> {
return c != null && c.getSlice(sliceName) != null;
});
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
throw new SolrException(ErrorCode.SERVER_ERROR, "Failed waiting for new slice", e);
}
return zkStateReader.getClusterState();
}
DocRouter.Range intersect(DocRouter.Range a, DocRouter.Range b) {
if (a == null || b == null || !a.overlaps(b)) {
return null;
} else if (a.isSubsetOf(b))
return a;
else if (b.isSubsetOf(a))
return b;
else if (b.includes(a.max)) {
return new DocRouter.Range(b.min, a.max);
} else {
return new DocRouter.Range(a.min, b.max);
}
}
void addPropertyParams(ZkNodeProps message, ModifiableSolrParams params) {
// Now add the property.key=value pairs
for (String key : message.keySet()) {
if (key.startsWith(CollectionAdminParams.PROPERTY_PREFIX)) {
params.set(key, message.getStr(key));
}
}
}
void addPropertyParams(ZkNodeProps message, Map<String, Object> map) {
// Now add the property.key=value pairs
for (String key : message.keySet()) {
if (key.startsWith(CollectionAdminParams.PROPERTY_PREFIX)) {
map.put(key, message.getStr(key));
}
}
}
void modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
//the rest of the processing is based on writing cluster state properties
//remove the property here to avoid any errors down the pipeline due to this property appearing
String configName = (String) message.getProperties().remove(CollectionAdminParams.COLL_CONF);
if(configName != null) {
validateConfigOrThrowSolrException(configName);
createConfNode(cloudManager.getDistribStateManager(), configName, collectionName);
reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
}
if (distributedClusterStateUpdater.isDistributedStateUpdate()) {
// Apply the state update right away. The wait will still be useful for the change to be visible in the local cluster state (watchers have fired).
distributedClusterStateUpdater.doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, message,
cloudManager, zkStateReader);
} else {
overseer.offerStateUpdate(Utils.toJSON(message));
}
try {
zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS, c -> {
if (c == null) return false;
for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
String updateKey = updateEntry.getKey();
if (!updateKey.equals(ZkStateReader.COLLECTION_PROP)
&& !updateKey.equals(Overseer.QUEUE_OPERATION)
&& updateEntry.getValue() != null // handled below in a separate conditional
&& !updateEntry.getValue().equals(c.get(updateKey))) {
return false;
}
if (updateEntry.getValue() == null && c.containsKey(updateKey)) {
return false;
}
}
return true;
});
} catch (TimeoutException | InterruptedException e) {
SolrZkClient.checkInterrupted(e);
log.debug("modifyCollection(ClusterState={}, ZkNodeProps={}, NamedList={})", clusterState, message, results, e);
throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to modify collection", e);
}
// if switching to/from read-only mode reload the collection
if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
}
}
void cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
log.error("Cleaning up collection [{}].", collectionName);
Map<String, Object> props = makeMap(
Overseer.QUEUE_OPERATION, DELETE.toLower(),
NAME, collectionName);
commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
}
Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
assert coreNames.size() > 0;
Map<String, Replica> results = new ConcurrentHashMap<>();
long maxWait = Long.getLong("solr.waitToSeeReplicasInStateTimeoutSeconds", 120); // could be a big cluster
try {
zkStateReader.waitForState(collectionName, maxWait, TimeUnit.SECONDS, c -> {
if (c == null) return false;
// We write into a ConcurrentHashMap, which will be ok if called multiple times by multiple threads
c.getSlices().stream().flatMap(slice -> slice.getReplicas().stream())
.filter(r -> coreNames.contains(r.getCoreName())) // Only the elements that were asked for...
.forEach(r -> results.putIfAbsent(r.getCoreName(), r)); // ...get added to the map
log.debug("Expecting {} cores, found {}", coreNames, results);
return results.size() == coreNames.size();
});
} catch (TimeoutException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, e.getMessage(), e);
}
return results;
}
@SuppressWarnings({"rawtypes"})
void cleanBackup(BackupRepository repository, URI backupUri, BackupId backupId) throws Exception {
((DeleteBackupCmd)commandMap.get(DELETEBACKUP))
.deleteBackupIds(backupUri, repository, Collections.singleton(backupId), new NamedList());
}
void deleteBackup(BackupRepository repository, URI backupPath,
int maxNumBackup,
@SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
((DeleteBackupCmd)commandMap.get(DELETEBACKUP))
.keepNumberOfBackup(repository, backupPath, maxNumBackup, results);
}
List<ZkNodeProps> addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
throws Exception {
return ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
}
void validateConfigOrThrowSolrException(String configName) throws IOException, KeeperException, InterruptedException {
boolean isValid = cloudManager.getDistribStateManager().hasData(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
if(!isValid) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Can not find the specified config set: " + configName);
}
}
/**
* This doesn't validate the config (path) itself and is just responsible for creating the confNode.
* That check should be done before the config node is created.
*/
public static void createConfNode(DistribStateManager stateManager, String configName, String coll) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException {
if (configName != null) {
String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
log.debug("creating collections conf node {} ", collDir);
byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
if (stateManager.hasData(collDir)) {
stateManager.setData(collDir, data, -1);
} else {
stateManager.makePath(collDir, data, CreateMode.PERSISTENT, false);
}
} else {
throw new SolrException(ErrorCode.BAD_REQUEST,"Unable to get config name");
}
}
/**
* Send request to all replicas of a collection
* @return List of replicas which is not live for receiving the request
*/
List<Replica> collectionCmd(ZkNodeProps message, ModifiableSolrParams params,
NamedList<Object> results, Replica.State stateMatcher, String asyncId, Set<String> okayExceptions) {
log.info("Executing Collection Cmd={}, asyncId={}", params, asyncId);
String collectionName = message.getStr(NAME);
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
ClusterState clusterState = zkStateReader.getClusterState();
DocCollection coll = clusterState.getCollection(collectionName);
List<Replica> notLivesReplicas = new ArrayList<>();
final ShardRequestTracker shardRequestTracker = new ShardRequestTracker(asyncId);
for (Slice slice : coll.getSlices()) {
notLivesReplicas.addAll(shardRequestTracker.sliceCmd(clusterState, params, stateMatcher, slice, shardHandler));
}
shardRequestTracker.processResponses(results, shardHandler, false, null, okayExceptions);
return notLivesReplicas;
}
private void processResponse(NamedList<Object> results, ShardResponse srsp, Set<String> okayExceptions) {
Throwable e = srsp.getException();
String nodeName = srsp.getNodeName();
SolrResponse solrResponse = srsp.getSolrResponse();
String shard = srsp.getShard();
processResponse(results, e, nodeName, solrResponse, shard, okayExceptions);
}
@SuppressWarnings("deprecation")
private void processResponse(NamedList<Object> results, Throwable e, String nodeName, SolrResponse solrResponse, String shard, Set<String> okayExceptions) {
String rootThrowable = null;
if (e instanceof BaseHttpSolrClient.RemoteSolrException) {
rootThrowable = ((BaseHttpSolrClient.RemoteSolrException) e).getRootThrowable();
}
if (e != null && (rootThrowable == null || !okayExceptions.contains(rootThrowable))) {
log.error("Error from shard: {}", shard, e);
addFailure(results, nodeName, e.getClass().getName() + ":" + e.getMessage());
} else {
addSuccess(results, nodeName, solrResponse.getResponse());
}
}
@SuppressWarnings("unchecked")
private static void addFailure(NamedList<Object> results, String key, Object value) {
SimpleOrderedMap<Object> failure = (SimpleOrderedMap<Object>) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap<>();
results.add("failure", failure);
}
failure.add(key, value);
}
@SuppressWarnings("unchecked")
private static void addSuccess(NamedList<Object> results, String key, Object value) {
SimpleOrderedMap<Object> success = (SimpleOrderedMap<Object>) results.get("success");
if (success == null) {
success = new SimpleOrderedMap<>();
results.add("success", success);
}
success.add(key, value);
}
private NamedList<Object> waitForCoreAdminAsyncCallToComplete(String nodeName, String requestId) {
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminAction.REQUESTSTATUS.toString());
params.set(CoreAdminParams.REQUESTID, requestId);
int counter = 0;
ShardRequest sreq;
do {
sreq = new ShardRequest();
params.set("qt", adminPath);
sreq.purpose = 1;
String replica = zkStateReader.getBaseUrlForNodeName(nodeName);
sreq.shards = new String[] {replica};
sreq.actualShards = sreq.shards;
sreq.params = params;
shardHandler.submit(sreq, replica, sreq.params);
ShardResponse srsp;
do {
srsp = shardHandler.takeCompletedOrError();
if (srsp != null) {
NamedList<Object> results = new NamedList<>();
processResponse(results, srsp, Collections.emptySet());
if (srsp.getSolrResponse().getResponse() == null) {
NamedList<Object> response = new NamedList<>();
response.add("STATUS", "failed");
return response;
}
String r = (String) srsp.getSolrResponse().getResponse().get("STATUS");
if (r.equals("running")) {
log.debug("The task is still RUNNING, continuing to wait.");
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
continue;
} else if (r.equals("completed")) {
log.debug("The task is COMPLETED, returning");
return srsp.getSolrResponse().getResponse();
} else if (r.equals("failed")) {
// TODO: Improve this. Get more information.
log.debug("The task is FAILED, returning");
return srsp.getSolrResponse().getResponse();
} else if (r.equals("notfound")) {
log.debug("The task is notfound, retry");
if (counter++ < 5) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
break;
}
throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid status request for requestId: " + requestId + "" + srsp.getSolrResponse().getResponse().get("STATUS") +
"retried " + counter + "times");
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid status request " + srsp.getSolrResponse().getResponse().get("STATUS"));
}
}
} while (srsp != null);
} while(true);
}
@Override
public String getName() {
return "Overseer Collection Message Handler";
@ -844,7 +203,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
message.getStr(COLLECTION_PROP) : message.getStr(NAME);
}
// -1 is not a possible batchSessionId so -1 will force initialization of lockSession
private long sessionId = -1;
private LockTree.Session lockSession;
@ -873,7 +231,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
);
}
@Override
public void close() throws IOException {
this.isClosed = true;
@ -888,134 +245,4 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
public boolean isClosed() {
return isClosed;
}
protected interface Cmd {
void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception;
}
/*
* backward compatibility reasons, add the response with the async ID as top level.
* This can be removed in Solr 9
*/
@Deprecated
static boolean INCLUDE_TOP_LEVEL_RESPONSE = true;
public ShardRequestTracker syncRequestTracker() {
return new ShardRequestTracker(null);
}
public ShardRequestTracker asyncRequestTracker(String asyncId) {
return new ShardRequestTracker(asyncId);
}
public class ShardRequestTracker{
private final String asyncId;
private final NamedList<String> shardAsyncIdByNode = new NamedList<String>();
private ShardRequestTracker(String asyncId) {
this.asyncId = asyncId;
}
/**
* Send request to all replicas of a slice
* @return List of replicas which is not live for receiving the request
*/
public List<Replica> sliceCmd(ClusterState clusterState, ModifiableSolrParams params, Replica.State stateMatcher,
Slice slice, ShardHandler shardHandler) {
List<Replica> notLiveReplicas = new ArrayList<>();
for (Replica replica : slice.getReplicas()) {
if ((stateMatcher == null || Replica.State.getState(replica.getStr(ZkStateReader.STATE_PROP)) == stateMatcher)) {
if (clusterState.liveNodesContain(replica.getStr(ZkStateReader.NODE_NAME_PROP))) {
// For thread safety, only simple clone the ModifiableSolrParams
ModifiableSolrParams cloneParams = new ModifiableSolrParams();
cloneParams.add(params);
cloneParams.set(CoreAdminParams.CORE, replica.getStr(ZkStateReader.CORE_NAME_PROP));
sendShardRequest(replica.getStr(ZkStateReader.NODE_NAME_PROP), cloneParams, shardHandler);
} else {
notLiveReplicas.add(replica);
}
}
}
return notLiveReplicas;
}
public void sendShardRequest(String nodeName, ModifiableSolrParams params,
ShardHandler shardHandler) {
sendShardRequest(nodeName, params, shardHandler, adminPath, zkStateReader);
}
public void sendShardRequest(String nodeName, ModifiableSolrParams params, ShardHandler shardHandler,
String adminPath, ZkStateReader zkStateReader) {
if (asyncId != null) {
String coreAdminAsyncId = asyncId + Math.abs(System.nanoTime());
params.set(ASYNC, coreAdminAsyncId);
track(nodeName, coreAdminAsyncId);
}
ShardRequest sreq = new ShardRequest();
params.set("qt", adminPath);
sreq.purpose = 1;
String replica = zkStateReader.getBaseUrlForNodeName(nodeName);
sreq.shards = new String[] {replica};
sreq.actualShards = sreq.shards;
sreq.nodeName = nodeName;
sreq.params = params;
shardHandler.submit(sreq, replica, sreq.params);
}
void processResponses(NamedList<Object> results, ShardHandler shardHandler, boolean abortOnError, String msgOnError) {
processResponses(results, shardHandler, abortOnError, msgOnError, Collections.emptySet());
}
void processResponses(NamedList<Object> results, ShardHandler shardHandler, boolean abortOnError, String msgOnError,
Set<String> okayExceptions) {
// Processes all shard responses
ShardResponse srsp;
do {
srsp = shardHandler.takeCompletedOrError();
if (srsp != null) {
processResponse(results, srsp, okayExceptions);
Throwable exception = srsp.getException();
if (abortOnError && exception != null) {
// drain pending requests
while (srsp != null) {
srsp = shardHandler.takeCompletedOrError();
}
throw new SolrException(ErrorCode.SERVER_ERROR, msgOnError, exception);
}
}
} while (srsp != null);
// If request is async wait for the core admin to complete before returning
if (asyncId != null) {
waitForAsyncCallsToComplete(results); // TODO: Shouldn't we abort with msgOnError exception when failure?
shardAsyncIdByNode.clear();
}
}
private void waitForAsyncCallsToComplete(NamedList<Object> results) {
for (Map.Entry<String,String> nodeToAsync:shardAsyncIdByNode) {
final String node = nodeToAsync.getKey();
final String shardAsyncId = nodeToAsync.getValue();
log.debug("I am Waiting for :{}/{}", node, shardAsyncId);
NamedList<Object> reqResult = waitForCoreAdminAsyncCallToComplete(node, shardAsyncId);
if (INCLUDE_TOP_LEVEL_RESPONSE) {
results.add(shardAsyncId, reqResult);
}
if ("failed".equalsIgnoreCase(((String)reqResult.get("STATUS")))) {
log.error("Error from shard {}: {}", node, reqResult);
addFailure(results, node, reqResult);
} else {
addSuccess(results, node, reqResult);
}
}
}
/** @deprecated consider to make it private after {@link CreateCollectionCmd} refactoring*/
@Deprecated void track(String nodeName, String coreAdminAsyncId) {
shardAsyncIdByNode.add(nodeName, coreAdminAsyncId);
}
}
}

View File

@ -40,17 +40,15 @@ import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
public class OverseerRoleCmd implements OverseerCollectionMessageHandler.Cmd {
public class OverseerRoleCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
private final CollectionAction operation;
private final OverseerNodePrioritizer overseerPrioritizer;
public OverseerRoleCmd(OverseerCollectionMessageHandler ocmh, CollectionAction operation, OverseerNodePrioritizer prioritizer) {
this.ocmh = ocmh;
public OverseerRoleCmd(CollectionCommandContext ccc, CollectionAction operation, OverseerNodePrioritizer prioritizer) {
this.ccc = ccc;
this.operation = operation;
this.overseerPrioritizer = prioritizer;
}
@ -58,7 +56,7 @@ public class OverseerRoleCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
SolrZkClient zkClient = zkStateReader.getZkClient();
Map roles = null;
String node = message.getStr("node");
@ -90,7 +88,7 @@ public class OverseerRoleCmd implements OverseerCollectionMessageHandler.Cmd {
// overseers are created when there are too many nodes . So , do this operation in a separate thread
new Thread(() -> {
try {
overseerPrioritizer.prioritizeOverseerNodes(ocmh.myId);
overseerPrioritizer.prioritizeOverseerNodes(ccc.getOverseerId());
} catch (Exception e) {
log.error("Error in prioritizing Overseer", e);
}

View File

@ -36,18 +36,18 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
public class OverseerStatusCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public OverseerStatusCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public OverseerStatusCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@SuppressWarnings("unchecked")
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
String leaderNode = OverseerTaskProcessor.getLeaderNode(zkStateReader.getZkClient());
results.add("leader", leaderNode);
Stat stat = new Stat();
@ -64,7 +64,7 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
// Sharing the ocmh.stats variable between the cluster state updater and the Collection API (this command) is by the way
// about the only thing that ties the cluster state updater to the collection api message handler and that takes
// advantage of the fact that both run on the same node (the Overseer node). (recently added PerReplicaStates also
// take advantage of this through method Overseer.submit()).
// take advantage of this through method Overseer.submit() accessed via CollectionCommandContext.submitIntraProcessMessage()).
// When distributed updates are enabled, cluster state updates are not done by the Overseer (it doesn't even see them)
// and therefore can't report them. The corresponding data in OVERSEERSTATUS (all data built below) is no longer returned.
// This means top level keys "overseer_operations", "collection_operations", "overseer_queue", "overseer_internal_queue"
@ -80,7 +80,9 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
NamedList workQueueStats = new NamedList();
@SuppressWarnings({"rawtypes"})
NamedList collectionQueueStats = new NamedList();
Stats stats = ocmh.stats;
// stats below do not make sense when cluster state updates are distributed. Return them empty.
if (!ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
Stats stats = ccc.getOverseerStats();
for (Map.Entry<String, Stats.Stat> entry : stats.getStats().entrySet()) {
String key = entry.getKey();
NamedList<Object> lst = new SimpleOrderedMap<>();
@ -118,6 +120,7 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
Timer timer = entry.getValue().requestTime;
MetricUtils.addMetrics(lst, timer);
}
}
results.add("overseer_operations", overseerStats);
results.add("collection_operations", collectionStats);
results.add("overseer_queue", stateUpdateQueueStats);

View File

@ -88,7 +88,7 @@ import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES
* <li>optionally delete the source collection.</li>
* </ol>
*/
public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String COMMAND = "cmd";
@ -114,7 +114,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM
);
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
private static AtomicInteger tmpCollectionSeq = new AtomicInteger();
@ -161,8 +161,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
private String zkHost;
public ReindexCollectionCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public ReindexCollectionCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -179,7 +179,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collection;
if (followAliases) {
collection = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollection);
collection = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollection);
} else {
collection = extCollection;
}
@ -191,7 +191,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
target = collection;
} else {
if (followAliases) {
target = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(target);
target = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(target);
}
}
boolean sameTarget = target.equals(collection) || target.equals(extCollection);
@ -200,7 +200,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
if (command == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command: " + message.getStr(COMMAND));
}
Map<String, Object> reindexingState = getReindexingState(ocmh.cloudManager.getDistribStateManager(), collection);
Map<String, Object> reindexingState = getReindexingState(ccc.getSolrCloudManager().getDistribStateManager(), collection);
if (!reindexingState.containsKey(STATE)) {
reindexingState.put(STATE, State.IDLE.toLower());
}
@ -244,7 +244,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
router = DocRouter.DEFAULT;
}
String configName = message.getStr(ZkStateReader.CONFIGNAME_PROP, ocmh.zkStateReader.readConfigName(collection));
String configName = message.getStr(ZkStateReader.CONFIGNAME_PROP, ccc.getZkStateReader().readConfigName(collection));
String targetCollection;
int seq = tmpCollectionSeq.getAndIncrement();
if (sameTarget) {
@ -263,7 +263,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
Exception exc = null;
boolean createdTarget = false;
try {
zkHost = ocmh.zkStateReader.getZkClient().getZkServerAddress();
zkHost = ccc.getZkStateReader().getZkClient().getZkServerAddress();
// set the running flag
reindexingState.clear();
reindexingState.put("actualSourceCollection", collection);
@ -286,8 +286,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CommonParams.NAME, chkCollection,
CoreAdminParams.DELETE_METRICS_HISTORY, "true"
);
ocmh.commandMap.get(CollectionParams.CollectionAction.DELETE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("deleting old checkpoint collection " + chkCollection, cmdResults, true);
new DeleteCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("deleting old checkpoint collection " + chkCollection, cmdResults, true);
}
if (maybeAbort(collection)) {
@ -332,9 +332,9 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
// create the target collection
cmd = new ZkNodeProps(propMap);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, cmd, cmdResults);
new CreateCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
createdTarget = true;
ocmh.checkResults("creating target collection " + targetCollection, cmdResults, true);
CollectionHandlingUtils.checkResults("creating target collection " + targetCollection, cmdResults, true);
// create the checkpoint collection - use RF=1 and 1 shard
cmd = new ZkNodeProps(
@ -346,15 +346,15 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CommonAdminParams.WAIT_FOR_FINAL_STATE, "true"
);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("creating checkpoint collection " + chkCollection, cmdResults, true);
new CreateCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("creating checkpoint collection " + chkCollection, cmdResults, true);
// wait for a while until we see both collections
TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, ocmh.timeSource);
TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, ccc.getSolrCloudManager().getTimeSource());
boolean created = false;
while (!waitUntil.hasTimedOut()) {
waitUntil.sleep(100);
// this also refreshes our local var clusterState
clusterState = ocmh.cloudManager.getClusterStateProvider().getClusterState();
clusterState = ccc.getSolrCloudManager().getClusterStateProvider().getClusterState();
created = clusterState.hasCollection(targetCollection) && clusterState.hasCollection(chkCollection);
if (created) break;
}
@ -371,11 +371,11 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.READ_ONLY, "true");
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, cmd,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, cmd,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(cmd));
ccc.offerStateUpdate(Utils.toJSON(cmd));
}
TestInjection.injectReindexLatch();
@ -407,7 +407,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
log.debug("- starting copying documents from {} to {}", collection, targetCollection);
SolrResponse rsp = null;
try {
rsp = ocmh.cloudManager.request(new QueryRequest(q));
rsp = ccc.getSolrCloudManager().request(new QueryRequest(q));
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to copy documents from " +
collection + " to " + targetCollection, e);
@ -439,8 +439,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CommonParams.NAME, extCollection,
"collections", targetCollection);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.CREATEALIAS).call(clusterState, cmd, cmdResults);
ocmh.checkResults("setting up alias " + extCollection + " -> " + targetCollection, cmdResults, true);
new CreateAliasCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("setting up alias " + extCollection + " -> " + targetCollection, cmdResults, true);
reindexingState.put("alias", extCollection + " -> " + targetCollection);
}
@ -462,8 +462,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CoreAdminParams.DELETE_METRICS_HISTORY, "true"
);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.DELETE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("deleting checkpoint collection " + chkCollection, cmdResults, true);
new DeleteCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("deleting checkpoint collection " + chkCollection, cmdResults, true);
// 7. optionally delete the source collection
if (removeSource) {
@ -475,19 +475,19 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CoreAdminParams.DELETE_METRICS_HISTORY, "true"
);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.DELETE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("deleting source collection " + collection, cmdResults, true);
new DeleteCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("deleting source collection " + collection, cmdResults, true);
} else {
// 8. clear readOnly on source
ZkNodeProps props = new ZkNodeProps(
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.READ_ONLY, null);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
}
}
// 9. set FINISHED state on the target and clear the state on the source
@ -495,11 +495,11 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
ZkStateReader.COLLECTION_PROP, targetCollection,
REINDEXING_STATE, State.FINISHED.toLower());
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
}
reindexingState.put(STATE, State.FINISHED.toLower());
@ -525,7 +525,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
private Map<String, Object> setReindexingState(String collection, State state, Map<String, Object> props) throws Exception {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + REINDEXING_STATE_PATH;
DistribStateManager stateManager = ocmh.cloudManager.getDistribStateManager();
DistribStateManager stateManager = ccc.getSolrCloudManager().getDistribStateManager();
if (props == null) { // retrieve existing props, if any
props = Utils.getJson(stateManager, path);
}
@ -541,7 +541,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
private void removeReindexingState(String collection) throws Exception {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + REINDEXING_STATE_PATH;
DistribStateManager stateManager = ocmh.cloudManager.getDistribStateManager();
DistribStateManager stateManager = ccc.getSolrCloudManager().getDistribStateManager();
if (stateManager.hasData(path)) {
stateManager.removeData(path, -1);
}
@ -555,7 +555,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
}
private long getNumberOfDocs(String collection) {
CloudSolrClient solrClient = ocmh.overseer.getCoreContainer().getSolrClientCache().getCloudSolrClient(zkHost);
CloudSolrClient solrClient = ccc.getCoreContainer().getSolrClientCache().getCloudSolrClient(zkHost);
try {
ModifiableSolrParams params = new ModifiableSolrParams();
params.add(CommonParams.Q, "*:*");
@ -568,13 +568,13 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
}
private boolean maybeAbort(String collection) throws Exception {
DocCollection coll = ocmh.cloudManager.getClusterStateProvider().getClusterState().getCollectionOrNull(collection);
DocCollection coll = ccc.getSolrCloudManager().getClusterStateProvider().getClusterState().getCollectionOrNull(collection);
if (coll == null) {
// collection no longer present - abort
log.info("## Aborting - collection {} no longer present.", collection);
return true;
}
Map<String, Object> reindexingState = getReindexingState(ocmh.cloudManager.getDistribStateManager(), collection);
Map<String, Object> reindexingState = getReindexingState(ccc.getSolrCloudManager().getDistribStateManager(), collection);
State state = State.get(reindexingState.getOrDefault(STATE, State.RUNNING.toLower()));
if (state != State.ABORTED) {
return false;
@ -639,7 +639,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
// XXX SolrCore where the daemon is running
@SuppressWarnings({"unchecked"})
private void waitForDaemon(String daemonName, String daemonUrl, String sourceCollection, String targetCollection, Map<String, Object> reindexingState) throws Exception {
HttpClient client = ocmh.overseer.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
HttpClient client = ccc.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
try (HttpSolrClient solrClient = new HttpSolrClient.Builder()
.withHttpClient(client)
.withBaseSolrUrl(daemonUrl).build()) {
@ -684,7 +684,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
reindexingState.put("processedDocs", getNumberOfDocs(targetCollection));
setReindexingState(sourceCollection, State.RUNNING, reindexingState);
}
ocmh.cloudManager.getTimeSource().sleep(2000);
ccc.getSolrCloudManager().getTimeSource().sleep(2000);
} while (isRunning && !maybeAbort(sourceCollection));
}
}
@ -692,7 +692,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
@SuppressWarnings({"unchecked"})
private void killDaemon(String daemonName, String daemonUrl) throws Exception {
log.debug("-- killing daemon {} at {}", daemonName, daemonUrl);
HttpClient client = ocmh.overseer.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
HttpClient client = ccc.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
try (HttpSolrClient solrClient = new HttpSolrClient.Builder()
.withHttpClient(client)
.withBaseSolrUrl(daemonUrl).build()) {
@ -733,7 +733,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
// now wait for the daemon to really stop
q.set("action", "list");
req = new QueryRequest(q);
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, ocmh.timeSource);
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, ccc.getSolrCloudManager().getTimeSource());
while (!timeOut.hasTimedOut()) {
rsp = solrClient.request(req);
rs = (Map<String, Object>) rsp.get("result-set");
@ -788,7 +788,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
if (daemonUrl != null) {
killDaemon(daemonName, daemonUrl);
}
ClusterState clusterState = ocmh.cloudManager.getClusterStateProvider().getClusterState();
ClusterState clusterState = ccc.getSolrCloudManager().getClusterStateProvider().getClusterState();
NamedList<Object> cmdResults = new NamedList<>();
if (createdTarget && !collection.equals(targetCollection) && clusterState.hasCollection(targetCollection)) {
log.debug(" -- removing {}", targetCollection);
@ -798,8 +798,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
FOLLOW_ALIASES, "false",
CoreAdminParams.DELETE_METRICS_HISTORY, "true"
);
ocmh.commandMap.get(CollectionParams.CollectionAction.DELETE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("CLEANUP: deleting target collection " + targetCollection, cmdResults, false);
new DeleteCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("CLEANUP: deleting target collection " + targetCollection, cmdResults, false);
}
// remove chk collection
@ -812,19 +812,19 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
CoreAdminParams.DELETE_METRICS_HISTORY, "true"
);
cmdResults = new NamedList<>();
ocmh.commandMap.get(CollectionParams.CollectionAction.DELETE).call(clusterState, cmd, cmdResults);
ocmh.checkResults("CLEANUP: deleting checkpoint collection " + chkCollection, cmdResults, false);
new DeleteCollectionCmd(ccc).call(clusterState, cmd, cmdResults);
CollectionHandlingUtils.checkResults("CLEANUP: deleting checkpoint collection " + chkCollection, cmdResults, false);
}
log.debug(" -- turning readOnly mode off for {}", collection);
ZkNodeProps props = new ZkNodeProps(
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.READ_ONLY, null);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionModifyCollection, props,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
}
removeReindexingState(collection);
}

View File

@ -34,13 +34,13 @@ import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES
/**
*
*/
public class RenameCmd implements OverseerCollectionMessageHandler.Cmd {
public class RenameCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public RenameCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public RenameCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@ -48,14 +48,14 @@ public class RenameCmd implements OverseerCollectionMessageHandler.Cmd {
String extCollectionName = message.getStr(CoreAdminParams.NAME);
String target = message.getStr(CollectionAdminParams.TARGET);
if (ocmh.zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
ocmh.zkStateReader.aliasesManager.update();
if (ccc.getZkStateReader().aliasesManager != null) { // not a mock ZkStateReader
ccc.getZkStateReader().aliasesManager.update();
}
if (extCollectionName == null || target == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "both collection 'name' and 'target' name must be specified");
}
Aliases aliases = ocmh.zkStateReader.getAliases();
Aliases aliases = ccc.getZkStateReader().getAliases();
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
@ -67,11 +67,11 @@ public class RenameCmd implements OverseerCollectionMessageHandler.Cmd {
if (!state.hasCollection(collectionName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "source collection '" + collectionName + "' not found.");
}
if (ocmh.zkStateReader.getAliases().hasAlias(target)) {
if (ccc.getZkStateReader().getAliases().hasAlias(target)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "target alias '" + target + "' exists: "
+ ocmh.zkStateReader.getAliases().getCollectionAliasListMap().get(target));
+ ccc.getZkStateReader().getAliases().getCollectionAliasListMap().get(target));
}
ocmh.zkStateReader.aliasesManager.applyModificationAndExportToZk(a -> a.cloneWithRename(extCollectionName, target));
ccc.getZkStateReader().aliasesManager.applyModificationAndExportToZk(a -> a.cloneWithRename(extCollectionName, target));
}
}

View File

@ -49,19 +49,19 @@ import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
public class ReplaceNodeCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public ReplaceNodeCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public ReplaceNodeCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@SuppressWarnings({"unchecked"})
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
String source = message.getStr(CollectionParams.SOURCE_NODE, message.getStr("source"));
String target = message.getStr(CollectionParams.TARGET_NODE, message.getStr("target"));
boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
@ -95,9 +95,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
List<ZkNodeProps> createdReplicas = new ArrayList<>();
AtomicBoolean anyOneFailed = new AtomicBoolean(false);
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ocmh);
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ccc.getCloseableToLatchOn());
SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ocmh);
SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ccc.getCloseableToLatchOn());
try {
for (ZkNodeProps sourceReplica : sourceReplicas) {
@SuppressWarnings({"rawtypes"})
@ -118,16 +118,16 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
.assignNrtReplicas(numNrtReplicas)
.assignTlogReplicas(numTlogReplicas)
.assignPullReplicas(numPullReplicas)
.onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes()))
.onNodes(new ArrayList<>(ccc.getSolrCloudManager().getClusterStateProvider().getLiveNodes()))
.build();
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
ocmh.overseer.getCoreContainer(),
ccc.getCoreContainer(),
clusterState, clusterState.getCollection(sourceCollection));
targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
targetNode = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequest).get(0).node;
}
ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
if (async != null) msg.getProperties().put(ASYNC, async);
final ZkNodeProps addedReplica = ocmh.addReplica(clusterState,
final ZkNodeProps addedReplica = new AddReplicaCmd(ccc).addReplica(clusterState,
msg, nl, () -> {
countDownLatch.countDown();
if (nl.get("failure") != null) {
@ -198,12 +198,12 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
}
if (anyOneFailed.get()) {
log.info("Failed to create some replicas. Cleaning up all replicas on target node");
SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ocmh);
SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
for (ZkNodeProps createdReplica : createdReplicas) {
@SuppressWarnings({"rawtypes"})
NamedList deleteResult = new NamedList();
try {
ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult, () -> {
new DeleteReplicaCmd(ccc).deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult, () -> {
cleanupLatch.countDown();
if (deleteResult.get("failure") != null) {
synchronized (results) {
@ -227,7 +227,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
// we have reached this far means all replicas could be recreated
//now cleanup the replicas in the source node
DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source, async);
DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
results.add("success", "REPLACENODE action completed successfully from : " + source + " to : " + target);
}

View File

@ -19,7 +19,7 @@ package org.apache.solr.cloud.api.collections;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -75,19 +75,19 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.CR
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
public class RestoreCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public RestoreCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public RestoreCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
try (RestoreContext restoreContext = new RestoreContext(message, ocmh)) {
try (RestoreContext restoreContext = new RestoreContext(message, ccc)) {
if (state.hasCollection(restoreContext.restoreCollectionName)) {
RestoreOnExistingCollection restoreOnExistingCollection = new RestoreOnExistingCollection(restoreContext);
restoreOnExistingCollection.process(restoreContext, results);
@ -106,7 +106,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
String repo,
ShardHandler shardHandler,
String asyncId) {
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
// Copy data from backed up index to each replica
for (Slice slice : restoreCollection.getSlices()) {
ModifiableSolrParams params = new ModifiableSolrParams();
@ -153,14 +153,14 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
final DocCollection backupCollectionState;
final ShardHandler shardHandler;
private RestoreContext(ZkNodeProps message, OverseerCollectionMessageHandler ocmh) throws IOException {
private RestoreContext(ZkNodeProps message, CollectionCommandContext ccc) throws IOException {
this.restoreCollectionName = message.getStr(COLLECTION_PROP);
this.backupName = message.getStr(NAME); // of backup
this.asyncId = message.getStr(ASYNC);
this.repo = message.getStr(CoreAdminParams.BACKUP_REPOSITORY);
this.backupId = message.getInt(CoreAdminParams.BACKUP_ID, -1);
this.container = ocmh.overseer.getCoreContainer();
this.container = ccc.getCoreContainer();
this.repository = this.container.newBackupRepository(repo);
this.location = repository.createURI(message.getStr(CoreAdminParams.BACKUP_LOCATION));
@ -170,7 +170,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
this.backupPath = (incremental) ?
repository.resolve(backupNameUri, entries[0]) : // incremental backups have an extra path component representing the backed up collection
backupNameUri;
this.zkStateReader = ocmh.zkStateReader;
this.zkStateReader = ccc.getZkStateReader();
this.backupManager = backupId == -1 ?
BackupManager.forRestore(repository, zkStateReader, backupPath) :
BackupManager.forRestore(repository, zkStateReader, backupPath, backupId);
@ -180,9 +180,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
this.restoreConfigName = message.getStr(CollectionAdminParams.COLL_CONF, this.backupProperties.getConfigName());
this.backupCollectionState = this.backupManager.readCollectionState(this.backupCollection);
this.shardHandler = ocmh.shardHandlerFactory.getShardHandler();
this.shardHandler = ccc.getShardHandler();
this.nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(
zkStateReader.getClusterState().getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
zkStateReader.getClusterState().getLiveNodes(), message, CollectionHandlingUtils.RANDOM);
}
@Override
@ -252,7 +252,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Object failures = results.get("failure");
if (failures != null && ((SimpleOrderedMap) failures).size() > 0) {
log.error("Restore failed to create initial replicas.");
ocmh.cleanupCollection(rc.restoreCollectionName, new NamedList<>());
CollectionHandlingUtils.cleanupCollection(rc.restoreCollectionName, new NamedList<>(), ccc);
return;
}
@ -298,7 +298,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(PULL_REPLICAS, numPullReplicas);
// inherit settings from input API, defaulting to the backup's setting. Ex: replicationFactor
for (String collProp : OverseerCollectionMessageHandler.COLLECTION_PROPS_AND_DEFAULTS.keySet()) {
for (String collProp : CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.keySet()) {
Object val = message.getProperties().getOrDefault(collProp, backupCollectionState.get(collProp));
if (val != null && propMap.get(collProp) == null) {
propMap.put(collProp, val);
@ -306,7 +306,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
propMap.put(NAME, restoreCollectionName);
propMap.put(OverseerCollectionMessageHandler.CREATE_NODE_SET, OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY); //no cores
propMap.put(CollectionHandlingUtils.CREATE_NODE_SET, CollectionHandlingUtils.CREATE_NODE_SET_EMPTY); //no cores
propMap.put(CollectionAdminParams.COLL_CONF, restoreConfigName);
// router.*
@ -317,9 +317,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Set<String> sliceNames = backupCollectionState.getActiveSlicesMap().keySet();
if (backupCollectionState.getRouter() instanceof ImplicitDocRouter) {
propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, StrUtils.join(sliceNames, ','));
propMap.put(CollectionHandlingUtils.SHARDS_PROP, StrUtils.join(sliceNames, ','));
} else {
propMap.put(OverseerCollectionMessageHandler.NUM_SLICES, sliceNames.size());
propMap.put(CollectionHandlingUtils.NUM_SLICES, sliceNames.size());
// ClusterStateMutator.createCollection detects that "slices" is in fact a slice structure instead of a
// list of names, and if so uses this instead of building it. We clear the replica list.
Collection<Slice> backupSlices = backupCollectionState.getActiveSlices();
@ -328,10 +328,10 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
newSlices.put(backupSlice.getName(),
new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(), restoreCollectionName));
}
propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, newSlices);
propMap.put(CollectionHandlingUtils.SHARDS_PROP, newSlices);
}
ocmh.commandMap.get(CREATE).call(clusterState, new ZkNodeProps(propMap), new NamedList());
new CreateCollectionCmd(ccc).call(clusterState, new ZkNodeProps(propMap), new NamedList());
// note: when createCollection() returns, the collection exists (no race)
}
@ -344,11 +344,11 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(shard.getName(), Slice.State.CONSTRUCTION.toString());
}
propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollection.getName());
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, new ZkNodeProps(propMap),
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, new ZkNodeProps(propMap),
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
ccc.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
}
}
@ -362,9 +362,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
.onNodes(nodeList)
.build();
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
ocmh.overseer.getCoreContainer(),
ccc.getCoreContainer(),
clusterState, restoreCollection);
return assignStrategy.assign(ocmh.cloudManager, assignRequest);
return assignStrategy.assign(ccc.getSolrCloudManager(), assignRequest);
}
@SuppressWarnings({"unchecked", "rawtypes"})
@ -407,9 +407,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
CollectionHandlingUtils.addPropertyParams(message, propMap);
final NamedList addReplicaResult = new NamedList();
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
new AddReplicaCmd(ccc).addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
Object addResultFailure = addReplicaResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
@ -437,7 +437,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
private void requestReplicasToApplyBufferUpdates(DocCollection restoreCollection, String asyncId, ShardHandler shardHandler) {
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (Slice s : restoreCollection.getSlices()) {
for (Replica r : s.getReplicas()) {
@ -467,11 +467,11 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
for (Slice shard : restoreCollection.getSlices()) {
propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
}
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, new ZkNodeProps(propMap),
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, new ZkNodeProps(propMap),
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
ccc.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
}
}
@ -534,9 +534,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
CollectionHandlingUtils.addPropertyParams(message, propMap);
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), results, null);
new AddReplicaCmd(ccc).addReplica(clusterState, new ZkNodeProps(propMap), results, null);
}
}
}
@ -547,7 +547,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
String backupCollectionAlias = properties.getCollectionAlias();
if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
ocmh.zkStateReader.aliasesManager
ccc.getZkStateReader().aliasesManager
.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
}
}
@ -590,7 +590,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.COLLECTION_PROP, restoreCollection.getName(),
ZkStateReader.READ_ONLY, null
);
ocmh.modifyCollection(clusterState, params, new NamedList<>());
new CollApiCmds.ModifyCollectionCmd(ccc).call(clusterState, params, new NamedList<>());
}
private void enableReadOnly(ClusterState clusterState, DocCollection restoreCollection) throws Exception {
@ -599,7 +599,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.COLLECTION_PROP, restoreCollection.getName(),
ZkStateReader.READ_ONLY, "true"
);
ocmh.modifyCollection(clusterState, params, new NamedList<>());
new CollApiCmds.ModifyCollectionCmd(ccc).call(clusterState, params, new NamedList<>());
}
}
}

View File

@ -30,27 +30,27 @@ import org.apache.solr.common.util.NamedList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.Cmd;
import static org.apache.solr.cloud.api.collections.CollApiCmds.CollectionApiCommand;
import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
import static org.apache.solr.common.params.CommonParams.NAME;
public class SetAliasPropCmd implements Cmd {
public class SetAliasPropCmd implements CollectionApiCommand {
public static final String PROPERTIES = "property";
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler messageHandler;
private final CollectionCommandContext ccc;
SetAliasPropCmd(OverseerCollectionMessageHandler messageHandler) {
this.messageHandler = messageHandler;
public SetAliasPropCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@Override
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String aliasName = message.getStr(NAME);
final ZkStateReader.AliasesManager aliasesManager = messageHandler.zkStateReader.aliasesManager;
final ZkStateReader.AliasesManager aliasesManager = ccc.getZkStateReader().aliasesManager;
// Ensure we see the alias. This may be redundant but SetAliasPropCmd isn't expected to be called very frequently
aliasesManager.update();

View File

@ -25,7 +25,7 @@ import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -62,17 +62,17 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final int MIN_NUM_SUB_SHARDS = 2;
// This is an arbitrary number that seems reasonable at this time.
private static final int MAX_NUM_SUB_SHARDS = 8;
private static final int DEFAULT_NUM_SUB_SHARDS = 2;
private final OverseerCollectionMessageHandler ocmh;
private final CollectionCommandContext ccc;
public SplitShardCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
public SplitShardCmd(CollectionCommandContext ccc) {
this.ccc = ccc;
}
@SuppressWarnings("unchecked")
@ -99,13 +99,13 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
if (followAliases) {
collectionName = ocmh.cloudManager.getClusterStateProvider().resolveSimpleAlias(extCollectionName);
collectionName = ccc.getSolrCloudManager().getClusterStateProvider().resolveSimpleAlias(extCollectionName);
} else {
collectionName = extCollectionName;
}
log.debug("Split shard invoked: {}", message);
ZkStateReader zkStateReader = ocmh.zkStateReader;
ZkStateReader zkStateReader = ccc.getZkStateReader();
zkStateReader.forceUpdateCollection(collectionName);
AtomicReference<String> slice = new AtomicReference<>();
slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
@ -132,9 +132,9 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
RTimerTree t;
if (ocmh.overseer.getCoreContainer().getNodeConfig().getMetricsConfig().isEnabled()) {
if (ccc.getCoreContainer().getNodeConfig().getMetricsConfig().isEnabled()) {
t = timings.sub("checkDiskSpace");
checkDiskSpace(collectionName, slice.get(), parentShardLeader, splitMethod, ocmh.cloudManager);
checkDiskSpace(collectionName, slice.get(), parentShardLeader, splitMethod, ccc.getSolrCloudManager());
t.stop();
}
@ -179,7 +179,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
// check for the lock
if (!lockForSplit(ocmh.cloudManager, collectionName, parentSlice.getName())) {
if (!lockForSplit(ccc.getSolrCloudManager(), collectionName, parentSlice.getName())) {
// mark as success to avoid clearing the lock in the "finally" block
success = true;
throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Can't lock parent slice for splitting (another split operation running?): " +
@ -189,7 +189,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2);
@SuppressWarnings("deprecation")
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ShardHandler shardHandler = ccc.getShardHandler();
if (message.getBool(CommonAdminParams.SPLIT_BY_PREFIX, false)) {
@ -205,7 +205,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
// params.set(NUM_SUB_SHARDS, Integer.toString(numSubShards));
{
final ShardRequestTracker shardRequestTracker = ocmh.syncRequestTracker();
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.syncRequestTracker(ccc);
shardRequestTracker.sendShardRequest(parentShardLeader.getNodeName(), params, shardHandler);
SimpleOrderedMap<Object> getRangesResults = new SimpleOrderedMap<>();
String msgOnError = "SPLITSHARD failed to invoke SPLIT.getRanges core admin command";
@ -231,7 +231,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
t = timings.sub("fillRanges");
String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
String rangesStr = fillRanges(ccc.getSolrCloudManager(), message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
t.stop();
boolean oldShardsDeleted = false;
@ -251,7 +251,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(SHARD_ID_PROP, subSlice);
ZkNodeProps m = new ZkNodeProps(propMap);
try {
ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList());
new DeleteShardCmd(ccc).call(clusterState, m, new NamedList());
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
e);
@ -289,15 +289,15 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put("shard_parent_node", nodeName);
propMap.put("shard_parent_zk_session", leaderZnodeStat.getEphemeralOwner());
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionCreateShard, new ZkNodeProps(propMap),
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.CollectionCreateShard, new ZkNodeProps(propMap),
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
ccc.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
}
// wait until we are able to see the new shard in cluster state and refresh the local view of the cluster state
clusterState = ocmh.waitForNewShard(collectionName, subSlice);
clusterState = CollectionHandlingUtils.waitForNewShard(collectionName, subSlice, ccc.getZkStateReader());
log.debug("Adding first replica {} as part of slice {} of collection {} on {}"
, subShardName, subSlice, collectionName, nodeName);
@ -319,12 +319,12 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), results, null);
new AddReplicaCmd(ccc).addReplica(clusterState, new ZkNodeProps(propMap), results, null);
}
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
final ShardRequestTracker syncRequestTracker = CollectionHandlingUtils.syncRequestTracker(ccc);
String msgOnError = "SPLITSHARD failed to create subshard leaders";
syncRequestTracker.processResponses(results, shardHandler, true, msgOnError);
handleFailureOnAsyncRequest(results, msgOnError);
@ -332,11 +332,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
t.stop();
t = timings.sub("waitForSubSliceLeadersAlive");
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (String subShardName : subShardNames) {
// wait for parent leader to acknowledge the sub-shard core
log.debug("Asking parent leader to wait for: {} to be alive on: {}", subShardName, nodeName);
String coreNodeName = ocmh.waitForCoreNodeName(collectionName, nodeName, subShardName);
String coreNodeName = CollectionHandlingUtils.waitForCoreNodeName(collectionName, nodeName, subShardName, ccc.getZkStateReader());
CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
cmd.setCoreName(subShardName);
cmd.setNodeName(nodeName);
@ -375,7 +375,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
t = timings.sub("splitParentCore");
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
shardRequestTracker.sendShardRequest(parentShardLeader.getNodeName(), params, shardHandler);
String msgOnError = "SPLITSHARD failed to invoke SPLIT core admin command";
@ -391,7 +391,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
t = timings.sub("applyBufferedUpdates");
// apply buffered updates on sub-shards
{
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
final ShardRequestTracker shardRequestTracker = CollectionHandlingUtils.asyncRequestTracker(asyncId, ccc);
for (int i = 0; i < subShardNames.size(); i++) {
String subShardName = subShardNames.get(i);
@ -442,16 +442,16 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
.onNodes(new ArrayList<>(clusterState.getLiveNodes()))
.build();
Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
ocmh.overseer.getCoreContainer(),
ccc.getCoreContainer(),
clusterState, collection);
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequest);
t.stop();
t = timings.sub("createReplicaPlaceholders");
final DistributedClusterStateUpdater.StateChangeRecorder scr;
boolean hasRecordedDistributedUpdate = false;
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
scr = ocmh.getDistributedClusterStateUpdater().createStateChangeRecorder(collectionName, false);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
scr = ccc.getDistributedClusterStateUpdater().createStateChangeRecorder(collectionName, false);
} else {
scr = null;
}
@ -475,11 +475,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.NODE_NAME_PROP, subShardNodeName,
CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
hasRecordedDistributedUpdate = true;
scr.record(DistributedClusterStateUpdater.MutatingCommand.SliceAddReplica, props);
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
ccc.offerStateUpdate(Utils.toJSON(props));
}
HashMap<String, Object> propMap = new HashMap<>();
@ -500,17 +500,17 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(ASYNC, asyncId);
}
// special flag param to instruct addReplica not to create the replica in cluster state again
propMap.put(OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, "true");
propMap.put(CollectionHandlingUtils.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, "true");
propMap.put(CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
replicas.add(propMap);
}
if (hasRecordedDistributedUpdate && ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
if (hasRecordedDistributedUpdate && ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
// Actually add the replicas to the collection state. Note that when Overseer takes care of the state,
// there is no wait here for the state update to be visible, but with distributed state update done synchronously
// we wait (we could in theory create a thread and have it do the work if we REALLY needed, but we likely don't).
scr.executeStateUpdates(ocmh.cloudManager, ocmh.zkStateReader);
scr.executeStateUpdates(ccc.getSolrCloudManager(), ccc.getZkStateReader());
}
t.stop();
assert TestInjection.injectSplitFailureBeforeReplicaCreation();
@ -528,11 +528,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
ZkNodeProps m = new ZkNodeProps(propMap);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
if (leaderZnodeStat == null) {
@ -555,7 +555,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
// (Note: This commit used to be after the state switch, but was brought here before the state switch
// as per SOLR-13945 so that sub shards don't come up empty, momentarily, after being marked active)
t = timings.sub("finalCommit");
ocmh.commit(results, slice.get(), parentShardLeader);
CollectionHandlingUtils.commit(results, slice.get(), parentShardLeader);
t.stop();
// switch sub shard states to 'active'
log.info("Replication factor is 1 so switching shard states");
@ -567,11 +567,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
ZkNodeProps m = new ZkNodeProps(propMap);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
} else {
log.info("Requesting shard state be set to 'recovery'");
@ -582,24 +582,24 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
ZkNodeProps m = new ZkNodeProps(propMap);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
}
t = timings.sub("createCoresForReplicas");
// now actually create replica cores on sub shard nodes
for (Map<String, Object> replica : replicas) {
ocmh.addReplica(clusterState, new ZkNodeProps(replica), results, null);
new AddReplicaCmd(ccc).addReplica(clusterState, new ZkNodeProps(replica), results, null);
}
assert TestInjection.injectSplitFailureAfterReplicaCreation();
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
final ShardRequestTracker syncRequestTracker = CollectionHandlingUtils.syncRequestTracker(ccc);
String msgOnError = "SPLITSHARD failed to create subshard replicas";
syncRequestTracker.processResponses(results, shardHandler, true, msgOnError);
handleFailureOnAsyncRequest(results, msgOnError);
@ -612,7 +612,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
// when the sub-shard replicas come up
if (repFactor > 1) {
t = timings.sub("finalCommit");
ocmh.commit(results, slice.get(), parentShardLeader);
CollectionHandlingUtils.commit(results, slice.get(), parentShardLeader);
t.stop();
}
@ -631,7 +631,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
} finally {
if (!success) {
cleanupAfterFailure(zkStateReader, collectionName, parentSlice.getName(), subSlices, offlineSlices);
unlockForSplit(ocmh.cloudManager, collectionName, parentSlice.getName());
unlockForSplit(ccc.getSolrCloudManager(), collectionName, parentSlice.getName());
}
}
}
@ -748,11 +748,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (sendUpdateState) {
try {
ZkNodeProps m = new ZkNodeProps(propMap);
if (ocmh.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ocmh.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ocmh.cloudManager, ocmh.zkStateReader);
if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
ccc.getDistributedClusterStateUpdater().doSingleStateUpdate(DistributedClusterStateUpdater.MutatingCommand.SliceUpdateShardState, m,
ccc.getSolrCloudManager(), ccc.getZkStateReader());
} else {
ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
ccc.offerStateUpdate(Utils.toJSON(m));
}
} catch (Exception e) {
// don't give up yet - just log the error, we may still be able to clean up
@ -773,7 +773,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
props.put(SHARD_ID_PROP, subSlice);
ZkNodeProps m = new ZkNodeProps(props);
try {
ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList<Object>());
new DeleteShardCmd(ccc).call(clusterState, m, new NamedList<Object>());
} catch (Exception e) {
log.warn("Cleanup failed after failed split of {}/{} : (deleting existing sub shard{})", collectionName, parentShard, subSlice, e);
}

View File

@ -26,7 +26,7 @@ import java.util.Map;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@ -96,10 +96,10 @@ public class ClusterStateMutator {
Map<String, Object> collectionProps = new HashMap<>();
for (Map.Entry<String, Object> e : OverseerCollectionMessageHandler.COLLECTION_PROPS_AND_DEFAULTS.entrySet()) {
for (Map.Entry<String, Object> e : CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.entrySet()) {
Object val = message.get(e.getKey());
if (val == null) {
val = OverseerCollectionMessageHandler.COLLECTION_PROPS_AND_DEFAULTS.get(e.getKey());
val = CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.get(e.getKey());
}
if (val != null) collectionProps.put(e.getKey(), val);
}

View File

@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.CloudUtil;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.Assign;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.cloud.api.collections.SplitShardCmd;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@ -127,14 +127,14 @@ public class ReplicaMutator {
}
property = property.toLowerCase(Locale.ROOT);
String propVal = message.getStr(ZkStateReader.PROPERTY_VALUE_PROP);
String shardUnique = message.getStr(OverseerCollectionMessageHandler.SHARD_UNIQUE);
String shardUnique = message.getStr(CollectionHandlingUtils.SHARD_UNIQUE);
boolean isUnique = false;
if (SliceMutator.SLICE_UNIQUE_BOOLEAN_PROPERTIES.contains(property)) {
if (StringUtils.isNotBlank(shardUnique) && Boolean.parseBoolean(shardUnique) == false) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Overseer ADDREPLICAPROP for " +
property + " cannot have " + OverseerCollectionMessageHandler.SHARD_UNIQUE + " set to anything other than" +
property + " cannot have " + CollectionHandlingUtils.SHARD_UNIQUE + " set to anything other than" +
"'true'. No action taken");
}
isUnique = true;

View File

@ -46,7 +46,7 @@ import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
import static org.apache.solr.client.solrj.SolrRequest.METHOD.PUT;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.REQUESTID;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.REQUESTID;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.OVERSEERSTATUS;

View File

@ -106,8 +106,15 @@ import java.util.stream.Collectors;
import static org.apache.solr.client.solrj.response.RequestStatusState.*;
import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.REQUESTID;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.*;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET_EMPTY;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.NUM_SLICES;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_IF_DOWN;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.REQUESTID;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARDS_PROP;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;

View File

@ -63,7 +63,7 @@ import org.apache.solr.client.solrj.response.GroupCommand;
import org.apache.solr.client.solrj.response.GroupResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
@ -781,9 +781,9 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("action", CollectionAction.CREATE.toString());
params.set(OverseerCollectionMessageHandler.NUM_SLICES, numShards);
params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
if (createNodeSetStr != null) params.set(OverseerCollectionMessageHandler.CREATE_NODE_SET, createNodeSetStr);
if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
List<Integer> list = new ArrayList<>();

View File

@ -42,7 +42,7 @@ import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.cloud.Overseer.LeaderStatus;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.cluster.placement.PlacementPluginFactory;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
@ -595,13 +595,13 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
ZkStateReader.REPLICATION_FACTOR, replicationFactor.toString(),
"name", COLLECTION_NAME,
"collection.configName", CONFIG_NAME,
OverseerCollectionMessageHandler.NUM_SLICES, numberOfSlices.toString()
CollectionHandlingUtils.NUM_SLICES, numberOfSlices.toString()
);
if (sendCreateNodeList) {
propMap.put(OverseerCollectionMessageHandler.CREATE_NODE_SET,
propMap.put(CollectionHandlingUtils.CREATE_NODE_SET,
(createNodeList != null)?StrUtils.join(createNodeList, ','):null);
if (OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE_DEFAULT != createNodeSetShuffle || random().nextBoolean()) {
propMap.put(OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE, createNodeSetShuffle);
if (CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE_DEFAULT != createNodeSetShuffle || random().nextBoolean()) {
propMap.put(CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE, createNodeSetShuffle);
}
}

View File

@ -48,9 +48,9 @@ public class OverseerStatusTest extends SolrCloudTestCase {
NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
SimpleOrderedMap<Object> createcollection
= (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
assertEquals("No stats for create in OverseerCollectionProcessor", numCollectionCreates + 1, createcollection.get("requests"));
// When cluster state updates are distributed, Overseer doesn't see the updates and doesn't report stats on them.
if (!cluster.getOpenOverseer().getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
assertEquals("No stats for create in OverseerCollectionProcessor", numCollectionCreates + 1, createcollection.get("requests"));
createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
assertEquals("No stats for create in Overseer", numOverseerCreates + 1, createcollection.get("requests"));
}

View File

@ -21,7 +21,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonAdminParams;
@ -52,7 +52,7 @@ public class OverseerTaskQueueTest extends DistributedQueueTest {
final Map<String, Object> props = new HashMap<>();
props.put(CommonParams.NAME, "coll1");
props.put(CollectionAdminParams.COLL_CONF, "myconf");
props.put(OverseerCollectionMessageHandler.NUM_SLICES, 1);
props.put(CollectionHandlingUtils.NUM_SLICES, 1);
props.put(ZkStateReader.REPLICATION_FACTOR, 3);
props.put(CommonAdminParams.ASYNC, requestId);
tq.offer(Utils.toJSON(props));

View File

@ -20,6 +20,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.util.NamedList;
@ -34,8 +35,8 @@ public class AsyncCallRequestStatusResponseTest extends SolrCloudTestCase {
@SuppressWarnings("deprecation")
@BeforeClass
public static void setupCluster() throws Exception {
oldResponseEntries = OverseerCollectionMessageHandler.INCLUDE_TOP_LEVEL_RESPONSE;
OverseerCollectionMessageHandler.INCLUDE_TOP_LEVEL_RESPONSE = random().nextBoolean();
oldResponseEntries = ShardRequestTracker.INCLUDE_TOP_LEVEL_RESPONSE;
ShardRequestTracker.INCLUDE_TOP_LEVEL_RESPONSE = random().nextBoolean();
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
@ -44,7 +45,7 @@ public class AsyncCallRequestStatusResponseTest extends SolrCloudTestCase {
@SuppressWarnings("deprecation")
@AfterClass
public static void restoreFlag() throws Exception {
OverseerCollectionMessageHandler.INCLUDE_TOP_LEVEL_RESPONSE = oldResponseEntries;
ShardRequestTracker.INCLUDE_TOP_LEVEL_RESPONSE = oldResponseEntries;
}
@SuppressWarnings("deprecation")
@ -65,7 +66,7 @@ public class AsyncCallRequestStatusResponseTest extends SolrCloudTestCase {
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus(asyncId);
CollectionAdminResponse rsp = requestStatus.process(cluster.getSolrClient());
NamedList<?> r = rsp.getResponse();
if (OverseerCollectionMessageHandler.INCLUDE_TOP_LEVEL_RESPONSE) {
if (ShardRequestTracker.INCLUDE_TOP_LEVEL_RESPONSE) {
final int actualNumOfElems = 3+(numShards*numReplicas);
// responseHeader, success, status, + old responses per every replica
assertEquals("Expected "+actualNumOfElems+" elements in the response" + r.jsonStr(),

View File

@ -797,7 +797,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
try (CloudSolrClient client = createCloudClient(null)) {
Map<String, Object> props = Utils.makeMap(
REPLICATION_FACTOR, replicationFactor,
OverseerCollectionMessageHandler.NUM_SLICES, numShards,
CollectionHandlingUtils.NUM_SLICES, numShards,
"router.field", shard_fld);
createCollection(collectionInfos, collectionName, props, client);
@ -856,7 +856,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
try (CloudSolrClient client = createCloudClient(null)) {
Map<String, Object> props = Utils.makeMap(
REPLICATION_FACTOR, replicationFactor,
OverseerCollectionMessageHandler.NUM_SLICES, numShards);
CollectionHandlingUtils.NUM_SLICES, numShards);
createCollection(collectionInfos, collectionName,props,client);
}

View File

@ -765,7 +765,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "testprop",
"property.value", "nonsense",
OverseerCollectionMessageHandler.SHARD_UNIQUE, "true");
CollectionHandlingUtils.SHARD_UNIQUE, "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
@ -782,7 +782,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "property.testprop",
"property.value", "true",
OverseerCollectionMessageHandler.SHARD_UNIQUE, "false");
CollectionHandlingUtils.SHARD_UNIQUE, "false");
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
@ -813,7 +813,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "preferredLeader",
"property.value", "true",
OverseerCollectionMessageHandler.SHARD_UNIQUE, "false");
CollectionHandlingUtils.SHARD_UNIQUE, "false");
fail("Should have thrown an exception, setting shardUnique=false is not allowed for 'preferredLeader'.");
} catch (SolrException se) {
assertTrue("Should have received a specific error message",

View File

@ -88,7 +88,7 @@ public class TestCollectionsAPIViaSolrCloudCluster extends SolrCloudTestCase {
}
if (createNodeSet != null && createNodeSet.equals(OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY)) {
if (createNodeSet != null && createNodeSet.equals(CollectionHandlingUtils.CREATE_NODE_SET_EMPTY)) {
cluster.waitForActiveCollection(collectionName, numShards, 0);
} else {
cluster.waitForActiveCollection(collectionName, numShards, numShards * numReplicas);
@ -190,7 +190,7 @@ public class TestCollectionsAPIViaSolrCloudCluster extends SolrCloudTestCase {
assertFalse(cluster.getJettySolrRunners().isEmpty());
// create collection
createCollection(collectionName, OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY);
createCollection(collectionName, CollectionHandlingUtils.CREATE_NODE_SET_EMPTY);
// check the collection's corelessness
int coreCount = 0;

View File

@ -68,7 +68,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionMessageHandler.REQUESTID, "1000");
params.set(CollectionHandlingUtils.REQUESTID, "1000");
NamedList<Object> createResponse =null;
try {
@ -85,7 +85,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
// Check for a random (hopefully non-existent request id
params = new ModifiableSolrParams();
params.set(CollectionParams.ACTION, CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionMessageHandler.REQUESTID, "9999999");
params.set(CollectionHandlingUtils.REQUESTID, "9999999");
try {
r = sendRequest(params);
status = (NamedList) r.get("status");
@ -110,7 +110,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
// Check for the request to be completed.
params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionMessageHandler.REQUESTID, "1001");
params.set(CollectionHandlingUtils.REQUESTID, "1001");
NamedList<Object> splitResponse=null;
try {
splitResponse = sendStatusRequestWithRetry(params, MAX_WAIT_TIMEOUT_SECONDS);
@ -140,7 +140,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionMessageHandler.REQUESTID, "1002");
params.set(CollectionHandlingUtils.REQUESTID, "1002");
try {
NamedList<Object> response = sendStatusRequestWithRetry(params, MAX_WAIT_TIMEOUT_SECONDS);

View File

@ -63,7 +63,7 @@ import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.cloud.ZkController.NotInClusterStateException;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
@ -1769,9 +1769,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
for (Map.Entry<String, Object> entry : collectionProps.entrySet()) {
if(entry.getValue() !=null) params.set(entry.getKey(), String.valueOf(entry.getValue()));
}
Integer numShards = (Integer) collectionProps.get(OverseerCollectionMessageHandler.NUM_SLICES);
Integer numShards = (Integer) collectionProps.get(CollectionHandlingUtils.NUM_SLICES);
if(numShards==null){
String shardNames = (String) collectionProps.get(OverseerCollectionMessageHandler.SHARDS_PROP);
String shardNames = (String) collectionProps.get(CollectionHandlingUtils.SHARDS_PROP);
numShards = StrUtils.splitSmart(shardNames,',').size();
}
Integer numNrtReplicas = (Integer) collectionProps.get(ZkStateReader.NRT_REPLICAS);
@ -1779,7 +1779,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
numNrtReplicas = (Integer) collectionProps.get(ZkStateReader.REPLICATION_FACTOR);
}
if(numNrtReplicas == null){
numNrtReplicas = (Integer) OverseerCollectionMessageHandler.COLLECTION_PROPS_AND_DEFAULTS.get(ZkStateReader.REPLICATION_FACTOR);
numNrtReplicas = (Integer) CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.get(ZkStateReader.REPLICATION_FACTOR);
}
if (numNrtReplicas == null) {
numNrtReplicas = Integer.valueOf(0);
@ -1837,11 +1837,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
int numTlogReplicas = useTlogReplicas()?replicationFactor:0;
return createCollection(collectionInfos, collectionName,
Utils.makeMap(
OverseerCollectionMessageHandler.NUM_SLICES, numShards,
CollectionHandlingUtils.NUM_SLICES, numShards,
ZkStateReader.NRT_REPLICAS, numNrtReplicas,
ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
OverseerCollectionMessageHandler.CREATE_NODE_SET, createNodeSetStr),
CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr),
client, configSetName);
}
@ -1852,11 +1852,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
int numTlogReplicas = useTlogReplicas()?replicationFactor:0;
return createCollection(collectionInfos, collectionName,
Utils.makeMap(
OverseerCollectionMessageHandler.NUM_SLICES, numShards,
CollectionHandlingUtils.NUM_SLICES, numShards,
ZkStateReader.NRT_REPLICAS, numNrtReplicas,
ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
OverseerCollectionMessageHandler.CREATE_NODE_SET, createNodeSetStr),
CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr),
client, configName);
}
@ -2049,7 +2049,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
ZkStateReader.NRT_REPLICAS, numNrtReplicas,
ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
OverseerCollectionMessageHandler.NUM_SLICES, numShards);
CollectionHandlingUtils.NUM_SLICES, numShards);
Map<String,List<Integer>> collectionInfos = new HashMap<>();
createCollection(collectionInfos, collName, props, client);
}