Skip to content

Commit

Permalink
SOLR-14341: Move configName into DocCollection (state.json) (apache#17)
Browse files Browse the repository at this point in the history
SolrCloud: move the reference a collection has to a configSet in ZooKeeper from the
  collections/collectionName into its state.json.  For many-collection clusters, this is an
  optimization when the cluster status is fetched.

Co-authored-by: Nazerke Seidan <[email protected]>
Co-authored-by: David Smiley <[email protected]>
  • Loading branch information
3 people authored Jun 25, 2021
1 parent a6d92b1 commit caca827
Show file tree
Hide file tree
Showing 35 changed files with 191 additions and 211 deletions.
4 changes: 4 additions & 0 deletions solr/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,10 @@ Other Changes

* SOLR-15451: SolrSchema (for Parallel SQL) should use PKI principal for internal request to /admin/luke to get table metadata (Timothy Potter)

* SOLR-14341: SolrCloud: move the reference a collection has to a configSet in ZooKeeper from the
collections/collectionName into its state.json. For many-collection clusters, this is an
optimization when the cluster status is fetched. (Nazerke Seidan, David Smiley)

Bug Fixes
---------------------
* SOLR-14546: Fix for a relatively hard to hit issue in OverseerTaskProcessor that could lead to out of order execution
Expand Down
9 changes: 1 addition & 8 deletions solr/core/src/java/org/apache/solr/cloud/ConfigSetCmds.java
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import org.apache.solr.core.ConfigSetProperties;
import org.apache.solr.core.ConfigSetService;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -181,13 +180,7 @@ private static void deleteConfigSet(String configSetName, boolean force, CoreCon
ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader();

for (Map.Entry<String, DocCollection> entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) {
String configName = null;
try {
configName = zkStateReader.readConfigName(entry.getKey());
} catch (KeeperException ex) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
}
String configName = entry.getValue().getConfigName();
if (configSetName.equals(configName))
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -524,7 +524,11 @@ private ClusterState fetchStateForCollection() throws KeeperException, Interrupt
String collectionStatePath = ZkStateReader.getCollectionPath(updater.getCollectionName());
Stat stat = new Stat();
byte[] data = zkStateReader.getZkClient().getData(collectionStatePath, null, stat, true);
ClusterState clusterState = ClusterState.createFromJson(stat.getVersion(), data, Collections.emptySet());

// This factory method can detect a missing configName and supply it by reading it from the old ZK location.
// TODO in Solr 10 remove that factory method
ClusterState clusterState = ClusterState.createFromJsonSupportingLegacyConfigName(
stat.getVersion(), data, Collections.emptySet(), updater.getCollectionName(), zkStateReader.getZkClient());
return clusterState;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,13 +93,9 @@ public SolrResourceLoader createCoreResourceLoader(CoreDescriptor cd) {
}

// The configSet is read from ZK and populated. Ignore CD's pre-existing configSet; only populated in standalone
final String configSetName;
try {
configSetName = zkController.getZkStateReader().readConfigName(colName);
String configSetName = zkController.getClusterState().getCollection(colName).getConfigName();
cd.setConfigSet(configSetName);
} catch (KeeperException ex) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "Trouble resolving configSet for collection " + colName + ": " + ex.getMessage());
}


return new ZkSolrResourceLoader(cd.getInstanceDir(), configSetName, parentLoader.getClassLoader(), zkController);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeP
params.set(CoreAdminParams.CORE_NODE_NAME,
CollectionHandlingUtils.waitToSeeReplicasInState(ccc.getZkStateReader(), ccc.getSolrCloudManager().getTimeSource(), collectionName, Collections.singleton(createReplica.coreName)).get(createReplica.coreName).getName());

String configName = zkStateReader.readConfigName(collectionName);
String configName = coll.getConfigName();
String routeKey = message.getStr(ShardParams._ROUTE_);
String dataDir = message.getStr(CoreAdminParams.DATA_DIR);
String ulogDir = message.getStr(CoreAdminParams.ULOG_DIR);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"ra
String backupName = message.getStr(NAME);
String repo = message.getStr(CoreAdminParams.BACKUP_REPOSITORY);
boolean incremental = message.getBool(CoreAdminParams.BACKUP_INCREMENTAL, true);
String configName = ccc.getZkStateReader().readConfigName(collectionName);
String configName = ccc.getSolrCloudManager().getClusterStateProvider().getCollection(collectionName).getConfigName();

BackupProperties backupProperties = BackupProperties.create(backupName, collectionName,
extCollectionName, configName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
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.CollectionParams;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
Expand All @@ -50,6 +49,7 @@
import org.slf4j.LoggerFactory;

import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CONFIGNAME_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;
Expand All @@ -59,6 +59,7 @@
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.CollectionAdminParams.COLL_CONF;
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;
Expand Down Expand Up @@ -296,14 +297,17 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin

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);
String configName = (String) message.getProperties().get(COLL_CONF);

if (configName != null) {
CollectionHandlingUtils.validateConfigOrThrowSolrException(ccc.getCoreContainer().getConfigSetService(), configName);

CollectionHandlingUtils.createConfNode(ccc.getSolrCloudManager().getDistribStateManager(), configName, collectionName);
new ReloadCollectionCmd(ccc).call(clusterState, new ZkNodeProps(NAME, collectionName), results);
// Back-compatibility reason: update configName in old location
// TODO in Solr 10 this code should go away
String collPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName;
if (ccc.getSolrCloudManager().getDistribStateManager().hasData(collPath)) {
ccc.getSolrCloudManager().getDistribStateManager().setData(collPath, Utils.toJSON(Map.of(ZkStateReader.CONFIGNAME_PROP, configName)), -1);
}
}

if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
Expand All @@ -321,6 +325,12 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin
for (Map.Entry<String, Object> updateEntry : message.getProperties().entrySet()) {
String updateKey = updateEntry.getKey();

// update key from collection.configName to configName;
// actual renaming happens in org.apache.solr.cloud.overseer.CollectionMutator#modifyCollection
if (updateKey.equals(COLL_CONF)) {
updateKey = CONFIGNAME_PROP;
}

if (!updateKey.equals(ZkStateReader.COLLECTION_PROP)
&& !updateKey.equals(Overseer.QUEUE_OPERATION)
&& !updateKey.equals(CommonAdminParams.ASYNC)
Expand All @@ -341,8 +351,8 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin
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)) {
// if switching to/from read-only mode or configName is not null reload the collection
if (message.keySet().contains(ZkStateReader.READ_ONLY) || configName != null) {
new ReloadCollectionCmd(ccc).call(clusterState, new ZkNodeProps(NAME, collectionName), results);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,17 +34,13 @@

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.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;
Expand All @@ -64,7 +60,6 @@
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;
Expand Down Expand Up @@ -307,26 +302,6 @@ static void validateConfigOrThrowSolrException(ConfigSetService configSetService
}
}

/**
* 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(Map.of(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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.ConfigSetService;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.admin.ConfigSetsHandler;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.util.TimeOut;
Expand Down Expand Up @@ -126,9 +127,7 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin
final String async = message.getStr(ASYNC);

ZkStateReader zkStateReader = ccc.getZkStateReader();

// this also creates the collection zk node as a side-effect
CollectionHandlingUtils.createConfNode(ccc.getSolrCloudManager().getDistribStateManager(), configName, collectionName);
message.getProperties().put(COLL_CONF, configName);

Map<String,String> collectionParams = new HashMap<>();
Map<String,Object> collectionProps = message.getProperties();
Expand Down Expand Up @@ -369,8 +368,9 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin
} else {
log.debug("Finished create command on all shards for collection: {}", collectionName);
// Emit a warning about production use of data driven functionality
// Note: isAutoGeneratedConfigSet is always a clone of the _default configset
boolean defaultConfigSetUsed = message.getStr(COLL_CONF) == null ||
message.getStr(COLL_CONF).equals(DEFAULT_CONFIGSET_NAME);
message.getStr(COLL_CONF).equals(DEFAULT_CONFIGSET_NAME) || ConfigSetsHandler.isAutoGeneratedConfigSet(message.getStr(COLL_CONF));
if (defaultConfigSetUsed) {
results.add("warning", "Using _default configset. Data driven schema functionality"
+ " is enabled by default, which is NOT RECOMMENDED for production use. To turn it off:"
Expand Down Expand Up @@ -560,9 +560,10 @@ public static void createCollectionZkNode(DistribStateManager stateManager, Stri
}

collectionProps.remove(ZkStateReader.NUM_SHARDS_PROP); // we don't put numShards in the collections properties
collectionProps.remove(ZkStateReader.CONFIGNAME_PROP); // we don't write configName on a zk collection node

ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
stateManager.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, false);
// create a node
stateManager.makePath(collectionPath);

} catch (KeeperException e) {
//TODO shouldn't the stateManager ensure this does not happen; should throw AlreadyExistsException
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,20 +155,15 @@ public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"ra
}

// delete related config set iff: it is auto generated AND not related to any other collection
String configSetName = zkStateReader.readConfigName(collection);
String configSetName = coll.getConfigName();

if (ConfigSetsHandler.isAutoGeneratedConfigSet(configSetName)) {
boolean configSetIsUsedByOtherCollection = false;

// make sure the configSet is not shared with other collections
// Similar to what happens in: ConfigSetCmds::deleteConfigSet
for (Map.Entry<String, DocCollection> entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) {
String otherConfigSetName = null;
try {
otherConfigSetName = zkStateReader.readConfigName(entry.getKey());
} catch (KeeperException ex) {
// ignore 'no config found' errors
}
String otherConfigSetName = entry.getValue().getConfigName();
if (configSetName.equals(otherConfigSetName)) {
configSetIsUsedByOtherCollection = true;
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ private void migrateKey(ClusterState clusterState, DocCollection sourceCollectio
Replica sourceLeader = zkStateReader.getLeaderRetry(sourceCollection.getName(), sourceSlice.getName(), 10000);

// create a temporary collection with just one node on the shard leader
String configName = zkStateReader.readConfigName(sourceCollection.getName());
String configName = sourceCollection.getConfigName();
Map<String, Object> props = Utils.makeMap(
Overseer.QUEUE_OPERATION, CREATE.toLower(),
NAME, tempSourceCollectionName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ public void call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnin
router = DocRouter.DEFAULT;
}

String configName = message.getStr(ZkStateReader.CONFIGNAME_PROP, ccc.getZkStateReader().readConfigName(collection));
String configName = message.getStr(ZkStateReader.CONFIGNAME_PROP, coll.getConfigName());
String targetCollection;
int seq = tmpCollectionSeq.getAndIncrement();
if (sameTarget) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -109,6 +110,14 @@ public ZkWriteCommand createCollection(ClusterState clusterState, ZkNodeProps me
collectionProps.put("autoCreated", "true");
}

// put configName in props so that it will appear in state.json
final String configName = (String) message.getProperties().get(CollectionAdminParams.COLL_CONF);

if (configName != null) {
collectionProps.put(ZkStateReader.CONFIGNAME_PROP, configName);
}

assert !collectionProps.containsKey(CollectionAdminParams.COLL_CONF);
DocCollection newCollection = new DocCollection(cName, slices, collectionProps, router, -1);

return new ZkWriteCommand(cName, newCollection);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,10 @@
import org.slf4j.LoggerFactory;

import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CONFIGNAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;

public class CollectionMutator {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
Expand Down Expand Up @@ -98,7 +100,7 @@ public ZkWriteCommand deleteShard(final ClusterState clusterState, ZkNodeProps m
public ZkWriteCommand modifyCollection(final ClusterState clusterState, ZkNodeProps message) {
if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
DocCollection coll = clusterState.getCollection(message.getStr(COLLECTION_PROP));
Map<String, Object> m = coll.shallowCopy();
Map<String, Object> props = coll.shallowCopy();
boolean hasAnyOps = false;
PerReplicaStatesOps replicaOps = null;
for (String prop : CollectionAdminRequest.MODIFIABLE_COLLECTION_PROPERTIES) {
Expand All @@ -118,12 +120,17 @@ public ZkWriteCommand modifyCollection(final ClusterState clusterState, ZkNodePr
if (message.containsKey(prop)) {
hasAnyOps = true;
if (message.get(prop) == null) {
m.remove(prop);
props.remove(prop);
} else {
m.put(prop, message.get(prop));
// rename key from collection.configName to configName
if (prop.equals(COLL_CONF)) {
props.put(CONFIGNAME_PROP, message.get(prop));
} else {
props.put(prop, message.get(prop));
}
}
if (prop == REPLICATION_FACTOR) { //SOLR-11676 : keep NRT_REPLICAS and REPLICATION_FACTOR in sync
m.put(NRT_REPLICAS, message.get(REPLICATION_FACTOR));
props.put(NRT_REPLICAS, message.get(REPLICATION_FACTOR));
}
}
}
Expand All @@ -132,9 +139,9 @@ public ZkWriteCommand modifyCollection(final ClusterState clusterState, ZkNodePr
if (prop.startsWith(CollectionAdminRequest.PROPERTY_PREFIX)) {
hasAnyOps = true;
if (message.get(prop) == null) {
m.remove(prop);
props.remove(prop);
} else {
m.put(prop, message.get(prop));
props.put(prop, message.get(prop));
}
}
}
Expand All @@ -143,7 +150,9 @@ public ZkWriteCommand modifyCollection(final ClusterState clusterState, ZkNodePr
return ZkStateWriter.NO_OP;
}

DocCollection collection = new DocCollection(coll.getName(), coll.getSlicesMap(), m, coll.getRouter(), coll.getZNodeVersion());
assert !props.containsKey(COLL_CONF);

DocCollection collection = new DocCollection(coll.getName(), coll.getSlicesMap(), props, coll.getRouter(), coll.getZNodeVersion());
if (replicaOps == null){
return new ZkWriteCommand(coll.getName(), collection);
} else {
Expand Down
Loading

0 comments on commit caca827

Please sign in to comment.